From 858aff396c0c593a37d17a2e8f09fad4a1a5b42e Mon Sep 17 00:00:00 2001 From: Glenn Nethercutt Date: Fri, 3 Oct 2014 23:06:42 -0400 Subject: [PATCH 01/71] move to latest AWS sdk (1.8.11) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 1ad6ccb9338..e915d5a1006 100644 --- a/pom.xml +++ b/pom.xml @@ -130,7 +130,7 @@ com.amazonaws aws-java-sdk - 1.6.0.1 + 1.8.11 javax.mail From 39a7af28d6ffbbfbd8cb2dca85970c93da50c66f Mon Sep 17 00:00:00 2001 From: Glenn Nethercutt Date: Fri, 3 Oct 2014 23:09:38 -0400 Subject: [PATCH 02/71] Custom AWSCredentialsProviderChain for the S3 storage module: supports existing druid config, file, environment variable, system property, profile and (most importantly) instance profile based credential options --- .../storage/s3/S3StorageDruidModule.java | 79 ++++++++++++++++--- 1 file changed, 66 insertions(+), 13 deletions(-) diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java index 3d2434365ae..5ee68e8fb44 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java @@ -19,7 +19,13 @@ package io.druid.storage.s3; +import com.amazonaws.AmazonClientException; import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.AWSCredentialsProviderChain; +import com.amazonaws.auth.EnvironmentVariableCredentialsProvider; +import com.amazonaws.auth.InstanceProfileCredentialsProvider; +import com.amazonaws.auth.SystemPropertiesCredentialsProvider; +import com.amazonaws.auth.profile.ProfileCredentialsProvider; import com.fasterxml.jackson.databind.Module; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; @@ -62,14 +68,18 @@ public class S3StorageDruidModule implements DruidModule binder.bind(S3TaskLogs.class).in(LazySingleton.class); } - @Provides - @LazySingleton - public AWSCredentialsProvider getAWSCredentialsProvider(final AWSCredentialsConfig config) + private class ConfigDrivenAwsCredentialsConfigProvider implements AWSCredentialsProvider { - if (!Strings.isNullOrEmpty(config.getAccessKey()) && !Strings.isNullOrEmpty(config.getSecretKey())) { - return new AWSCredentialsProvider() { - @Override - public com.amazonaws.auth.AWSCredentials getCredentials() { + private AWSCredentialsConfig config; + + public ConfigDrivenAwsCredentialsConfigProvider(AWSCredentialsConfig config) { + this.config = config; + } + + @Override + public com.amazonaws.auth.AWSCredentials getCredentials() + { + if (!Strings.isNullOrEmpty(config.getAccessKey()) && !Strings.isNullOrEmpty(config.getSecretKey())) { return new com.amazonaws.auth.AWSCredentials() { @Override public String getAWSAccessKeyId() { @@ -82,13 +92,56 @@ public class S3StorageDruidModule implements DruidModule } }; } - - @Override - public void refresh() {} - }; - } else { - return new FileSessionCredentialsProvider(config.getFileSessionCredentials()); + throw new AmazonClientException("Unable to load AWS credentials from druid AWSCredentialsConfig"); } + + @Override + public void refresh() {} + } + + private class LazyFileSessionCredentialsProvider implements AWSCredentialsProvider + { + private AWSCredentialsConfig config; + private FileSessionCredentialsProvider provider; + + public LazyFileSessionCredentialsProvider(AWSCredentialsConfig config) { + this.config = config; + } + + private FileSessionCredentialsProvider getUnderlyingProvider() { + if (provider == null) { + synchronized (config) { + if (provider == null) { + provider = new FileSessionCredentialsProvider(config.getFileSessionCredentials()); + } + } + } + return provider; + } + + @Override + public com.amazonaws.auth.AWSCredentials getCredentials() + { + return getUnderlyingProvider().getCredentials(); + } + + @Override + public void refresh() { + getUnderlyingProvider().refresh(); + } + } + + @Provides + @LazySingleton + public AWSCredentialsProvider getAWSCredentialsProvider(final AWSCredentialsConfig config) + { + return new AWSCredentialsProviderChain( + new ConfigDrivenAwsCredentialsConfigProvider(config), + new LazyFileSessionCredentialsProvider(config), + new EnvironmentVariableCredentialsProvider(), + new SystemPropertiesCredentialsProvider(), + new ProfileCredentialsProvider(), + new InstanceProfileCredentialsProvider()); } @Provides From 8bebb24fd5aec1991b4a81116dcea05377afca18 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Fri, 7 Nov 2014 18:05:16 +0530 Subject: [PATCH 03/71] Union Queries move merge to historical --- .../io/druid/timeline/TimelineLookup.java | 44 +++++ .../druid/timeline/UnionTimeLineLookup.java | 54 +++++++ .../timeline/VersionedIntervalTimeline.java | 2 +- .../java/io/druid/query/UnionQueryRunner.java | 26 ++- .../io/druid/query/QueryRunnerTestHelper.java | 51 ++++-- .../TimeSeriesUnionQueryRunnerTest.java | 152 ++++++++++-------- .../druid/query/topn/TopNUnionQueryTest.java | 6 +- .../io/druid/client/BrokerServerView.java | 30 +++- .../druid/client/CachingClusteredClient.java | 3 +- .../io/druid/client/TimelineServerView.java | 3 +- .../segment/realtime/RealtimeManager.java | 28 +++- .../server/ClientQuerySegmentWalker.java | 5 +- .../server/coordination/ServerManager.java | 39 +++-- 13 files changed, 316 insertions(+), 127 deletions(-) create mode 100644 common/src/main/java/io/druid/timeline/TimelineLookup.java create mode 100644 common/src/main/java/io/druid/timeline/UnionTimeLineLookup.java diff --git a/common/src/main/java/io/druid/timeline/TimelineLookup.java b/common/src/main/java/io/druid/timeline/TimelineLookup.java new file mode 100644 index 00000000000..2fc343ac9b2 --- /dev/null +++ b/common/src/main/java/io/druid/timeline/TimelineLookup.java @@ -0,0 +1,44 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.timeline; + +import io.druid.timeline.partition.PartitionHolder; +import org.joda.time.Interval; + +import java.util.List; + + +public interface TimelineLookup +{ + + /** + * Does a lookup for the objects representing the given time interval. Will *only* return + * PartitionHolders that are complete. + * + * @param interval interval to find objects for + * + * @return Holders representing the interval that the objects exist for, PartitionHolders + * are guaranteed to be complete + */ + public List> lookup(Interval interval); + + public PartitionHolder findEntry(Interval interval, VersionType version); + +} diff --git a/common/src/main/java/io/druid/timeline/UnionTimeLineLookup.java b/common/src/main/java/io/druid/timeline/UnionTimeLineLookup.java new file mode 100644 index 00000000000..d5cf8f89f08 --- /dev/null +++ b/common/src/main/java/io/druid/timeline/UnionTimeLineLookup.java @@ -0,0 +1,54 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.timeline; + +import com.google.common.collect.Lists; +import io.druid.timeline.partition.PartitionHolder; +import org.joda.time.Interval; + +import java.util.List; + + +public class UnionTimeLineLookup implements TimelineLookup +{ + Iterable> delegates; + public UnionTimeLineLookup( Iterable> delegates){ + this.delegates = delegates; + } + @Override + public List> lookup(Interval interval) + { + List> rv = Lists.newArrayList(); + for(TimelineLookup delegate : delegates){ + rv.addAll(delegate.lookup(interval)); + } + return rv; + } + + public PartitionHolder findEntry(Interval interval, VersionType version){ + for(TimelineLookup delegate : delegates){ + final PartitionHolder entry = delegate.findEntry(interval, version); + if(entry != null){ + return entry; + } + } + return null; + } +} diff --git a/common/src/main/java/io/druid/timeline/VersionedIntervalTimeline.java b/common/src/main/java/io/druid/timeline/VersionedIntervalTimeline.java index 903480e2dac..1c73782c8d5 100644 --- a/common/src/main/java/io/druid/timeline/VersionedIntervalTimeline.java +++ b/common/src/main/java/io/druid/timeline/VersionedIntervalTimeline.java @@ -58,7 +58,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; * to achieve "atomic" updates. First add new items, then check if those items caused anything to be overshadowed, if * so, remove the overshadowed elements and you have effectively updated your data set without any user impact. */ -public class VersionedIntervalTimeline +public class VersionedIntervalTimeline implements TimelineLookup { private static final Logger log = new Logger(VersionedIntervalTimeline.class); diff --git a/processing/src/main/java/io/druid/query/UnionQueryRunner.java b/processing/src/main/java/io/druid/query/UnionQueryRunner.java index f14bb180f62..09eba3bf5be 100644 --- a/processing/src/main/java/io/druid/query/UnionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/UnionQueryRunner.java @@ -22,49 +22,47 @@ package io.druid.query; import com.google.common.base.Function; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import java.util.List; + public class UnionQueryRunner implements QueryRunner { - private final QueryRunner baseRunner; + private final Iterable> baseRunners; private final QueryToolChest> toolChest; public UnionQueryRunner( - QueryRunner baseRunner, + Iterable> baseRunners, QueryToolChest> toolChest ) { - this.baseRunner = baseRunner; + this.baseRunners = baseRunners; this.toolChest = toolChest; } @Override public Sequence run(final Query query) { - DataSource dataSource = query.getDataSource(); - if (dataSource instanceof UnionDataSource) { return toolChest.mergeSequencesUnordered( Sequences.simple( - Lists.transform( - ((UnionDataSource) dataSource).getDataSources(), - new Function>() + Iterables.transform( + baseRunners, + new Function, Sequence>() { @Override - public Sequence apply(DataSource singleSource) + public Sequence apply(QueryRunner singleRunner) { - return baseRunner.run( - query.withDataSource(singleSource) + return singleRunner.run( + query ); } } ) ) ); - } else { - return baseRunner.run(query); - } } } diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index deeaff563e8..15ba20ddb50 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -20,6 +20,7 @@ package io.druid.query; import com.google.common.base.Function; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.util.concurrent.ListenableFuture; import io.druid.granularity.QueryGranularity; @@ -70,14 +71,14 @@ public class QueryRunnerTestHelper public static final UnionDataSource unionDataSource = new UnionDataSource( Lists.transform( Lists.newArrayList(dataSource, dataSource, dataSource, dataSource), new Function() - { - @Nullable - @Override - public TableDataSource apply(@Nullable String input) - { - return new TableDataSource(input); - } - } + { + @Nullable + @Override + public TableDataSource apply(@Nullable String input) + { + return new TableDataSource(input); + } + } ) ); public static final QueryGranularity dayGran = QueryGranularity.DAY; @@ -214,7 +215,8 @@ public class QueryRunnerTestHelper @SuppressWarnings("unchecked") public static Collection makeUnionQueryRunners( - QueryRunnerFactory factory + QueryRunnerFactory factory, + DataSource unionDataSource ) throws IOException { @@ -224,13 +226,17 @@ public class QueryRunnerTestHelper return Arrays.asList( new Object[][]{ { - makeUnionQueryRunner(factory, new IncrementalIndexSegment(rtIndex, segmentId)) + makeUnionQueryRunner(factory, new IncrementalIndexSegment(rtIndex, segmentId), unionDataSource) }, { - makeUnionQueryRunner(factory, new QueryableIndexSegment(segmentId, mMappedTestIndex)) + makeUnionQueryRunner(factory, new QueryableIndexSegment(segmentId, mMappedTestIndex), unionDataSource) }, { - makeUnionQueryRunner(factory, new QueryableIndexSegment(segmentId, mergedRealtimeIndex)) + makeUnionQueryRunner( + factory, + new QueryableIndexSegment(segmentId, mergedRealtimeIndex), + unionDataSource + ) } } ); @@ -251,17 +257,28 @@ public class QueryRunnerTestHelper } public static QueryRunner makeUnionQueryRunner( - QueryRunnerFactory> factory, - Segment adapter + final QueryRunnerFactory> factory, + final Segment adapter, + final DataSource unionDataSource ) { return new FinalizeResultsQueryRunner( factory.getToolchest().postMergeQueryDecoration( factory.getToolchest().mergeResults( new UnionQueryRunner( - new BySegmentQueryRunner( - segmentId, adapter.getDataInterval().getStart(), - factory.createRunner(adapter) + Iterables.transform( + unionDataSource.getNames(), new Function>() + { + @Nullable + @Override + public QueryRunner apply(@Nullable String input) + { + return new BySegmentQueryRunner( + segmentId, adapter.getDataInterval().getStart(), + factory.createRunner(adapter) + ); + } + } ), factory.getToolchest() ) diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java index 84ace0a798f..9a56a0208e4 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java @@ -19,7 +19,9 @@ package io.druid.query.timeseries; +import com.google.common.base.Function; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; @@ -41,9 +43,11 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import javax.annotation.Nullable; import java.io.IOException; import java.util.Arrays; import java.util.Collection; +import java.util.Iterator; import java.util.List; @RunWith(Parameterized.class) @@ -66,7 +70,8 @@ public class TimeSeriesUnionQueryRunnerTest new TimeseriesQueryQueryToolChest(new QueryConfig()), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + ), + QueryRunnerTestHelper.unionDataSource ); } @@ -139,79 +144,86 @@ public class TimeSeriesUnionQueryRunnerTest QueryToolChest toolChest = new TimeseriesQueryQueryToolChest(new QueryConfig()); QueryRunner mergingrunner = toolChest.mergeResults( new UnionQueryRunner>( - new QueryRunner>() - { - @Override - public Sequence> run(Query> query) - { - if (query.getDataSource().equals(new TableDataSource("ds1"))) { - return Sequences.simple( - Lists.newArrayList( - new Result( - new DateTime("2011-04-02"), - new TimeseriesResultValue( - ImmutableMap.of( - "rows", - 1L, - "idx", - 2L + (Iterable)Arrays.asList( + new QueryRunner>() + { + @Override + public Sequence> run(Query> query) + { + return Sequences.simple( + Lists.newArrayList( + new Result( + new DateTime("2011-04-02"), + new TimeseriesResultValue( + ImmutableMap.of( + "rows", + 1L, + "idx", + 2L + ) + ) + ), + new Result( + new DateTime("2011-04-03"), + new TimeseriesResultValue( + ImmutableMap.of( + "rows", + 3L, + "idx", + 4L + ) + ) + ) + ) + ); + } + }, + new QueryRunner>(){ + + @Override + public Sequence> run(Query> query) + { + { + return Sequences.simple( + Lists.newArrayList( + new Result( + new DateTime("2011-04-01"), + new TimeseriesResultValue( + ImmutableMap.of( + "rows", + 5L, + "idx", + 6L + ) ) - ) - ), - new Result( - new DateTime("2011-04-03"), - new TimeseriesResultValue( - ImmutableMap.of( - "rows", - 3L, - "idx", - 4L + ), + new Result( + new DateTime("2011-04-02"), + new TimeseriesResultValue( + ImmutableMap.of( + "rows", + 7L, + "idx", + 8L + ) + ) + ), + new Result( + new DateTime("2011-04-04"), + new TimeseriesResultValue( + ImmutableMap.of( + "rows", + 9L, + "idx", + 10L + ) ) ) ) - ) - ); - } else { - return Sequences.simple( - Lists.newArrayList( - new Result( - new DateTime("2011-04-01"), - new TimeseriesResultValue( - ImmutableMap.of( - "rows", - 5L, - "idx", - 6L - ) - ) - ), - new Result( - new DateTime("2011-04-02"), - new TimeseriesResultValue( - ImmutableMap.of( - "rows", - 7L, - "idx", - 8L - ) - ) - ), - new Result( - new DateTime("2011-04-04"), - new TimeseriesResultValue( - ImmutableMap.of( - "rows", - 9L, - "idx", - 10L - ) - ) - ) - ) - ); - } - } - }, + ); + } + } + }), toolChest ) ); diff --git a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java index 7dc7b645cad..5034fda9865 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java @@ -67,7 +67,8 @@ public class TopNUnionQueryTest TestQueryRunners.getPool(), new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + ), + QueryRunnerTestHelper.unionDataSource ) ); retVal.addAll( @@ -85,7 +86,8 @@ public class TopNUnionQueryTest ), new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + ), + QueryRunnerTestHelper.unionDataSource ) ); diff --git a/server/src/main/java/io/druid/client/BrokerServerView.java b/server/src/main/java/io/druid/client/BrokerServerView.java index c0403c129e7..76b1207e508 100644 --- a/server/src/main/java/io/druid/client/BrokerServerView.java +++ b/server/src/main/java/io/druid/client/BrokerServerView.java @@ -20,6 +20,8 @@ package io.druid.client; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.client.util.Lists; +import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; @@ -37,10 +39,16 @@ import io.druid.query.QueryToolChestWarehouse; import io.druid.query.QueryWatcher; import io.druid.server.coordination.DruidServerMetadata; import io.druid.timeline.DataSegment; +import io.druid.timeline.TimelineLookup; +import io.druid.timeline.TimelineObjectHolder; +import io.druid.timeline.UnionTimeLineLookup; import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.PartitionChunk; +import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; @@ -245,11 +253,27 @@ public class BrokerServerView implements TimelineServerView @Override - public VersionedIntervalTimeline getTimeline(DataSource dataSource) + public TimelineLookup getTimeline(DataSource dataSource) { - String table = Iterables.getOnlyElement(dataSource.getNames()); + final List tables = dataSource.getNames(); synchronized (lock) { - return timelines.get(table); + if (tables.size() == 1) { + return timelines.get(tables.get(0)); + } else { + return new UnionTimeLineLookup<>( + Iterables.transform( + tables, new Function>() + { + + @Override + public TimelineLookup apply(String input) + { + return timelines.get(input); + } + } + ) + ); + } } } diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index f189e060670..087bc96927d 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -56,6 +56,7 @@ import io.druid.query.aggregation.MetricManipulatorFns; import io.druid.query.spec.MultipleSpecificSegmentSpec; import io.druid.server.coordination.DruidServerMetadata; import io.druid.timeline.DataSegment; +import io.druid.timeline.TimelineLookup; import io.druid.timeline.TimelineObjectHolder; import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.PartitionChunk; @@ -149,7 +150,7 @@ public class CachingClusteredClient implements QueryRunner final Query rewrittenQuery = query.withOverriddenContext(contextBuilder.build()); - VersionedIntervalTimeline timeline = serverView.getTimeline(query.getDataSource()); + TimelineLookup timeline = serverView.getTimeline(query.getDataSource()); if (timeline == null) { return Sequences.empty(); } diff --git a/server/src/main/java/io/druid/client/TimelineServerView.java b/server/src/main/java/io/druid/client/TimelineServerView.java index 0a6a43c8fdb..b3cc8cf6832 100644 --- a/server/src/main/java/io/druid/client/TimelineServerView.java +++ b/server/src/main/java/io/druid/client/TimelineServerView.java @@ -22,12 +22,13 @@ package io.druid.client; import io.druid.client.selector.ServerSelector; import io.druid.query.DataSource; import io.druid.query.QueryRunner; +import io.druid.timeline.TimelineLookup; import io.druid.timeline.VersionedIntervalTimeline; /** */ public interface TimelineServerView extends ServerView { - VersionedIntervalTimeline getTimeline(DataSource dataSource); + TimelineLookup getTimeline(DataSource dataSource); QueryRunner getQueryRunner(DruidServer server); } diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java index a1cfb220972..3998c7503b1 100644 --- a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java +++ b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java @@ -19,6 +19,7 @@ package io.druid.segment.realtime; +import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; @@ -40,6 +41,7 @@ import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChest; import io.druid.query.SegmentDescriptor; +import io.druid.query.UnionQueryRunner; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.realtime.plumber.Plumber; @@ -47,6 +49,7 @@ import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; +import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; import java.util.List; @@ -117,11 +120,28 @@ public class RealtimeManager implements QuerySegmentWalker } @Override - public QueryRunner getQueryRunnerForSegments(Query query, Iterable specs) + public QueryRunner getQueryRunnerForSegments(final Query query, Iterable specs) { - final FireChief chief = chiefs.get(getDataSourceName(query)); - - return chief == null ? new NoopQueryRunner() : chief.getQueryRunner(query); + final List names = query.getDataSource().getNames(); + if (names.size() == 1) { + final FireChief chief = chiefs.get(names.get(0)); + return chief == null ? new NoopQueryRunner() : chief.getQueryRunner(query); + } else { + return new UnionQueryRunner<>( + Iterables.transform( + names, new Function>() + { + @Nullable + @Override + public QueryRunner apply(@Nullable String input) + { + final FireChief chief = chiefs.get(input); + return chief == null ? new NoopQueryRunner() : chief.getQueryRunner(query); + } + } + ), conglomerate.findFactory(query).getToolchest() + ); + } } private String getDataSourceName(Query query) diff --git a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java index 311f0162c5a..c81fa618b2f 100644 --- a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java @@ -82,7 +82,6 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker final FinalizeResultsQueryRunner baseRunner = new FinalizeResultsQueryRunner( toolChest.postMergeQueryDecoration( toolChest.mergeResults( - new UnionQueryRunner( new MetricsEmittingQueryRunner( emitter, new Function, ServiceMetricEvent.Builder>() @@ -94,9 +93,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker } }, toolChest.preMergeQueryDecoration(baseClient) - ).withWaitMeasuredFromNow(), - toolChest - ) + ).withWaitMeasuredFromNow() ) ), toolChest diff --git a/server/src/main/java/io/druid/server/coordination/ServerManager.java b/server/src/main/java/io/druid/server/coordination/ServerManager.java index e79e4a3a978..e5445774e0a 100644 --- a/server/src/main/java/io/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/io/druid/server/coordination/ServerManager.java @@ -33,6 +33,7 @@ import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.client.CachingQueryRunner; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; +import io.druid.client.selector.ServerSelector; import io.druid.collections.CountingMap; import io.druid.guice.annotations.Processing; import io.druid.guice.annotations.Smile; @@ -42,6 +43,7 @@ import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.MetricsEmittingQueryRunner; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryDataSource; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -57,7 +59,9 @@ import io.druid.segment.Segment; import io.druid.segment.loading.SegmentLoader; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; +import io.druid.timeline.TimelineLookup; import io.druid.timeline.TimelineObjectHolder; +import io.druid.timeline.UnionTimeLineLookup; import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.PartitionChunk; import io.druid.timeline.partition.PartitionHolder; @@ -67,6 +71,7 @@ import javax.annotation.Nullable; import java.io.IOException; import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; @@ -163,7 +168,7 @@ public class ServerManager implements QuerySegmentWalker VersionedIntervalTimeline loadedIntervals = dataSources.get(dataSource); if (loadedIntervals == null) { - loadedIntervals = new VersionedIntervalTimeline(Ordering.natural()); + loadedIntervals = new VersionedIntervalTimeline<>(Ordering.natural()); dataSources.put(dataSource, loadedIntervals); } @@ -250,12 +255,11 @@ public class ServerManager implements QuerySegmentWalker final QueryToolChest> toolChest = factory.getToolchest(); DataSource dataSource = query.getDataSource(); - if (!(dataSource instanceof TableDataSource)) { + if (dataSource instanceof QueryDataSource) { throw new UnsupportedOperationException("data source type '" + dataSource.getClass().getName() + "' unsupported"); } - String dataSourceName = getDataSourceName(dataSource); - final VersionedIntervalTimeline timeline = dataSources.get(dataSourceName); + final TimelineLookup timeline = getTimelineLookup(query.getDataSource()); if (timeline == null) { return new NoopQueryRunner(); @@ -302,7 +306,6 @@ public class ServerManager implements QuerySegmentWalker holder.getVersion(), input.getChunkNumber() ) - ); } } @@ -319,9 +322,26 @@ public class ServerManager implements QuerySegmentWalker return new FinalizeResultsQueryRunner(toolChest.mergeResults(factory.mergeRunners(exec, adapters)), toolChest); } - private String getDataSourceName(DataSource dataSource) + private TimelineLookup getTimelineLookup(DataSource dataSource) { - return Iterables.getOnlyElement(dataSource.getNames()); + final List names = dataSource.getNames(); + if(names.size() == 1){ + return dataSources.get(names.get(0)); + } else { + return new UnionTimeLineLookup<>( + Iterables.transform( + names, new Function>() + { + + @Override + public TimelineLookup apply(String input) + { + return dataSources.get(input); + } + } + ) + ); + } } @Override @@ -337,9 +357,7 @@ public class ServerManager implements QuerySegmentWalker final QueryToolChest> toolChest = factory.getToolchest(); - String dataSourceName = getDataSourceName(query.getDataSource()); - - final VersionedIntervalTimeline timeline = dataSources.get(dataSourceName); + final TimelineLookup timeline = getTimelineLookup(query.getDataSource()); if (timeline == null) { return new NoopQueryRunner(); @@ -354,6 +372,7 @@ public class ServerManager implements QuerySegmentWalker @SuppressWarnings("unchecked") public Iterable> apply(SegmentDescriptor input) { + final PartitionHolder entry = timeline.findEntry( input.getInterval(), input.getVersion() ); From fd8eb7742b55685d5d73e998275d9c807515dc1c Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Fri, 7 Nov 2014 23:27:50 +0530 Subject: [PATCH 04/71] handle union query on realtime node --- .../overlord/ThreadPoolTaskRunner.java | 28 +++++++++++++++---- .../indexing/overlord/TaskLifecycleTest.java | 2 +- .../worker/WorkerTaskMonitorTest.java | 3 +- .../java/io/druid/query/UnionQueryRunner.java | 8 +++--- .../io/druid/query/QueryRunnerTestHelper.java | 4 +-- .../segment/realtime/RealtimeManager.java | 6 ++-- 6 files changed, 34 insertions(+), 17 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java index 0b310aa401d..231799d3a31 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java @@ -40,13 +40,16 @@ import io.druid.indexing.common.task.Task; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QuerySegmentWalker; import io.druid.query.SegmentDescriptor; +import io.druid.query.UnionQueryRunner; import org.apache.commons.io.FileUtils; import org.joda.time.Interval; import java.io.File; import java.util.Collection; +import java.util.List; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentSkipListSet; @@ -59,16 +62,18 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker private final TaskToolboxFactory toolboxFactory; private final ListeningExecutorService exec; private final Set runningItems = new ConcurrentSkipListSet<>(); - + private final QueryRunnerFactoryConglomerate conglomerate; private static final EmittingLogger log = new EmittingLogger(ThreadPoolTaskRunner.class); @Inject public ThreadPoolTaskRunner( - TaskToolboxFactory toolboxFactory + TaskToolboxFactory toolboxFactory, + QueryRunnerFactoryConglomerate conglomerate ) { this.toolboxFactory = Preconditions.checkNotNull(toolboxFactory, "toolboxFactory"); this.exec = MoreExecutors.listeningDecorator(Execs.singleThreaded("task-runner-%d")); + this.conglomerate = conglomerate; } @LifecycleStop @@ -152,8 +157,10 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker private QueryRunner getQueryRunnerImpl(Query query) { - QueryRunner queryRunner = null; - final String queryDataSource = Iterables.getOnlyElement(query.getDataSource().getNames()); + final List dataSources = query.getDataSource().getNames(); + List runners = Lists.newArrayList(); + for(String queryDataSource : dataSources) { + QueryRunner queryRunner = null; for (final ThreadPoolTaskRunnerWorkItem taskRunnerWorkItem : ImmutableList.copyOf(runningItems)) { final Task task = taskRunnerWorkItem.getTask(); @@ -170,9 +177,18 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker } } } + } + if(queryRunner!= null) { + runners.add(queryRunner); + } + } + if(runners.size() == 0){ + return new NoopQueryRunner(); + } else if (runners.size() == 1){ + return runners.get(0); + } else { + return new UnionQueryRunner<>(runners, conglomerate.findFactory(query).getToolchest()); } - - return queryRunner == null ? new NoopQueryRunner() : queryRunner; } private static class ThreadPoolTaskRunnerWorkItem extends TaskRunnerWorkItem diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 44bbb4c8f6d..1a3c9cec42f 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -208,7 +208,7 @@ public class TaskLifecycleTest ), new DefaultObjectMapper() ); - tr = new ThreadPoolTaskRunner(tb); + tr = new ThreadPoolTaskRunner(tb, null); tq = new TaskQueue(tqc, ts, tr, tac, tl, emitter); tq.start(); } diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index 11756824c34..478a77494a3 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -180,7 +180,8 @@ public class WorkerTaskMonitorTest } ) ), jsonMapper - ) + ), + null ), new WorkerConfig().setCapacity(1) ); diff --git a/processing/src/main/java/io/druid/query/UnionQueryRunner.java b/processing/src/main/java/io/druid/query/UnionQueryRunner.java index 09eba3bf5be..acd07e67404 100644 --- a/processing/src/main/java/io/druid/query/UnionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/UnionQueryRunner.java @@ -31,11 +31,11 @@ import java.util.List; public class UnionQueryRunner implements QueryRunner { - private final Iterable> baseRunners; + private final Iterable baseRunners; private final QueryToolChest> toolChest; public UnionQueryRunner( - Iterable> baseRunners, + Iterable baseRunners, QueryToolChest> toolChest ) { @@ -50,10 +50,10 @@ public class UnionQueryRunner implements QueryRunner Sequences.simple( Iterables.transform( baseRunners, - new Function, Sequence>() + new Function>() { @Override - public Sequence apply(QueryRunner singleRunner) + public Sequence apply(QueryRunner singleRunner) { return singleRunner.run( query diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index 15ba20ddb50..2163dc12f72 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -267,11 +267,11 @@ public class QueryRunnerTestHelper factory.getToolchest().mergeResults( new UnionQueryRunner( Iterables.transform( - unionDataSource.getNames(), new Function>() + unionDataSource.getNames(), new Function() { @Nullable @Override - public QueryRunner apply(@Nullable String input) + public QueryRunner apply(@Nullable String input) { return new BySegmentQueryRunner( segmentId, adapter.getDataInterval().getStart(), diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java index 3998c7503b1..c8a6fb30668 100644 --- a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java +++ b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java @@ -125,18 +125,18 @@ public class RealtimeManager implements QuerySegmentWalker final List names = query.getDataSource().getNames(); if (names.size() == 1) { final FireChief chief = chiefs.get(names.get(0)); - return chief == null ? new NoopQueryRunner() : chief.getQueryRunner(query); + return chief == null ? new NoopQueryRunner() : chief.getQueryRunner(query); } else { return new UnionQueryRunner<>( Iterables.transform( - names, new Function>() + names, new Function() { @Nullable @Override public QueryRunner apply(@Nullable String input) { final FireChief chief = chiefs.get(input); - return chief == null ? new NoopQueryRunner() : chief.getQueryRunner(query); + return chief == null ? new NoopQueryRunner() : chief.getQueryRunner(query); } } ), conglomerate.findFactory(query).getToolchest() From e30ac109962a82dbfe5757287052f38a769081fb Mon Sep 17 00:00:00 2001 From: Glenn Nethercutt Date: Tue, 11 Nov 2014 08:22:00 -0500 Subject: [PATCH 05/71] make the inner provider classes static --- .../main/java/io/druid/storage/s3/S3StorageDruidModule.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java index 5ee68e8fb44..792136ae8b9 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java @@ -68,7 +68,7 @@ public class S3StorageDruidModule implements DruidModule binder.bind(S3TaskLogs.class).in(LazySingleton.class); } - private class ConfigDrivenAwsCredentialsConfigProvider implements AWSCredentialsProvider + private static class ConfigDrivenAwsCredentialsConfigProvider implements AWSCredentialsProvider { private AWSCredentialsConfig config; @@ -99,7 +99,7 @@ public class S3StorageDruidModule implements DruidModule public void refresh() {} } - private class LazyFileSessionCredentialsProvider implements AWSCredentialsProvider + private static class LazyFileSessionCredentialsProvider implements AWSCredentialsProvider { private AWSCredentialsConfig config; private FileSessionCredentialsProvider provider; From ad1dd161e784ca9fb9476403ec6fefb03895922b Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Mon, 17 Nov 2014 17:44:26 +0530 Subject: [PATCH 06/71] formatting --- .../overlord/ThreadPoolTaskRunner.java | 57 +++++++++---------- 1 file changed, 28 insertions(+), 29 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java index 231799d3a31..72e3660a538 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java @@ -22,7 +22,6 @@ package io.druid.indexing.overlord; import com.google.api.client.repackaged.com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; @@ -91,19 +90,19 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker runningItems.add(taskRunnerWorkItem); Futures.addCallback( statusFuture, new FutureCallback() - { - @Override - public void onSuccess(TaskStatus result) - { - runningItems.remove(taskRunnerWorkItem); - } + { + @Override + public void onSuccess(TaskStatus result) + { + runningItems.remove(taskRunnerWorkItem); + } - @Override - public void onFailure(Throwable t) - { - runningItems.remove(taskRunnerWorkItem); - } - } + @Override + public void onFailure(Throwable t) + { + runningItems.remove(taskRunnerWorkItem); + } + } ); return statusFuture; @@ -159,32 +158,32 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker { final List dataSources = query.getDataSource().getNames(); List runners = Lists.newArrayList(); - for(String queryDataSource : dataSources) { + for (String queryDataSource : dataSources) { QueryRunner queryRunner = null; - for (final ThreadPoolTaskRunnerWorkItem taskRunnerWorkItem : ImmutableList.copyOf(runningItems)) { - final Task task = taskRunnerWorkItem.getTask(); - if (task.getDataSource().equals(queryDataSource)) { - final QueryRunner taskQueryRunner = task.getQueryRunner(query); + for (final ThreadPoolTaskRunnerWorkItem taskRunnerWorkItem : ImmutableList.copyOf(runningItems)) { + final Task task = taskRunnerWorkItem.getTask(); + if (task.getDataSource().equals(queryDataSource)) { + final QueryRunner taskQueryRunner = task.getQueryRunner(query); - if (taskQueryRunner != null) { - if (queryRunner == null) { - queryRunner = taskQueryRunner; - } else { - log.makeAlert("Found too many query runners for datasource") - .addData("dataSource", queryDataSource) - .emit(); + if (taskQueryRunner != null) { + if (queryRunner == null) { + queryRunner = taskQueryRunner; + } else { + log.makeAlert("Found too many query runners for datasource") + .addData("dataSource", queryDataSource) + .emit(); + } } } } - } - if(queryRunner!= null) { + if (queryRunner != null) { runners.add(queryRunner); } } - if(runners.size() == 0){ + if (runners.size() == 0) { return new NoopQueryRunner(); - } else if (runners.size() == 1){ + } else if (runners.size() == 1) { return runners.get(0); } else { return new UnionQueryRunner<>(runners, conglomerate.findFactory(query).getToolchest()); From bd9bbf396c164f095d724d60c915b771c2e8cfa0 Mon Sep 17 00:00:00 2001 From: Maarten Rijke Date: Mon, 8 Dec 2014 20:11:58 +0100 Subject: [PATCH 07/71] Fix NullPointerException in PartialDimExtractionFn by explicity checking for dimValue == null --- .../java/io/druid/query/extraction/PartialDimExtractionFn.java | 1 + .../query/extraction/extraction/PartialDimExtractionFnTest.java | 1 + 2 files changed, 2 insertions(+) diff --git a/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java index 323c7989387..7e2bd294378 100644 --- a/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java @@ -58,6 +58,7 @@ public class PartialDimExtractionFn implements DimExtractionFn @Override public String apply(String dimValue) { + if (dimValue == null) return null; Matcher matcher = pattern.matcher(dimValue); return matcher.find() ? dimValue : null; } diff --git a/processing/src/test/java/io/druid/query/extraction/extraction/PartialDimExtractionFnTest.java b/processing/src/test/java/io/druid/query/extraction/extraction/PartialDimExtractionFnTest.java index 8c9cb6b093a..deeb7ac05ce 100644 --- a/processing/src/test/java/io/druid/query/extraction/extraction/PartialDimExtractionFnTest.java +++ b/processing/src/test/java/io/druid/query/extraction/extraction/PartialDimExtractionFnTest.java @@ -41,6 +41,7 @@ public class PartialDimExtractionFnTest "Vancouver", "Pretoria", "Wellington", + null, "Ontario" }; From 90670a9c7e32b0aa83f829d7064c7efdf0dbd09d Mon Sep 17 00:00:00 2001 From: Maarten Rijke Date: Mon, 8 Dec 2014 20:11:58 +0100 Subject: [PATCH 08/71] Fix NullPointerException in PartialDimExtractionFn by explicity checking for dimValue == null, attempt 2 --- .../java/io/druid/query/extraction/PartialDimExtractionFn.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java index 7e2bd294378..34c026f3db3 100644 --- a/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java @@ -58,7 +58,7 @@ public class PartialDimExtractionFn implements DimExtractionFn @Override public String apply(String dimValue) { - if (dimValue == null) return null; + dimValue = (dimValue == null) ? "" : dimValue; Matcher matcher = pattern.matcher(dimValue); return matcher.find() ? dimValue : null; } From 0de56efe5cea746f72cce77155d135ddf565fe69 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 10 Dec 2014 17:03:33 -0800 Subject: [PATCH 09/71] fix gzip compression - fixes compression not working for POST requests - router now forwards raw bytes instead of decompressing - cleanded up router servlet initialization - add test for gzip on get and post methods - use Jersey annotation when possible in QueryResource --- .../guice/http/JettyHttpClientModule.java | 4 ++ .../java/io/druid/server/QueryResource.java | 24 +++++--- .../BaseJettyServerInitializer.java | 47 +++++++++++++++ .../server/initialization/JettyTest.java | 57 ++++++++++++++++--- .../main/java/io/druid/cli/CliOverlord.java | 6 +- .../CoordinatorJettyServerInitializer.java | 9 ++- .../MiddleManagerJettyServerInitializer.java | 7 +-- .../cli/QueryJettyServerInitializer.java | 11 ++-- .../cli/RouterJettyServerInitializer.java | 23 ++++---- 9 files changed, 140 insertions(+), 48 deletions(-) create mode 100644 server/src/main/java/io/druid/server/initialization/BaseJettyServerInitializer.java diff --git a/server/src/main/java/io/druid/guice/http/JettyHttpClientModule.java b/server/src/main/java/io/druid/guice/http/JettyHttpClientModule.java index add7927bf42..04b3fc29caf 100644 --- a/server/src/main/java/io/druid/guice/http/JettyHttpClientModule.java +++ b/server/src/main/java/io/druid/guice/http/JettyHttpClientModule.java @@ -127,6 +127,10 @@ public class JettyHttpClientModule implements Module public void start() throws Exception { httpClient.start(); + + // forwards raw bytes, don't decode gzip + // decoders are populated on start, so this has to be done after start() is called + httpClient.getContentDecoderFactories().clear(); } @Override diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index a928e8142c3..318c0d4df15 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -48,18 +48,22 @@ import org.joda.time.DateTime; import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; +import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; import javax.ws.rs.GET; +import javax.ws.rs.HeaderParam; import javax.ws.rs.POST; import javax.ws.rs.Path; import javax.ws.rs.PathParam; import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import javax.ws.rs.core.StreamingOutput; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; import java.util.Map; import java.util.UUID; @@ -116,26 +120,30 @@ public class QueryResource } @POST + @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) public Response doPost( - @Context HttpServletRequest req, - @Context final HttpServletResponse resp - ) throws ServletException, IOException + InputStream in, + @QueryParam("pretty") String pretty, + @Context HttpServletRequest req // used only to get request content-type and remote address + ) throws IOException { final long start = System.currentTimeMillis(); Query query = null; byte[] requestQuery = null; String queryId = null; - final boolean isSmile = SmileMediaTypes.APPLICATION_JACKSON_SMILE.equals(req.getContentType()) || APPLICATION_SMILE.equals(req.getContentType()); + final String reqContentType = req.getContentType(); + final boolean isSmile = SmileMediaTypes.APPLICATION_JACKSON_SMILE.equals(reqContentType) || APPLICATION_SMILE.equals(reqContentType); final String contentType = isSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON; ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper; - final ObjectWriter jsonWriter = req.getParameter("pretty") == null - ? objectMapper.writer() - : objectMapper.writerWithDefaultPrettyPrinter(); + final ObjectWriter jsonWriter = pretty != null + ? objectMapper.writerWithDefaultPrettyPrinter() + : objectMapper.writer(); try { - query = objectMapper.readValue(req.getInputStream(), Query.class); + query = objectMapper.readValue(in, Query.class); queryId = query.getId(); if (queryId == null) { queryId = UUID.randomUUID().toString(); diff --git a/server/src/main/java/io/druid/server/initialization/BaseJettyServerInitializer.java b/server/src/main/java/io/druid/server/initialization/BaseJettyServerInitializer.java new file mode 100644 index 00000000000..22c3b2a7d7f --- /dev/null +++ b/server/src/main/java/io/druid/server/initialization/BaseJettyServerInitializer.java @@ -0,0 +1,47 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.server.initialization; + +import com.google.common.base.Joiner; +import org.eclipse.jetty.servlet.FilterHolder; +import org.eclipse.jetty.servlets.AsyncGzipFilter; +import org.eclipse.jetty.servlets.GzipFilter; + +import javax.ws.rs.HttpMethod; + +public abstract class BaseJettyServerInitializer implements JettyServerInitializer +{ + + public static final String GZIP_METHODS = Joiner.on(",").join(HttpMethod.GET, HttpMethod.POST); + + public FilterHolder defaultGzipFilterHolder() { + final FilterHolder gzipFilterHolder = new FilterHolder(GzipFilter.class); + gzipFilterHolder.setInitParameter("minGzipSize", "0"); + gzipFilterHolder.setInitParameter("methods", GZIP_METHODS); + return gzipFilterHolder; + } + + public FilterHolder defaultAsyncGzipFilterHolder() { + final FilterHolder gzipFilterHolder = new FilterHolder(AsyncGzipFilter.class); + gzipFilterHolder.setInitParameter("minGzipSize", "0"); + gzipFilterHolder.setInitParameter("methods", GZIP_METHODS); + return gzipFilterHolder; + } +} diff --git a/server/src/test/java/io/druid/server/initialization/JettyTest.java b/server/src/test/java/io/druid/server/initialization/JettyTest.java index 917a9abfdb0..80a7b7cd935 100644 --- a/server/src/test/java/io/druid/server/initialization/JettyTest.java +++ b/server/src/test/java/io/druid/server/initialization/JettyTest.java @@ -45,12 +45,10 @@ import io.druid.server.DruidNode; import org.apache.commons.io.IOUtils; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerList; import org.eclipse.jetty.servlet.DefaultServlet; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.servlets.GzipFilter; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -60,6 +58,7 @@ import org.junit.Test; import javax.servlet.ServletOutputStream; import javax.servlet.http.HttpServletResponse; import javax.ws.rs.GET; +import javax.ws.rs.POST; import javax.ws.rs.Path; import javax.ws.rs.Produces; import javax.ws.rs.core.Context; @@ -68,6 +67,7 @@ import javax.ws.rs.core.Response; import java.io.IOException; import java.io.InputStream; import java.io.StringWriter; +import java.net.HttpURLConnection; import java.net.URL; import java.nio.charset.Charset; import java.util.Random; @@ -81,6 +81,7 @@ public class JettyTest { private Lifecycle lifecycle; private HttpClient client; + private int port = -1; public static void setProperties() { @@ -101,16 +102,20 @@ public class JettyTest public void configure(Binder binder) { JsonConfigProvider.bindInstance( - binder, Key.get(DruidNode.class, Self.class), new DruidNode("test", "localhost", 9999) + binder, Key.get(DruidNode.class, Self.class), new DruidNode("test", "localhost", null) ); binder.bind(JettyServerInitializer.class).to(JettyServerInit.class).in(LazySingleton.class); Jerseys.addResource(binder, SlowResource.class); Jerseys.addResource(binder, ExceptionResource.class); + Jerseys.addResource(binder, DefaultResource.class); LifecycleModule.register(binder, Server.class); } } ) ); + final DruidNode node = injector.getInstance(Key.get(DruidNode.class, Self.class)); + port = node.getPort(); + lifecycle = injector.getInstance(Lifecycle.class); lifecycle.start(); ClientHolder holder = injector.getInstance(ClientHolder.class); @@ -142,7 +147,7 @@ public class JettyTest long startTime2 = 0; try { ListenableFuture go = - client.get(new URL("http://localhost:9999/slow/hello")) + client.get(new URL("http://localhost:" + port + "/slow/hello")) .go(new StatusResponseHandler(Charset.defaultCharset())); startTime2 = System.currentTimeMillis(); go.get(); @@ -173,6 +178,21 @@ public class JettyTest latch.await(); } + @Test + public void testGzipCompression() throws Exception + { + final URL url = new URL("http://localhost:" + port + "/default"); + final HttpURLConnection get = (HttpURLConnection) url.openConnection(); + get.setRequestProperty("Accept-Encoding", "gzip"); + Assert.assertEquals("gzip", get.getContentEncoding()); + + final HttpURLConnection post = (HttpURLConnection) url.openConnection(); + post.setRequestProperty("Accept-Encoding", "gzip"); + post.setRequestMethod("POST"); + + Assert.assertEquals("gzip", post.getContentEncoding()); + } + // Tests that threads are not stuck when partial chunk is not finalized // https://bugs.eclipse.org/bugs/show_bug.cgi?id=424107 @Test @@ -181,7 +201,7 @@ public class JettyTest public void testChunkNotFinalized() throws Exception { ListenableFuture go = - client.get(new URL("http://localhost:9999/exception/exception")) + client.get(new URL("http://localhost:" + port + "/exception/exception")) .go(new InputStreamResponseHandler()); try { StringWriter writer = new StringWriter(); @@ -207,7 +227,7 @@ public class JettyTest try { ListenableFuture go = client.get( new URL( - "http://localhost:9999/exception/exception" + "http://localhost:" + port + "/exception/exception" ) ) @@ -251,7 +271,7 @@ public class JettyTest } } - public static class JettyServerInit implements JettyServerInitializer + public static class JettyServerInit extends BaseJettyServerInitializer { @Override @@ -259,11 +279,11 @@ public class JettyTest { final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - root.addFilter(GzipFilter.class, "/*", null); + root.addFilter(defaultGzipFilterHolder(), "/*", null); root.addFilter(GuiceFilter.class, "/*", null); final HandlerList handlerList = new HandlerList(); - handlerList.setHandlers(new Handler[]{root, new DefaultHandler()}); + handlerList.setHandlers(new Handler[]{root}); server.setHandler(handlerList); } } @@ -289,6 +309,25 @@ public class JettyTest } } + @Path("/default") + public static class DefaultResource + { + @GET + @Produces(MediaType.APPLICATION_JSON) + public Response get() + { + return Response.ok("hello").build(); + } + + @POST + @Produces(MediaType.APPLICATION_JSON) + public Response post() + { + return Response.ok("hello").build(); + } + + } + @Path("/exception") public static class ExceptionResource { diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index c0be380ac14..a5351dd8968 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -74,6 +74,7 @@ import io.druid.indexing.worker.config.WorkerConfig; import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.server.http.RedirectFilter; import io.druid.server.http.RedirectInfo; +import io.druid.server.initialization.BaseJettyServerInitializer; import io.druid.server.initialization.JettyServerInitializer; import io.druid.tasklogs.TaskLogStreamer; import io.druid.tasklogs.TaskLogs; @@ -84,7 +85,6 @@ import org.eclipse.jetty.servlet.DefaultServlet; import org.eclipse.jetty.servlet.FilterHolder; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.servlets.GzipFilter; import org.eclipse.jetty.util.resource.ResourceCollection; import java.util.List; @@ -221,7 +221,7 @@ public class CliOverlord extends ServerRunnable /** */ - private static class OverlordJettyServerInitializer implements JettyServerInitializer + private static class OverlordJettyServerInitializer extends BaseJettyServerInitializer { @Override public void initialize(Server server, Injector injector) @@ -239,7 +239,7 @@ public class CliOverlord extends ServerRunnable } ) ); - root.addFilter(GzipFilter.class, "/*", null); + root.addFilter(defaultGzipFilterHolder(), "/*", null); // /status should not redirect, so add first root.addFilter(GuiceFilter.class, "/status/*", null); diff --git a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java index 9b3a69252cc..5316ac7841f 100644 --- a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java @@ -24,7 +24,7 @@ import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; import io.druid.server.coordinator.DruidCoordinatorConfig; import io.druid.server.http.RedirectFilter; -import io.druid.server.initialization.JettyServerInitializer; +import io.druid.server.initialization.BaseJettyServerInitializer; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.HandlerList; @@ -32,12 +32,11 @@ import org.eclipse.jetty.servlet.DefaultServlet; import org.eclipse.jetty.servlet.FilterHolder; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.servlets.GzipFilter; import org.eclipse.jetty.util.resource.Resource; /** */ -class CoordinatorJettyServerInitializer implements JettyServerInitializer +class CoordinatorJettyServerInitializer extends BaseJettyServerInitializer { private final DruidCoordinatorConfig config; @@ -60,7 +59,7 @@ class CoordinatorJettyServerInitializer implements JettyServerInitializer } else { root.setResourceBase(config.getConsoleStatic()); } - root.addFilter(GzipFilter.class, "/*", null); + root.addFilter(defaultGzipFilterHolder(), "/*", null); // /status should not redirect, so add first root.addFilter(GuiceFilter.class, "/status/*", null); @@ -68,8 +67,8 @@ class CoordinatorJettyServerInitializer implements JettyServerInitializer // redirect anything other than status to the current lead root.addFilter(new FilterHolder(injector.getInstance(RedirectFilter.class)), "/*", null); - // Can't use '/*' here because of Guice and Jetty static content conflicts // The coordinator really needs a standarized api path + // Can't use '/*' here because of Guice and Jetty static content conflicts root.addFilter(GuiceFilter.class, "/info/*", null); root.addFilter(GuiceFilter.class, "/druid/coordinator/*", null); // this will be removed in the next major release diff --git a/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java b/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java index 965ee7a2be0..1c6bfa33638 100644 --- a/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java @@ -21,7 +21,7 @@ package io.druid.cli; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; -import io.druid.server.initialization.JettyServerInitializer; +import io.druid.server.initialization.BaseJettyServerInitializer; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.DefaultHandler; @@ -29,18 +29,17 @@ import org.eclipse.jetty.server.handler.HandlerList; import org.eclipse.jetty.servlet.DefaultServlet; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.servlets.GzipFilter; /** */ -class MiddleManagerJettyServerInitializer implements JettyServerInitializer +class MiddleManagerJettyServerInitializer extends BaseJettyServerInitializer { @Override public void initialize(Server server, Injector injector) { final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - root.addFilter(GzipFilter.class, "/*", null); + root.addFilter(defaultGzipFilterHolder(), "/*", null); root.addFilter(GuiceFilter.class, "/*", null); final HandlerList handlerList = new HandlerList(); diff --git a/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java b/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java index 34039e2cb1c..f72994a5299 100644 --- a/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java @@ -21,30 +21,29 @@ package io.druid.cli; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; -import io.druid.server.initialization.JettyServerInitializer; +import io.druid.server.initialization.BaseJettyServerInitializer; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerList; import org.eclipse.jetty.servlet.DefaultServlet; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.servlets.GzipFilter; /** */ -public class QueryJettyServerInitializer implements JettyServerInitializer +public class QueryJettyServerInitializer extends BaseJettyServerInitializer { @Override public void initialize(Server server, Injector injector) { final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - root.addFilter(GzipFilter.class, "/*", null); + root.addFilter(defaultGzipFilterHolder(), "/*", null); + root.addFilter(GuiceFilter.class, "/*", null); final HandlerList handlerList = new HandlerList(); - handlerList.setHandlers(new Handler[]{root, new DefaultHandler()}); + handlerList.setHandlers(new Handler[]{root}); server.setHandler(handlerList); } } diff --git a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java index 72db25e758a..6f58d5e3687 100644 --- a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java @@ -27,23 +27,21 @@ import com.metamx.emitter.service.ServiceEmitter; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.server.AsyncQueryForwardingServlet; -import io.druid.server.initialization.JettyServerInitializer; +import io.druid.server.initialization.BaseJettyServerInitializer; import io.druid.server.log.RequestLogger; import io.druid.server.router.QueryHostFinder; import io.druid.server.router.Router; import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerList; import org.eclipse.jetty.servlet.DefaultServlet; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.servlets.AsyncGzipFilter; /** */ -public class RouterJettyServerInitializer implements JettyServerInitializer +public class RouterJettyServerInitializer extends BaseJettyServerInitializer { private final ObjectMapper jsonMapper; private final ObjectMapper smileMapper; @@ -73,8 +71,10 @@ public class RouterJettyServerInitializer implements JettyServerInitializer @Override public void initialize(Server server, Injector injector) { - final ServletContextHandler queries = new ServletContextHandler(ServletContextHandler.SESSIONS); - queries.addServlet( + final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); + + root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); + root.addServlet( new ServletHolder( new AsyncQueryForwardingServlet( jsonMapper, @@ -86,15 +86,12 @@ public class RouterJettyServerInitializer implements JettyServerInitializer ) ), "/druid/v2/*" ); - queries.addFilter(AsyncGzipFilter.class, "/druid/v2/*", null); - queries.addFilter(GuiceFilter.class, "/status/*", null); - - final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); - root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - root.addFilter(GuiceFilter.class, "/*", null); + root.addFilter(defaultAsyncGzipFilterHolder(), "/*", null); + // Can't use '/*' here because of Guice conflicts with AsyncQueryForwardingServlet path + root.addFilter(GuiceFilter.class, "/status/*", null); final HandlerList handlerList = new HandlerList(); - handlerList.setHandlers(new Handler[]{queries, root, new DefaultHandler()}); + handlerList.setHandlers(new Handler[]{root}); server.setHandler(handlerList); } } From eb661d3130a393340ac4fcf78a09914bc8037ef4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 10 Dec 2014 17:35:05 -0800 Subject: [PATCH 10/71] /status replaces /health --- server/src/main/java/io/druid/server/StatusResource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/server/StatusResource.java b/server/src/main/java/io/druid/server/StatusResource.java index 10fccf383ab..d21e7c88245 100644 --- a/server/src/main/java/io/druid/server/StatusResource.java +++ b/server/src/main/java/io/druid/server/StatusResource.java @@ -34,7 +34,7 @@ import java.util.List; /** */ -@Path("/{a:status|health}") +@Path("/status") public class StatusResource { @GET From 3abf348d2ca7d8e0f1f4d74acd90242e7ba468d0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 10 Dec 2014 17:32:34 -0800 Subject: [PATCH 11/71] fix router discovering brokers with default names --- .../io/druid/server/router/TieredBrokerHostSelector.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java b/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java index 9e5daa3bc62..1f3cbddad6a 100644 --- a/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java +++ b/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java @@ -184,7 +184,7 @@ public class TieredBrokerHostSelector implements HostSelector brokerServiceName = tierConfig.getDefaultBrokerServiceName(); } - ServerDiscoverySelector retVal = selectorMap.get(CuratorServiceUtils.makeCanonicalServiceName(brokerServiceName)); + ServerDiscoverySelector retVal = selectorMap.get(brokerServiceName); if (retVal == null) { log.error( @@ -192,7 +192,7 @@ public class TieredBrokerHostSelector implements HostSelector brokerServiceName, tierConfig.getDefaultBrokerServiceName() ); - retVal = selectorMap.get(CuratorServiceUtils.makeCanonicalServiceName(tierConfig.getDefaultBrokerServiceName())); + retVal = selectorMap.get(tierConfig.getDefaultBrokerServiceName()); } return new Pair<>(brokerServiceName, retVal); @@ -201,7 +201,7 @@ public class TieredBrokerHostSelector implements HostSelector public Pair getDefaultLookup() { final String brokerServiceName = tierConfig.getDefaultBrokerServiceName(); - final ServerDiscoverySelector retVal = selectorMap.get(CuratorServiceUtils.makeCanonicalServiceName(brokerServiceName)); + final ServerDiscoverySelector retVal = selectorMap.get(brokerServiceName); return new Pair<>(brokerServiceName, retVal); } } From 352480ac77e949a84bab5e8deb0cad74e6a08d5e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 11 Dec 2014 15:18:34 -0800 Subject: [PATCH 12/71] make method protected to prevent accidental usage - add documentation as to why --- .../curator/discovery/CuratorServiceUtils.java | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/curator/discovery/CuratorServiceUtils.java b/server/src/main/java/io/druid/curator/discovery/CuratorServiceUtils.java index ad7781622a8..4f23fed5ac1 100644 --- a/server/src/main/java/io/druid/curator/discovery/CuratorServiceUtils.java +++ b/server/src/main/java/io/druid/curator/discovery/CuratorServiceUtils.java @@ -21,7 +21,22 @@ package io.druid.curator.discovery; public class CuratorServiceUtils { - public static String makeCanonicalServiceName(String serviceName) { + /** + * Replacing '/' with ':' in service names makes it easier to provide an HTTP interface using + * curator-x-discovery-server + * + * This method is marked protected because it should never be used outside of the io.druid.curator.discovery + * packakge. If you are tempted to use this method anywhere else you are most likely doing something wrong. + * Mapping the actual service name to the name used within curator should be left to {@link CuratorServiceAnnouncer} + * and {@link ServerDiscoveryFactory} + * + * @see io.druid.curator.discovery.CuratorServiceAnnouncer + * @see io.druid.curator.discovery.ServerDiscoveryFactory + * + * @param serviceName + * @return + */ + protected static String makeCanonicalServiceName(String serviceName) { return serviceName.replaceAll("/", ":"); } } From 818f85e745125157fede9ca44233e5eb99e6d072 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 11 Dec 2014 15:52:26 -0800 Subject: [PATCH 13/71] rewrite config docs --- docs/content/Broker-Config.md | 87 +++++-- docs/content/Configuration.md | 298 ++++++++---------------- docs/content/Coordinator-Config.md | 20 +- docs/content/Historical-Config.md | 94 +++++++- docs/content/Indexing-Service-Config.md | 155 ++++++++++-- docs/content/Ingestion-FAQ.md | 10 +- docs/content/Realtime-Config.md | 136 ++++------- 7 files changed, 452 insertions(+), 348 deletions(-) diff --git a/docs/content/Broker-Config.md b/docs/content/Broker-Config.md index 45237be311e..1898c8f5bc2 100644 --- a/docs/content/Broker-Config.md +++ b/docs/content/Broker-Config.md @@ -8,34 +8,93 @@ For general Broker Node information, see [here](Broker.html). Runtime Configuration --------------------- -The broker module uses several of the default modules in [Configuration](Configuration.html) and has the following set of configurations as well: +The broker node uses several of the global configs in [Configuration](Configuration.html) and has the following set of configurations as well: + +### Node Configs + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|none| +|`druid.port`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|none| +|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|none| + +### Query Configs + +#### Query Prioritization |Property|Possible Values|Description|Default| |--------|---------------|-----------|-------| |`druid.broker.balancer.type`|`random`, `connectionCount`|Determines how the broker balances connections to historical nodes. `random` choose randomly, `connectionCount` picks the node with the fewest number of active connections to|`random`| |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`| |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None| -|`druid.broker.cache.type`|`local`, `memcached`|The type of cache to use for queries.|`local`| -|`druid.broker.cache.unCacheable`|All druid query types|All query types to not cache.|["groupBy", "select"]| -|`druid.broker.cache.numBackgroundThreads`|Non-negative integer|Number of background threads in the thread pool to use for eventual-consistency caching results if caching is used. It is recommended to set this value greater or equal to the number of processing threads. To force caching to execute in the same thread as the query (query results are blocked on caching completion), use a thread count of 0. Setups who use a Druid backend in programatic settings (sub-second re-querying) should consider setting this to 0 to prevent eventual consistency from biting overall performance in the ass. If this is you, please experiment to find out what setting works best. |`0`| +#### Concurrent Requests + +Druid uses Jetty to serve HTTP requests. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.server.http.numThreads`|Number of threads for HTTP requests.|10| +|`druid.server.http.maxIdleTime`|The Jetty max idle time for a connection.|PT5m| +|`druid.broker.http.numConnections`|Size of connection pool for the Broker to connect to historical and real-time nodes. If there are more queries than this number that all need to speak to the same node, then they will queue up.|5| +|`druid.broker.http.readTimeout`|The timeout for data reads.|PT15M| + +#### Processing + +The broker only uses processing configs for nested groupBy queries. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)| +|`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s| +|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)| +|`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)| + +#### General Query Configuration + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.query.chunkPeriod`|Long-interval queries (of any type) may be broken into shorter interval queries, reducing the impact on resources. Use ISO 8601 periods. For example, if this property is set to `P1M` (one month), then a query covering a year would be broken into 12 smaller queries. |0 (off)| + +##### GroupBy Query Config + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.query.groupBy.singleThreaded`|Run single threaded group By queries.|false| +|`druid.query.groupBy.maxIntermediateRows`|Maximum number of intermediate rows.|50000| +|`druid.query.groupBy.maxResults`|Maximum number of results.|500000| + +##### Search Query Config + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.query.search.maxSearchLimit`|Maximum number of search results to return.|1000| + +### Caching + +You can optionally only configure caching to be enabled on the broker by setting caching configs here. + +|Property|Possible Values|Description|Default| +|--------|---------------|-----------|-------| +|`druid.broker.cache.useCache`|Enable the cache on the broker.|false| +|`druid.broker.cache.populateCache`|Populate the cache on the broker.|false| +|`druid.cache.type`|`local`, `memcached`|The type of cache to use for queries.|`local`| +|`druid.cache.unCacheable`|All druid query types|All query types to not cache.|["groupBy", "select"]| #### Local Cache |Property|Description|Default| |--------|-----------|-------| -|`druid.broker.cache.sizeInBytes`|Maximum cache size in bytes. Zero disables caching.|0| -|`druid.broker.cache.initialSize`|Initial size of the hashtable backing the cache.|500000| -|`druid.broker.cache.logEvictionCount`|If non-zero, log cache eviction every `logEvictionCount` items.|0| -|`druid.broker.cache.numBackgroundThreads`|Number of background threads in the thread pool to use for eventual-consistency caching results if caching is used. It is recommended to set this value greater or equal to the number of processing threads. To force caching to execute in the same thread as the query (query results are blocked on caching completion), use a thread count of 0. Setups who use a Druid backend in programatic settings (sub-second re-querying) should consider setting this to 0 to prevent eventual consistency from biting overall performance in the ass. If this is you, please experiment to find out what setting works best. |`0`| - +|`druid.cache.sizeInBytes`|Maximum cache size in bytes. Zero disables caching.|0| +|`druid.cache.initialSize`|Initial size of the hashtable backing the cache.|500000| +|`druid.cache.logEvictionCount`|If non-zero, log cache eviction every `logEvictionCount` items.|0| #### Memcache |Property|Description|Default| |--------|-----------|-------| -|`druid.broker.cache.expiration`|Memcached [expiration time](https://code.google.com/p/memcached/wiki/NewCommands#Standard_Protocol).|2592000 (30 days)| -|`druid.broker.cache.timeout`|Maximum time in milliseconds to wait for a response from Memcached.|500| -|`druid.broker.cache.hosts`|Comma separated list of Memcached hosts ``.|none| -|`druid.broker.cache.maxObjectSize`|Maximum object size in bytes for a Memcached object.|52428800 (50 MB)| -|`druid.broker.cache.memcachedPrefix`|Key prefix for all keys in Memcached.|druid| +|`druid.cache.expiration`|Memcached [expiration time](https://code.google.com/p/memcached/wiki/NewCommands#Standard_Protocol).|2592000 (30 days)| +|`druid.cache.timeout`|Maximum time in milliseconds to wait for a response from Memcached.|500| +|`druid.cache.hosts`|Command separated list of Memcached hosts ``.|none| +|`druid.cache.maxObjectSize`|Maximum object size in bytes for a Memcached object.|52428800 (50 MB)| +|`druid.cache.memcachedPrefix`|Key prefix for all keys in Memcached.|druid| diff --git a/docs/content/Configuration.md b/docs/content/Configuration.md index 1b31a3865de..4e7ad0cfd0a 100644 --- a/docs/content/Configuration.md +++ b/docs/content/Configuration.md @@ -4,7 +4,7 @@ layout: doc_page # Configuring Druid -This describes the basic server configuration that is loaded by all Druid server processes; the same file is loaded by all. See also the JSON "specFile" descriptions in [Realtime](Realtime.html) and [Batch-ingestion](Batch-ingestion.html). +This describes the common configuration shared by all Druid nodes. These configurations can be defined in the `common.runtime.properties` file. ## JVM Configuration Best Practices @@ -14,51 +14,17 @@ There are three JVM parameters that we set on all of our processes: 2. `-Dfile.encoding=UTF-8` This is similar to timezone, we test assuming UTF-8. Local encodings might work, but they also might result in weird and interesting bugs. 3. `-Djava.io.tmpdir=` Various parts of the system that interact with the file system do it via temporary files, and these files can get somewhat large. Many production systems are set up to have small (but fast) `/tmp` directories, which can be problematic with Druid so we recommend pointing the JVM’s tmp directory to something with a little more meat. -## Modules +### Extensions -As of Druid v0.6, most core Druid functionality has been compartmentalized into modules. There are a set of default modules that may apply to any node type, and there are specific modules for the different node types. Default modules are __lazily instantiated__. Each module has its own set of configuration. - -This page describes the configuration of the default modules. Node-specific configuration is discussed on each node's respective page. In addition, you can add custom modules to [extend Druid](Modules.html). - -Configuration of the various modules is done via Java properties. These can either be provided as `-D` system properties on the java command line or they can be passed in via a file called `runtime.properties` that exists on the classpath. - -Note: as a future item, we’d like to consolidate all of the various configuration into a yaml/JSON based configuration file. - -### Emitter Module - -The Druid servers emit various metrics and alerts via something we call an Emitter. There are two emitter implementations included with the code, one that just logs to log4j ("logging", which is used by default if no emitter is specified) and one that does POSTs of JSON events to a server ("http"). The properties for using the logging emitter are described below. +Many of Druid's external dependencies can be plugged in as modules. Extensions can be provided using the following configs: |Property|Description|Default| |--------|-----------|-------| -|`druid.emitter`|Setting this value to either "logging" or "http" will instantialize one of the emitter modules.|logging| +|`druid.extensions.remoteRepositories`|If this is not set to '[]', Druid will try to download extensions at the specified remote repository.|["http://repo1.maven.org/maven2/","https://metamx.artifactoryonline.com/metamx/pub-libs-releases-local"]| +|`druid.extensions.localRepository`|The local maven directory where extensions are installed. If this is set, remoteRepositories is not required.|[]| +|`druid.extensions.coordinates`|The list of extensions to include.|[]| - -#### Logging Emitter Module - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.emitter.logging.loggerClass`|Choices: HttpPostEmitter, LoggingEmitter, NoopServiceEmitter, ServiceEmitter. The class used for logging.|LoggingEmitter| -|`druid.emitter.logging.logLevel`|Choices: debug, info, warn, error. The log level at which message are logged.|info| - -#### Http Emitter Module - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.emitter.http.timeOut`|The timeout for data reads.|PT5M| -|`druid.emitter.http.flushMillis`|How often to internal message buffer is flushed (data is sent).|60000| -|`druid.emitter.http.flushCount`|How many messages can the internal message buffer hold before flushing (sending).|500| -|`druid.emitter.http.recipientBaseUrl`|The base URL to emit messages to. Druid will POST JSON to be consumed at the HTTP endpoint specified by this property.|none| - -### Http Client Module - -This is the HTTP client used by [Broker](Broker.html) nodes. - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.broker.http.numConnections`|Size of connection pool for the Broker to connect to historical and real-time nodes. If there are more queries than this number that all need to speak to the same node, then they will queue up.|5| -|`druid.broker.http.readTimeout`|The timeout for data reads.|PT15M| - -### Curator Module +### Zookeeper Druid uses [Curator](http://curator.incubator.apache.org/) for all [Zookeeper](http://zookeeper.apache.org/) interactions. @@ -68,110 +34,36 @@ Druid uses [Curator](http://curator.incubator.apache.org/) for all [Zookeeper](h |`druid.zk.service.sessionTimeoutMs`|ZooKeeper session timeout, in milliseconds.|30000| |`druid.curator.compress`|Boolean flag for whether or not created Znodes should be compressed.|false| -### Announcer Module - -The announcer module is used to announce and unannounce Znodes in ZooKeeper (using Curator). - -#### ZooKeeper Paths - -See [ZooKeeper](ZooKeeper.html). - -#### Data Segment Announcer - -Data segment announcers are used to announce segments. +We recommend just setting the base ZK path, but all ZK paths that Druid uses can be overwritten. |Property|Description|Default| |--------|-----------|-------| -|`druid.announcer.type`|Choices: legacy or batch. The type of data segment announcer to use.|batch| +|`druid.zk.paths.base`|Base Zookeeper path.|druid| +|`druid.zk.paths.propertiesPath`|Zookeeper properties path.|druid/properties| +|`druid.zk.paths.announcementsPath`|Druid node announcement path.|druid/announcements| +|`druid.zk.paths.liveSegmentsPath`|Current path for where Druid nodes announce their segments.|druid/segments| +|`druid.zk.paths.loadQueuePath`|Entries here cause historical nodes to load and drop segments.|druid/loadQueue| +|`druid.zk.paths.coordinatorPath`|Used by the coordinator for leader election.|druid/coordinator| +|`druid.zk.paths.servedSegmentsPath`|@Deprecated. Legacy path for where Druid nodes announce their segments.|druid/servedSegments| -##### Single Data Segment Announcer - -In legacy Druid, each segment served by a node would be announced as an individual Znode. - -##### Batch Data Segment Announcer - -In current Druid, multiple data segments may be announced under the same Znode. +The indexing service also uses its own set of paths. These configs can be included in the common configuration. |Property|Description|Default| |--------|-----------|-------| -|`druid.announcer.segmentsPerNode`|Each Znode contains info for up to this many segments.|50| -|`druid.announcer.maxBytesPerNode`|Max byte size for Znode.|524288| +|`druid.zk.paths.indexer.announcementsPath`|Middle managers announce themselves here.|druid/indexer/announcements| +|`druid.zk.paths.indexer.tasksPath`|Used to assign tasks to middle managers.|druid/indexer/tasks| +|`druid.zk.paths.indexer.statusPath`|Parent path for announcement of task statuses.|druid/indexer/status| +|`druid.zk.paths.indexer.leaderLatchPath`|Used for Overlord leader election.|druid/indexer/leaderLatchPath| -### Druid Processing Module - -This module contains query processing functionality. +The following path is used service discovery. |Property|Description|Default| |--------|-----------|-------| -|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)| -|`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s| -|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)| -|`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)| +|`druid.discovery.curator.path`|Services announce themselves under this ZooKeeper path.|/druid/discovery| +### Request Logging -### Metrics Module - -The metrics module is used to track Druid metrics. - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.monitoring.emissionPeriod`|How often metrics are emitted.|PT1m| -|`druid.monitoring.monitors`|Sets list of Druid monitors used by a node. Each monitor is specified as `com.metamx.metrics.` (see below for names and more information). For example, you can specify monitors for a Broker with `druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor"]`.|none (no monitors)| - -The following monitors are available: - -* CacheMonitor – Emits metrics (to logs) about the segment results cache for Historical and Broker nodes. Reports typical cache statistics include hits, misses, rates, and size (bytes and number of entries), as well as timeouts and and errors. -* SysMonitor – This uses the [SIGAR library](http://www.hyperic.com/products/sigar) to report on various system activities and statuses. -* ServerMonitor – Reports statistics on Historical nodes. -* JvmMonitor – Reports JVM-related statistics. -* RealtimeMetricsMonitor – Reports statistics on Realtime nodes. - -### Server Module - -This module is used for Druid server nodes. - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|none| -|`druid.port`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|none| -|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|none| - -### Storage Node Module - -This module is used by nodes that store data (Historical and Realtime). - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.server.maxSize`|The maximum number of bytes-worth of segments that the node wants assigned to it. This is not a limit that Historical nodes actually enforce, just a value published to the Coordinator node so it can plan accordingly.|0| -|`druid.server.tier`| A string to name the distribution tier that the storage node belongs to. Many of the [rules Coordinator nodes use](Rule-Configuration.html) to manage segments can be keyed on tiers. | `_default_tier` | -|`druid.server.priority`|In a tiered architecture, the priority of the tier, thus allowing control over which nodes are queried. Higher numbers mean higher priority. The default (no priority) works for architecture with no cross replication (tiers that have no data-storage overlap). Data centers typically have equal priority. | 0 | - - -#### Segment Cache - -Druid storage nodes maintain information about segments they have already downloaded, and a disk cache to store that data. - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.segmentCache.locations`|Segments assigned to a Historical node are first stored on the local file system (in a disk cache) and then served by the Historical node. These locations define where that local cache resides. | none (no caching) | -|`druid.segmentCache.deleteOnRemove`|Delete segment files from cache once a node is no longer serving a segment.|true| -|`druid.segmentCache.dropSegmentDelayMillis`|How long a node delays before completely dropping segment.|30000 (30 seconds)| -|`druid.segmentCache.infoDir`|Historical nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir| -|`druid.segmentCache.announceIntervalMillis`|How frequently to announce segments while segments are loading from cache. Set this value to zero to wait for all segments to be loaded before announcing.|5000 (5 seconds)| -|`druid.segmentCache.numLoadingThreads`|How many segments to load concurrently from from deep storage.|1| - -### Jetty Server Module - -Druid uses Jetty to serve HTTP requests. - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.server.http.numThreads`|Number of threads for HTTP requests.|10| -|`druid.server.http.maxIdleTime`|The Jetty max idle time for a connection.|PT5m| - -### Queryable Module - -This module is used by all nodes that can serve queries. +All nodes that can serve queries can also log the requests they see. |Property|Description|Default| |--------|-----------|-------| @@ -193,58 +85,54 @@ Every request is emitted to some external location. |--------|-----------|-------| |`druid.request.logging.feed`|Feed name for requests.|none| -### Query Runner Factory Module +### Enabling Metrics -This module is required by nodes that can serve queries. +Druid nodes periodically emit metrics and different metrics monitors can be included. Each node can overwrite the default list of monitors. |Property|Description|Default| |--------|-----------|-------| -|`druid.query.chunkPeriod`|Long-interval queries (of any type) may be broken into shorter interval queries, reducing the impact on resources. Use ISO 8601 periods. For example, if this property is set to `P1M` (one month), then a query covering a year would be broken into 12 smaller queries. |0 (off)| +|`druid.monitoring.emissionPeriod`|How often metrics are emitted.|PT1m| +|`druid.monitoring.monitors`|Sets list of Druid monitors used by a node. Each monitor is specified as `com.metamx.metrics.` (see below for names and more information). For example, you can specify monitors for a Broker with `druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor"]`.|none (no monitors)| -#### GroupBy Query Config +The following monitors are available: + +* CacheMonitor – Emits metrics (to logs) about the segment results cache for Historical and Broker nodes. Reports typical cache statistics include hits, misses, rates, and size (bytes and number of entries), as well as timeouts and and errors. +* SysMonitor – This uses the [SIGAR library](http://www.hyperic.com/products/sigar) to report on various system activities and statuses. +* ServerMonitor – Reports statistics on Historical nodes. +* JvmMonitor – Reports JVM-related statistics. +* RealtimeMetricsMonitor – Reports statistics on Realtime nodes. + +### Emitting Metrics + +The Druid servers emit various metrics and alerts via something we call an Emitter. There are three emitter implementations included with the code, a "noop" emitter, one that just logs to log4j ("logging", which is used by default if no emitter is specified) and one that does POSTs of JSON events to a server ("http"). The properties for using the logging emitter are described below. |Property|Description|Default| |--------|-----------|-------| -|`druid.query.groupBy.singleThreaded`|Run single threaded group By queries.|false| -|`druid.query.groupBy.maxIntermediateRows`|Maximum number of intermediate rows.|50000| -|`druid.query.groupBy.maxResults`|Maximum number of results.|500000| +|`druid.emitter`|Setting this value to "noop", "logging", or "http" will instantialize one of the emitter modules.|logging| - -#### Search Query Config +#### Logging Emitter Module |Property|Description|Default| |--------|-----------|-------| -|`druid.query.search.maxSearchLimit`|Maximum number of search results to return.|1000| +|`druid.emitter.logging.loggerClass`|Choices: HttpPostEmitter, LoggingEmitter, NoopServiceEmitter, ServiceEmitter. The class used for logging.|LoggingEmitter| +|`druid.emitter.logging.logLevel`|Choices: debug, info, warn, error. The log level at which message are logged.|info| - -### Discovery Module - -The discovery module is used for service discovery. +#### Http Emitter Module |Property|Description|Default| |--------|-----------|-------| -|`druid.discovery.curator.path`|Services announce themselves under this ZooKeeper path.|/druid/discovery| +|`druid.emitter.http.timeOut`|The timeout for data reads.|PT5M| +|`druid.emitter.http.flushMillis`|How often to internal message buffer is flushed (data is sent).|60000| +|`druid.emitter.http.flushCount`|How many messages can the internal message buffer hold before flushing (sending).|500| +|`druid.emitter.http.recipientBaseUrl`|The base URL to emit messages to. Druid will POST JSON to be consumed at the HTTP endpoint specified by this property.|none| +### Metadata Storage -#### Indexing Service Discovery Module - -This module is used to find the [Indexing Service](Indexing-Service.html) using Curator service discovery. - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.selectors.indexing.serviceName`|The druid.service name of the indexing service Overlord node. To start the Overlord with a different name, set it with this property. |overlord| - - -### Server Inventory View Module - -This module is used to read announcements of segments in ZooKeeper. The configs are identical to the Announcer Module. - -### Database Connector Module - -These properties specify the jdbc connection and other configuration around the database. The only processes that connect to the DB with these properties are the [Coordinator](Coordinator.html) and [Indexing service](Indexing-service.html). This is tested on metadata storage. +These properties specify the jdbc connection and other configuration around the metadata storage. The only processes that connect to the metadata storage with these properties are the [Coordinator](Coordinator.html) and [Indexing service](Indexing-service.html). |Property|Description|Default| |--------|-----------|-------| +|`druid.metadata.storage.type`|The type of metadata storage to use. Choose from "mysql", "postgres", or "derby".|derby| |`druid.metadata.storage.connector.user`|The username to connect with.|none| |`druid.metadata.storage.connector.password`|The password to connect with.|none| |`druid.metadata.storage.connector.createTables`|If Druid requires a table and it doesn't exist, create it?|true| @@ -258,18 +146,9 @@ These properties specify the jdbc connection and other configuration around the |`druid.metadata.storage.tables.taskLog`|Used by the indexing service to store task logs.|druid_taskLog| |`druid.metadata.storage.tables.taskLock`|Used by the indexing service to store task locks.|druid_taskLock| -### Jackson Config Manager Module +### Deep Storage -The Jackson Config manager reads and writes config entries from the Druid config table using [Jackson](http://jackson.codehaus.org/). - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.manager.config.pollDuration`|How often the manager polls the config table for updates.|PT1m| - - -### DataSegment Pusher/Puller Module - -This module is used to configure Druid deep storage. The configurations concern how to push and pull [Segments](Segments.html) from deep storage. +The configurations concern how to push and pull [Segments](Segments.html) from deep storage. |Property|Description|Default| |--------|-----------|-------| @@ -293,22 +172,14 @@ This deep storage is used to interface with Amazon's S3. |Property|Description|Default| |--------|-----------|-------| +|`druid.s3.accessKey`|The access key to use to access S3.|none| +|`druid.s3.secretKey`|The secret key to use to access S3.|none| |`druid.storage.bucket`|S3 bucket name.|none| |`druid.storage.baseKey`|S3 object key prefix for storage.|none| |`druid.storage.disableAcl`|Boolean flag for ACL.|false| |`druid.storage.archiveBucket`|S3 bucket name for archiving when running the indexing-service *archive task*.|none| |`druid.storage.archiveBaseKey`|S3 object key prefix for archiving.|none| -#### AWS Module - -This module is used to interact with S3. - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.s3.accessKey`|The access key to use to access S3.|none| -|`druid.s3.secretKey`|The secret key to use to access S3.|none| - - #### HDFS Deep Storage This deep storage is used to interface with HDFS. @@ -326,35 +197,62 @@ This deep storage is used to interface with Cassandra. |`druid.storage.host`|Cassandra host.|none| |`druid.storage.keyspace`|Cassandra key space.|none| -### Task Log Module +### Caching -This module is used to configure the [Indexing Service](Indexing-Service.html) task logs. +If you are using a distributed cache such as memcached, you can include the configuration here. |Property|Description|Default| |--------|-----------|-------| -|`druid.indexer.logs.type`|Choices:noop, s3, file. Where to store task logs|file| +|`druid.cache.type`|`local`, `memcached`|The type of cache to use for queries.|`local`| +|`druid.cache.unCacheable`|All druid query types|All query types to not cache.|["groupBy", "select"]| -#### File Task Logs - -Store task logs in the local filesystem. +#### Local Cache |Property|Description|Default| |--------|-----------|-------| -|`druid.indexer.logs.directory`|Local filesystem path.|log| +|`druid.cache.sizeInBytes`|Maximum cache size in bytes. Zero disables caching.|0| +|`druid.cache.initialSize`|Initial size of the hashtable backing the cache.|500000| +|`druid.cache.logEvictionCount`|If non-zero, log cache eviction every `logEvictionCount` items.|0| -#### S3 Task Logs - -Store task logs in S3. +#### Memcache |Property|Description|Default| |--------|-----------|-------| -|`druid.indexer.logs.s3Bucket`|S3 bucket name.|none| -|`druid.indexer.logs.s3Prefix`|S3 key prefix.|none| +|`druid.cache.expiration`|Memcached [expiration time](https://code.google.com/p/memcached/wiki/NewCommands#Standard_Protocol).|2592000 (30 days)| +|`druid.cache.timeout`|Maximum time in milliseconds to wait for a response from Memcached.|500| +|`druid.cache.hosts`|Command separated list of Memcached hosts ``.|none| +|`druid.cache.maxObjectSize`|Maximum object size in bytes for a Memcached object.|52428800 (50 MB)| +|`druid.cache.memcachedPrefix`|Key prefix for all keys in Memcached.|druid| -#### Noop Task Logs +### Indexing Service Discovery -No task logs are actually stored. +This config is used to find the [Indexing Service](Indexing-Service.html) using Curator service discovery. Only required if you are actually running an indexing service. -### Firehose Module +|Property|Description|Default| +|--------|-----------|-------| +|`druid.selectors.indexing.serviceName`|The druid.service name of the indexing service Overlord node. To start the Overlord with a different name, set it with this property. |overlord| -The Firehose module lists all available firehoses. There are no configurations. +### Announcing Segments + +You can optionally configure how to announce and unannounce Znodes in ZooKeeper (using Curator). For normal operations you do not need to override any of these configs. + +#### Data Segment Announcer + +Data segment announcers are used to announce segments. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.announcer.type`|Choices: legacy or batch. The type of data segment announcer to use.|batch| + +##### Single Data Segment Announcer + +In legacy Druid, each segment served by a node would be announced as an individual Znode. + +##### Batch Data Segment Announcer + +In current Druid, multiple data segments may be announced under the same Znode. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.announcer.segmentsPerNode`|Each Znode contains info for up to this many segments.|50| +|`druid.announcer.maxBytesPerNode`|Max byte size for Znode.|524288| diff --git a/docs/content/Coordinator-Config.md b/docs/content/Coordinator-Config.md index 0f02da7cac1..5a8d860c5d4 100644 --- a/docs/content/Coordinator-Config.md +++ b/docs/content/Coordinator-Config.md @@ -8,7 +8,17 @@ For general Coordinator Node information, see [here](Coordinator.html). Runtime Configuration --------------------- -The coordinator module uses several of the default modules in [Configuration](Configuration.html) and has the following set of configurations as well: +The coordinator node uses several of the global configs in [Configuration](Configuration.html) and has the following set of configurations as well: + +### Node Config + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|none| +|`druid.port`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|none| +|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|none| + +### Coordinator Operation |Property|Description|Default| |--------|-----------|-------| @@ -17,7 +27,13 @@ The coordinator module uses several of the default modules in [Configuration](Co |`druid.coordinator.startDelay`|The operation of the Coordinator works on the assumption that it has an up-to-date view of the state of the world when it runs, the current ZK interaction code, however, is written in a way that doesn’t allow the Coordinator to know for a fact that it’s done loading the current state of the world. This delay is a hack to give it enough time to believe that it has all the data.|PT300S| |`druid.coordinator.merge.on`|Boolean flag for whether or not the coordinator should try and merge small segments into a more optimal segment size.|PT300S| |`druid.coordinator.conversion.on`|Boolean flag for converting old segment indexing versions to the latest segment indexing version.|false| -|`druid.coordinator.load.timeout`|The timeout duration for when the coordinator assigns a segment to a historical node.|15 minutes| +|`druid.coordinator.load.timeout`|The timeout duration for when the coordinator assigns a segment to a historical node.|PT15M| + +### Metadata Retrieval + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.manager.config.pollDuration`|How often the manager polls the config table for updates.|PT1m| |`druid.manager.segment.pollDuration`|The duration between polls the Coordinator does for updates to the set of active segments. Generally defines the amount of lag time it can take for the coordinator to notice new segments.|PT1M| |`druid.manager.rules.pollDuration`|The duration between polls the Coordinator does for updates to the set of active rules. Generally defines the amount of lag time it can take for the coordinator to notice rules.|PT1M| |`druid.manager.rules.defaultTier`|The default tier from which default rules will be loaded from.|_default| diff --git a/docs/content/Historical-Config.md b/docs/content/Historical-Config.md index 6a42ecfad04..75a66c97fdd 100644 --- a/docs/content/Historical-Config.md +++ b/docs/content/Historical-Config.md @@ -5,18 +5,98 @@ Historical Node Configuration ============================= For general Historical Node information, see [here](Historical.html). - Runtime Configuration --------------------- -The historical module uses several of the default modules in [Configuration](Configuration.html) and has a few configs of its own. +The historical node uses several of the global configs in [Configuration](Configuration.html) and has the following set of configurations as well: -#### Local Cache +### Node Configs + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|none| +|`druid.port`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|none| +|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|none| + +### General Configuration + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.server.tier`| A string to name the distribution tier that the storage node belongs to. Many of the [rules Coordinator nodes use](Rule-Configuration.html) to manage segments can be keyed on tiers. | `_default_tier` | +|`druid.server.priority`|In a tiered architecture, the priority of the tier, thus allowing control over which nodes are queried. Higher numbers mean higher priority. The default (no priority) works for architecture with no cross replication (tiers that have no data-storage overlap). Data centers typically have equal priority. | 0 | + +### Storing Segments + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.segmentCache.locations`|Segments assigned to a Historical node are first stored on the local file system (in a disk cache) and then served by the Historical node. These locations define where that local cache resides. | none (no caching) | +|`druid.segmentCache.deleteOnRemove`|Delete segment files from cache once a node is no longer serving a segment.|true| +|`druid.segmentCache.dropSegmentDelayMillis`|How long a node delays before completely dropping segment.|30000 (30 seconds)| +|`druid.segmentCache.infoDir`|Historical nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir| +|`druid.segmentCache.announceIntervalMillis`|How frequently to announce segments while segments are loading from cache. Set this value to zero to wait for all segments to be loaded before announcing.|5000 (5 seconds)| +|`druid.segmentCache.numLoadingThreads`|How many segments to load concurrently from from deep storage.|1| + +### Query Configs + +#### Concurrent Requests + +Druid uses Jetty to serve HTTP requests. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.server.http.numThreads`|Number of threads for HTTP requests.|10| +|`druid.server.http.maxIdleTime`|The Jetty max idle time for a connection.|PT5m| + +#### Processing + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)| +|`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s| +|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)| +|`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)| + +#### General Query Configuration + +##### GroupBy Query Config + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.query.groupBy.singleThreaded`|Run single threaded group By queries.|false| +|`druid.query.groupBy.maxIntermediateRows`|Maximum number of intermediate rows.|50000| +|`druid.query.groupBy.maxResults`|Maximum number of results.|500000| + +##### Search Query Config + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.query.search.maxSearchLimit`|Maximum number of search results to return.|1000| + +### Caching + +You can optionally only configure caching to be enabled on the historical by setting caching configs here. |Property|Possible Values|Description|Default| |--------|---------------|-----------|-------| -|`druid.historical.cache.useCache`|`true`,`false`|Allow cache to be used. Cache will NOT be used unless this is set.|`false`| -|`druid.historical.cache.populateCache`|`true`,`false`|Allow cache to be populated. Cache will NOT be populated unless this is set.|`false`| -|`druid.historical.cache.unCacheable`|All druid query types|Do not attempt to cache queries whose types are in this array|`["groupBy","select"]`| -|`druid.historical.cache.numBackgroundThreads`|Non-negative integer|Number of background threads in the thread pool to use for eventual-consistency caching results if caching is used. It is recommended to set this value greater or equal to the number of processing threads. To force caching to execute in the same thread as the query (query results are blocked on caching completion), use a thread count of 0. Setups who use a Druid backend in programatic settings (sub-second re-querying) should consider setting this to 0 to prevent eventual consistency from biting overall performance in the ass. If this is you, please experiment to find out what setting works best.|`0`| +|`druid.historical.cache.useCache`|Enable the cache on the broker.|false| +|`druid.historical.cache.populateCache`|Populate the cache on the broker.|false| +|`druid.cache.type`|`local`, `memcached`|The type of cache to use for queries.|`local`| +|`druid.cache.unCacheable`|All druid query types|All query types to not cache.|["groupBy", "select"]| +#### Local Cache + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.cache.sizeInBytes`|Maximum cache size in bytes. Zero disables caching.|0| +|`druid.cache.initialSize`|Initial size of the hashtable backing the cache.|500000| +|`druid.cache.logEvictionCount`|If non-zero, log cache eviction every `logEvictionCount` items.|0| + +#### Memcache + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.cache.expiration`|Memcached [expiration time](https://code.google.com/p/memcached/wiki/NewCommands#Standard_Protocol).|2592000 (30 days)| +|`druid.cache.timeout`|Maximum time in milliseconds to wait for a response from Memcached.|500| +|`druid.cache.hosts`|Command separated list of Memcached hosts ``.|none| +|`druid.cache.maxObjectSize`|Maximum object size in bytes for a Memcached object.|52428800 (50 MB)| +|`druid.cache.memcachedPrefix`|Key prefix for all keys in Memcached.|druid| diff --git a/docs/content/Indexing-Service-Config.md b/docs/content/Indexing-Service-Config.md index 4517e5b6580..795953435d1 100644 --- a/docs/content/Indexing-Service-Config.md +++ b/docs/content/Indexing-Service-Config.md @@ -3,9 +3,54 @@ layout: doc_page --- For general Indexing Service information, see [here](Indexing-Service.html). -#### Runtime Configuration +## Runtime Configuration -In addition to the configuration of some of the default modules in [Configuration](Configuration.html), the overlord has the following basic configs: +The indexing service uses several of the global configs in [Configuration](Configuration.html) and has the following set of configurations as well: + +### Must be set on Overlord and Middle Manager + +#### Node Configs + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|none| +|`druid.port`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|none| +|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|none| + +#### Task Logging + +If you are running the indexing service in remote mode, the task logs must S3 or HDFS. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.indexer.logs.type`|Choices:noop, s3, hdfs, file. Where to store task logs|file| + +##### File Task Logs + +Store task logs in the local filesystem. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.indexer.logs.directory`|Local filesystem path.|log| + +##### S3 Task Logs + +Store task logs in S3. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.indexer.logs.s3Bucket`|S3 bucket name.|none| +|`druid.indexer.logs.s3Prefix`|S3 key prefix.|none| + +##### HDFS Task Logs + +Store task logs in HDFS. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.indexer.logs.directory`|The directory to store logs.|none| + +### Overlord Configs |Property|Description|Default| |--------|-----------|-------| @@ -23,7 +68,7 @@ The following configs only apply if the overlord is running in remote mode: |--------|-----------|-------| |`druid.indexer.runner.taskAssignmentTimeout`|How long to wait after a task as been assigned to a middle manager before throwing an error.|PT5M| |`druid.indexer.runner.minWorkerVersion`|The minimum middle manager version to send tasks to. |"0"| -|`druid.indexer.runner.compressZnodes`|Indicates whether or not the overlord should expect middle managers to compress Znodes.|false| +|`druid.indexer.runner.compressZnodes`|Indicates whether or not the overlord should expect middle managers to compress Znodes.|true| |`druid.indexer.runner.maxZnodeBytes`|The maximum size Znode in bytes that can be created in Zookeeper.|524288| There are additional configs for autoscaling (if it is enabled): @@ -44,43 +89,105 @@ There are additional configs for autoscaling (if it is enabled): #### Dynamic Configuration -Overlord dynamic configuration is mainly for autoscaling. The overlord reads a worker setup spec as a JSON object from the Druid [metadata storage](Metadata-storage.html) config table. This object contains information about the version of middle managers to create, the maximum and minimum number of middle managers in the cluster at one time, and additional information required to automatically create middle managers. +The overlord can dynamically change worker behavior. The JSON object can be submitted to the overlord via a POST request at: ``` -http://:/druid/indexer/v1/worker/setup +http://:/druid/indexer/v1/worker ``` -A sample worker setup spec is shown below: +A sample worker config spec is shown below: ```json { - "minVersion":"some_version", - "minNumWorkers":"0", - "maxNumWorkers":"10", - "nodeData": { - "type":"ec2", - "amiId":"ami-someId", - "instanceType":"m1.xlarge", - "minInstances":"1", - "maxInstances":"1", - "securityGroupIds":["securityGroupIds"], - "keyName":"keyName" + "selectStrategy": { + "type": "fillCapacityWithAffinity", + "affinityConfig": { + "affinity": { + "datasource1": ["ip1:port", "ip2:port"], + "datasource2": ["ip3:port"] + } + } }, - "userData":{ - "impl":"string", - "data":"version=:VERSION:", - "versionReplacementString":":VERSION:" + "autoScaler": { + "type": "ec2", + "minNumWorkers": 2, + "maxNumWorkers": 12, + "envConfig": { + "availabilityZone": "us-east-1a", + "nodeData": { + "amiId": "${AMI}", + "instanceType": "c3.8xlarge", + "minInstances": 1, + "maxInstances": 1, + "securityGroupIds": ["${IDs}"], + "keyName": ${KEY_NAME} + }, + "userData": { + "impl": "string", + "data": "${SCRIPT_COMMAND}", + "versionReplacementString": ":VERSION:", + "version": null + } + } } } ``` -Issuing a GET request at the same URL will return the current worker setup spec that is currently in place. The worker setup spec list above is just a sample and it is possible to extend the code base for other deployment environments. A description of the worker setup spec is shown below. +Issuing a GET request at the same URL will return the current worker config spec that is currently in place. The worker config spec list above is just a sample for EC2 and it is possible to extend the code base for other deployment environments. A description of the worker config spec is shown below. + +|Property|Description|Default| +|--------|-----------|-------| +|`selectStrategy`|How to assign tasks to middlemanagers. Choices are `fillCapacity` and `fillCapacityWithAffinity`.|fillCapacity| +|`autoScaler`|Only used if autoscaling is enabled. See below.|null| + +#### Worker Select Strategy + +##### Fill Capacity + +Workers are assigned tasks until capacity. + +|Property|Description|Default| +|--------|-----------|-------| +|`type`|`fillCapacity`.|fillCapacity| + +##### Fill Capacity With Affinity + +An affinity config can be provided. + +|Property|Description|Default| +|--------|-----------|-------| +|`type`|`fillCapacityWithAffinity`.|fillCapacityWithAffinity| +|`affinity`|A map to String to list of String host names.|{}| + +Tasks will try to be assigned to preferred workers. Fill capacity strategy is used if no preference for a datasource specified. + +#### Autoscaler + +Amazon's EC2 is currently the only supported autoscaler. |Property|Description|Default| |--------|-----------|-------| |`minNumWorkers`|The minimum number of workers that can be in the cluster at any given time.|0| |`maxNumWorkers`|The maximum number of workers that can be in the cluster at any given time.|0| -|`nodeData`|A JSON object that describes how to launch new nodes. Currently, only EC2 is supported.|none; required| -|`userData`|A JSON object that describes how to configure new nodes. Currently, only EC2 is supported. If you have set druid.indexer.autoscale.workerVersion, this must have a versionReplacementString. Otherwise, a versionReplacementString is not necessary.|none; optional| +|`availabilityZone`|What availability zone to run in.|none| +|`nodeData`|A JSON object that describes how to launch new nodes.|none; required| +|`userData`|A JSON object that describes how to configure new nodes. If you have set druid.indexer.autoscale.workerVersion, this must have a versionReplacementString. Otherwise, a versionReplacementString is not necessary.|none; optional| + +### MiddleManager Configs + +Middle managers pass their configurations down to their child peons. The middle manager requires the following configs: + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.worker.ip`|The IP of the worker.|localhost| +|`druid.worker.version`|Version identifier for the middle manager.|0| +|`druid.worker.capacity`|Maximum number of tasks the middle manager can accept.|Number of available processors - 1| +|`druid.indexer.runner.compressZnodes`|Indicates whether or not the middle managers should compress Znodes.|false| +|`druid.indexer.runner.maxZnodeBytes`|The maximum size Znode in bytes that can be created in Zookeeper.|524288| +|`druid.indexer.runner.javaCommand`|Command required to execute java.|java| +|`druid.indexer.runner.javaOpts`|-X Java options to run the peon in its own JVM.|""| +|`druid.indexer.runner.classpath`|Java classpath for the peon.|System.getProperty("java.class.path")| +|`druid.indexer.runner.startPort`|The port that peons begin running on.|8081| +|`druid.indexer.runner.allowedPrefixes`|Whitelist of prefixes for configs that can be passed down to child peons.|"com.metamx", "druid", "io.druid", "user.timezone","file.encoding"| diff --git a/docs/content/Ingestion-FAQ.md b/docs/content/Ingestion-FAQ.md index f09d653476c..982b3123658 100644 --- a/docs/content/Ingestion-FAQ.md +++ b/docs/content/Ingestion-FAQ.md @@ -12,14 +12,14 @@ Depending on what `druid.storage.type` is set to, Druid will upload segments to ## My realtime node is not handing segments off -Make sure that the `druid.publish.type` on your real-time nodes is set to `metadata`. Also make sure that `druid.storage.type` is set to a deep storage that makes sense. Some example configs: +Make sure that the `druid.publish.type` on your real-time nodes is set to "metadata". Also make sure that `druid.storage.type` is set to a deep storage that makes sense. Some example configs: ``` -druid.publish.type=metadata +druid.publish.type=db -druid.metadata.storage.connector.connectURI=jdbc\:metadata storage\://localhost\:3306/druid -druid.metadata.storage.connector.user=druid -druid.metadata.storage.connector.password=diurd +druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid +druid.db.connector.user=druid +druid.db.connector.password=diurd druid.storage.type=s3 druid.storage.bucket=druid diff --git a/docs/content/Realtime-Config.md b/docs/content/Realtime-Config.md index 6e73c84c665..be7c4851480 100644 --- a/docs/content/Realtime-Config.md +++ b/docs/content/Realtime-Config.md @@ -2,118 +2,62 @@ layout: doc_page --- Realtime Node Configuration -=========================== -For general Real-time Node information, see [here](Realtime.html). +============================== +For general Realtime Node information, see [here](Realtime.html). -For Real-time Ingestion, see [Realtime Ingestion](Realtime-ingestion.html). +Runtime Configuration +--------------------- -Quick Start ------------ -Run: +The realtime node uses several of the global configs in [Configuration](Configuration.html) and has the following set of configurations as well: -``` -io.druid.cli.Main server realtime -``` +### Node Config -With the following JVM configuration: +|Property|Description|Default| +|--------|-----------|-------| +|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|none| +|`druid.port`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|none| +|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|none| -``` --server --Xmx256m --Duser.timezone=UTC --Dfile.encoding=UTF-8 +### Realtime Operation -druid.host=localhost -druid.service=realtime -druid.port=8083 +|Property|Description|Default| +|--------|-----------|-------| +|`druid.publish.type`|Where to publish segments. Choices are "noop" or "metadata".|metadata| +|`druid.realtime.specFile`|File location of realtime specFile.|none| -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.160"] +### Storing Intermediate Segments + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.segmentCache.locations`|Where intermediate segments are stored. The maxSize should always be zero.|none| -druid.zk.service.host=localhost +### Query Configs -# The realtime config file. -druid.realtime.specFile=/path/to/specFile +#### Processing -# Choices: metadata (hand off segments), noop (do not hand off segments). -druid.publish.type=metadata +|Property|Description|Default| +|--------|-----------|-------| +|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)| +|`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s| +|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)| +|`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)| -druid.metadata.storage.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid -druid.metadata.storage.connector.user=druid -druid.metadata.storage.connector.password=diurd +#### General Query Configuration -druid.processing.buffer.sizeBytes=100000000 -``` +##### GroupBy Query Config -Production Configs ------------------- -These production configs are using S3 as a deep store. +|Property|Description|Default| +|--------|-----------|-------| +|`druid.query.groupBy.singleThreaded`|Run single threaded group By queries.|false| +|`druid.query.groupBy.maxIntermediateRows`|Maximum number of intermediate rows.|50000| +|`druid.query.groupBy.maxResults`|Maximum number of results.|500000| -JVM settings: +##### Search Query Config -``` --server --Xmx#{HEAP_MAX}g --Xms#{HEAP_MIN}g --XX:NewSize=#{NEW_SIZE}g --XX:MaxNewSize=#{MAX_NEW_SIZE}g --XX:+UseConcMarkSweepGC --XX:+PrintGCDetails --XX:+PrintGCTimeStamps --Duser.timezone=UTC --Dfile.encoding=UTF-8 --Djava.io.tmpdir=/mnt/tmp +|Property|Description|Default| +|--------|-----------|-------| +|`druid.query.search.maxSearchLimit`|Maximum number of search results to return.|1000| --Dcom.sun.management.jmxremote.port=17071 --Dcom.sun.management.jmxremote.authenticate=false --Dcom.sun.management.jmxremote.ssl=false -``` -Runtime.properties: -``` -druid.host=#{IP_ADDR}:8080 -druid.port=8080 -druid.service=druid/prod/realtime - -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.160","io.druid.extensions:druid-kafka-seven:0.6.160"] - -druid.zk.service.host=#{ZK_IPs} -druid.zk.paths.base=/druid/prod - -druid.s3.accessKey=#{ACCESS_KEY} -druid.s3.secretKey=#{SECRET_KEY} - -druid.metadata.storage.connector.connectURI=jdbc:mysql://#{MYSQL_URL}:3306/druid -druid.metadata.storage.connector.user=#{MYSQL_USER} -druid.metadata.storage.connector.password=#{MYSQL_PW} -druid.metadata.storage.connector.useValidationQuery=true -druid.metadata.storage.tables.base=prod - -druid.publish.type=metadata - -druid.processing.numThreads=3 - -druid.request.logging.type=file -druid.request.logging.dir=request_logs/ - -druid.realtime.specFile=conf/schemas.json - -druid.segmentCache.locations=[{"path": "/mnt/persistent/zk_druid", "maxSize": 0}] - -druid.storage.type=s3 -druid.storage.bucket=#{S3_STORAGE_BUCKET} -druid.storage.baseKey=prod-realtime/v1 - -druid.monitoring.monitors=["com.metamx.metrics.SysMonitor", "io.druid.segment.realtime.RealtimeMetricsMonitor"] - -# Emit metrics over http -druid.emitter=http -druid.emitter.http.recipientBaseUrl=#{EMITTER_URL} - -# If you choose to compress ZK announcements, you must do so for every node type -druid.announcer.type=batch -druid.curator.compress=true -``` - -The realtime module also uses several of the default modules in [Configuration](Configuration.html). For more information on the realtime spec file (or configuration file), see [realtime ingestion](Realtime-ingestion.html) page. From 92ea82da6d30264bb18c6bcf5d94e0a564d86748 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 11 Dec 2014 16:18:12 -0800 Subject: [PATCH 14/71] Fix the twitter firehose * It was missing some json annotations --- .../TwitterSpritzerFirehoseFactory.java | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java b/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java index e211bf8ba5c..6e338a27939 100644 --- a/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java +++ b/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java @@ -210,22 +210,22 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory theMap = new HashMap(2); // DIY json parsing // private final ObjectMapper omapper = new ObjectMapper(); private boolean maxTimeReached() { - if (maxRunMinutes <= 0) { + if (getMaxRunMinutes() <= 0) { return false; } else { - return (System.currentTimeMillis() - startMsec) / 60000L >= maxRunMinutes; + return (System.currentTimeMillis() - startMsec) / 60000L >= getMaxRunMinutes(); } } private boolean maxCountReached() { - return maxEventCount >= 0 && rowCount >= maxEventCount; + return getMaxEventCount() >= 0 && rowCount >= getMaxEventCount(); } @Override @@ -311,4 +311,15 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory Date: Fri, 12 Dec 2014 12:59:49 -0800 Subject: [PATCH 15/71] add loadstatus endpoint for serverView status --- .../io/druid/client/BrokerServerView.java | 8 +++ .../coordination/BaseZkCoordinator.java | 2 +- .../io/druid/server/http/BrokerResource.java | 50 +++++++++++++++++++ .../src/main/java/io/druid/cli/CliBroker.java | 3 ++ 4 files changed, 62 insertions(+), 1 deletion(-) create mode 100644 server/src/main/java/io/druid/server/http/BrokerResource.java diff --git a/server/src/main/java/io/druid/client/BrokerServerView.java b/server/src/main/java/io/druid/client/BrokerServerView.java index 543951df1c3..ca2d4d6cb99 100644 --- a/server/src/main/java/io/druid/client/BrokerServerView.java +++ b/server/src/main/java/io/druid/client/BrokerServerView.java @@ -66,6 +66,8 @@ public class BrokerServerView implements TimelineServerView private final ServerInventoryView baseView; private final TierSelectorStrategy tierSelectorStrategy; + private volatile boolean initialized = false; + @Inject public BrokerServerView( QueryToolChestWarehouse warehouse, @@ -109,6 +111,7 @@ public class BrokerServerView implements TimelineServerView @Override public CallbackAction segmentViewInitialized() { + initialized = true; return ServerView.CallbackAction.CONTINUE; } } @@ -128,6 +131,11 @@ public class BrokerServerView implements TimelineServerView ); } + public boolean isInitialized() + { + return initialized; + } + public void clear() { synchronized (lock) { diff --git a/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java b/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java index 43776480829..97fa1253ec5 100644 --- a/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java @@ -54,7 +54,7 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler private final CuratorFramework curator; private volatile PathChildrenCache loadQueueCache; - private volatile boolean started; + private volatile boolean started = false; private final ListeningExecutorService loadingExec; public BaseZkCoordinator( diff --git a/server/src/main/java/io/druid/server/http/BrokerResource.java b/server/src/main/java/io/druid/server/http/BrokerResource.java new file mode 100644 index 00000000000..d843ca50e63 --- /dev/null +++ b/server/src/main/java/io/druid/server/http/BrokerResource.java @@ -0,0 +1,50 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.server.http; + +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import io.druid.client.BrokerServerView; + +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; + +@Path("/druid/broker/v1") +public class BrokerResource +{ + private final BrokerServerView brokerServerView; + + @Inject + public BrokerResource(BrokerServerView brokerServerView) + { + this.brokerServerView = brokerServerView; + } + + @GET + @Path("/loadstatus") + @Produces(MediaType.APPLICATION_JSON) + public Response getLoadStatus() + { + return Response.ok(ImmutableMap.of("inventoryInitialized", brokerServerView.isInitialized())).build(); + } +} diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index 9050bc827aa..e5ac925da02 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -48,6 +48,7 @@ import io.druid.server.ClientInfoResource; import io.druid.server.ClientQuerySegmentWalker; import io.druid.server.QueryResource; import io.druid.server.coordination.broker.DruidBroker; +import io.druid.server.http.BrokerResource; import io.druid.server.initialization.JettyServerInitializer; import io.druid.server.metrics.MetricsModule; import io.druid.server.router.TieredBrokerConfig; @@ -87,6 +88,7 @@ public class CliBroker extends ServerRunnable binder.bind(QueryToolChestWarehouse.class).to(MapQueryToolChestWarehouse.class); binder.bind(CachingClusteredClient.class).in(LazySingleton.class); + binder.bind(BrokerServerView.class).in(LazySingleton.class); binder.bind(TimelineServerView.class).to(BrokerServerView.class).in(LazySingleton.class); binder.bind(Cache.class).toProvider(CacheProvider.class).in(ManageLifecycle.class); @@ -101,6 +103,7 @@ public class CliBroker extends ServerRunnable binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class).in(LazySingleton.class); Jerseys.addResource(binder, QueryResource.class); + Jerseys.addResource(binder, BrokerResource.class); Jerseys.addResource(binder, ClientInfoResource.class); LifecycleModule.register(binder, QueryResource.class); LifecycleModule.register(binder, DruidBroker.class); From ca8300a461d814d10197d177795c0586616390b5 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 19 Nov 2014 17:12:04 -0800 Subject: [PATCH 16/71] Replace AppendableByteArrayInputStream in DirectDruidClient * Replace with SequenceInputStream fueled by an enumeration of ChannelBufferInputStream which directly wrap the response context ChannelBuffer * Added zero-length byte array when DirectDruidClient is done. This prevents an odd race condition on `done`. --- .../io/druid/client/DirectDruidClient.java | 237 +++++++++++++----- 1 file changed, 174 insertions(+), 63 deletions(-) diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index 32a84a05da3..3e562457248 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc. * * This program is free software; you can redistribute it and/or * modify it under the terms of the GNU General Public License @@ -31,6 +31,7 @@ import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.Maps; +import com.google.common.io.ByteSource; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -43,9 +44,8 @@ import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.logger.Logger; 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 com.metamx.http.client.response.HttpResponseHandler; import com.metamx.http.client.response.StatusResponseHandler; import com.metamx.http.client.response.StatusResponseHolder; import io.druid.query.BySegmentResultValueClass; @@ -57,6 +57,8 @@ import io.druid.query.QueryToolChestWarehouse; import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.query.aggregation.MetricManipulatorFns; +import org.jboss.netty.buffer.ChannelBuffer; +import org.jboss.netty.buffer.ChannelBufferInputStream; import org.jboss.netty.handler.codec.http.HttpChunk; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpResponse; @@ -65,13 +67,19 @@ import javax.ws.rs.core.MediaType; import java.io.Closeable; import java.io.IOException; import java.io.InputStream; +import java.io.SequenceInputStream; import java.net.URL; +import java.util.Enumeration; import java.util.Iterator; import java.util.Map; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; /** */ @@ -143,68 +151,168 @@ public class DirectDruidClient implements QueryRunner try { log.debug("Querying url[%s]", url); + + final HttpResponseHandler responseHandler = new HttpResponseHandler() + { + private long startTime; + private final AtomicLong byteCount = new AtomicLong(0); + private final BlockingQueue queue = new LinkedBlockingQueue<>(); + private final AtomicBoolean done = new AtomicBoolean(false); + + @Override + public ClientResponse handleResponse(HttpResponse response) + { + log.debug("Initial response from url[%s]", url); + startTime = System.currentTimeMillis(); + try { + final String responseContext = response.headers().get("X-Druid-Response-Context"); + // context may be null in case of error or query timeout + if (responseContext != null) { + context.putAll( + objectMapper.>readValue( + responseContext, new TypeReference>() + { + } + ) + ); + } + queue.put(new ChannelBufferInputStream(response.getContent())); + } + catch (final IOException e) { + log.error(e, "Error parsing response context from url [%s]", url); + return ClientResponse.finished( + new InputStream() + { + @Override + public int read() throws IOException + { + throw e; + } + } + ); + } + catch (InterruptedException e) { + log.error(e, "Queue appending interrupted"); + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + byteCount.addAndGet(response.getContent().readableBytes()); + return ClientResponse.finished( + new SequenceInputStream( + new Enumeration() + { + @Override + public boolean hasMoreElements() + { + // Done is always true until the last stream has be put in the queue. + // Then the stream should be spouting good InputStreams. + synchronized (done) { + return !done.get() || !queue.isEmpty(); + } + } + + @Override + public InputStream nextElement() + { + synchronized (done) { + try { + // Ensures more elements are expected via `done` + return queue.take(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + } + } + } + ) + ); + } + + @Override + public ClientResponse handleChunk( + ClientResponse clientResponse, HttpChunk chunk + ) + { + final ChannelBuffer channelBuffer = chunk.getContent(); + final int bytes = channelBuffer.readableBytes(); + if (bytes > 0) { + try { + queue.put(new ChannelBufferInputStream(channelBuffer)); + } + catch (InterruptedException e) { + log.error(e, "Unable to put finalizing input stream into Sequence queue for url [%s]", url); + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + byteCount.addAndGet(bytes); + } + return clientResponse; + } + + @Override + public ClientResponse done(ClientResponse clientResponse) + { + long stopTime = System.currentTimeMillis(); + log.debug( + "Completed request to url[%s] with %,d bytes returned in %,d millis [%,f b/s].", + url, + byteCount.get(), + stopTime - startTime, + byteCount.get() / (0.0001 * (stopTime - startTime)) + ); + synchronized (done) { + try { + // An empty byte array is put at the end to give the SequenceInputStream.close() as something to close out + // after done is set to true, regardless of the rest of the stream's state. + queue.put(ByteSource.empty().openStream()); + } + catch (InterruptedException e) { + log.error(e, "Unable to put finalizing input stream into Sequence queue for url [%s]", url); + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + catch (IOException e) { + // This should never happen + throw Throwables.propagate(e); + } + finally { + done.set(true); + } + } + return ClientResponse.finished(clientResponse.getObj()); + } + + @Override + public void exceptionCaught(final ClientResponse clientResponse, final Throwable e) + { + // Don't wait for lock in case the lock had something to do with the error + synchronized (done) { + done.set(true); + // Make a best effort to put a zero length buffer into the queue in case something is waiting on the take() + // If nothing is waiting on take(), this will be closed out anyways. + queue.offer( + new InputStream() + { + @Override + public int read() throws IOException + { + throw new IOException(e); + } + } + ); + } + } + }; future = httpClient .post(new URL(url)) .setContent(objectMapper.writeValueAsBytes(query)) - .setHeader(HttpHeaders.Names.CONTENT_TYPE, isSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON) - .go( - new InputStreamResponseHandler() - { - long startTime; - long byteCount = 0; - - @Override - public ClientResponse handleResponse(HttpResponse response) - { - log.debug("Initial response from url[%s]", url); - startTime = System.currentTimeMillis(); - byteCount += response.getContent().readableBytes(); - - try { - final String responseContext = response.headers().get("X-Druid-Response-Context"); - // context may be null in case of error or query timeout - if (responseContext != null) { - context.putAll( - objectMapper.>readValue( - responseContext, new TypeReference>() - { - } - ) - ); - } - } - catch (IOException e) { - log.error(e, "Unable to parse response context from url[%s]", url); - } - - return super.handleResponse(response); - } - - @Override - public ClientResponse handleChunk( - ClientResponse clientResponse, HttpChunk chunk - ) - { - final int bytes = chunk.getContent().readableBytes(); - byteCount += bytes; - return super.handleChunk(clientResponse, chunk); - } - - @Override - public ClientResponse done(ClientResponse 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); - } - } - ); + .setHeader( + HttpHeaders.Names.CONTENT_TYPE, + isSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON + ) + .go(responseHandler); queryWatcher.registerQuery(query, future); @@ -228,7 +336,10 @@ public class DirectDruidClient implements QueryRunner StatusResponseHolder res = httpClient .delete(new URL(cancelUrl)) .setContent(objectMapper.writeValueAsBytes(query)) - .setHeader(HttpHeaders.Names.CONTENT_TYPE, isSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON) + .setHeader( + HttpHeaders.Names.CONTENT_TYPE, + isSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON + ) .go(new StatusResponseHandler(Charsets.UTF_8)) .get(); if (res.getStatus().getCode() >= 500) { From 687c82daa882e7d5b494aff766a4913b4b10f632 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 11 Dec 2014 16:33:28 -0800 Subject: [PATCH 17/71] Added more Twitter fields to TwitterSpritzerFirehoseFactory * Now with GEOGRAPHY support! --- examples/bin/examples/twitter/query.body | 33 +++- examples/bin/examples/twitter/topN_query.body | 94 ++++++++++ .../examples/twitter/twitter_realtime.spec | 164 +++++++++++------- .../TwitterSpritzerFirehoseFactory.java | 114 ++++++++---- pom.xml | 2 +- 5 files changed, 299 insertions(+), 108 deletions(-) create mode 100644 examples/bin/examples/twitter/topN_query.body diff --git a/examples/bin/examples/twitter/query.body b/examples/bin/examples/twitter/query.body index b41ea3b46d5..76d3b9baae6 100644 --- a/examples/bin/examples/twitter/query.body +++ b/examples/bin/examples/twitter/query.body @@ -1,4 +1,31 @@ { - "queryType": "timeBoundary", - "dataSource": "twitterstream" -} + "description": "Simple data split up by hour", + "aggregations": [ + { + "name": "tweets", + "type": "longSum", + "fieldName": "tweets" + }, + { + "fieldName": "text_hll", + "name": "text_hll", + "type": "hyperUnique" + }, + { + "fieldName": "htags_hll", + "name": "htag_hll", + "type": "hyperUnique" + }, + { + "fieldName": "user_id_hll", + "name": "user_id_hll", + "type": "hyperUnique" + } + ], + "dataSource": "twitterstream", + "granularity": "hour", + "intervals": [ + "1970-01-01T00:00:00.000/2019-01-03T00:00:00.000" + ], + "queryType": "timeseries" +} \ No newline at end of file diff --git a/examples/bin/examples/twitter/topN_query.body b/examples/bin/examples/twitter/topN_query.body new file mode 100644 index 00000000000..fcd644a4f1c --- /dev/null +++ b/examples/bin/examples/twitter/topN_query.body @@ -0,0 +1,94 @@ +{ + "description": "Top 10 languages by count of tweets in the contiguous US", + "aggregations": [ + { + "fieldName": "tweets", + "name": "tweets", + "type": "longSum" + }, + { + "fieldName": "user_id_hll", + "name": "user_id_hll", + "type": "hyperUnique" + }, + { + "fieldName": "contributors_hll", + "name": "contributors_hll", + "type": "hyperUnique" + }, + { + "fieldName": "htags_hll", + "name": "htags_hll", + "type": "hyperUnique" + }, + { + "fieldName": "text_hll", + "name": "text_hll", + "type": "hyperUnique" + }, + { + "fieldName": "min_follower_count", + "name": "min_follower_count", + "type": "min" + }, + { + "fieldName": "max_follower_count", + "name": "max_follower_count", + "type": "max" + }, + { + "fieldName": "min_friends_count", + "name": "min_friends_count", + "type": "min" + }, + { + "fieldName": "max_friends_count", + "name": "max_friends_count", + "type": "max" + }, + { + "fieldName": "min_statuses_count", + "name": "min_statuses_count", + "type": "min" + }, + { + "fieldName": "max_statuses_count", + "name": "max_statuses_count", + "type": "max" + }, + { + "fieldName": "min_retweet_count", + "name": "min_retweet_count", + "type": "min" + }, + { + "fieldName": "max_retweet_count", + "name": "max_retweet_count", + "type": "max" + } + ], + "dataSource": "twitterstream", + "dimension": "lang", + "filter": { + "bound": { + "maxCoords": [ + 50, + -65 + ], + "minCoords": [ + 25, + -127 + ], + "type": "rectangular" + }, + "dimension": "geo", + "type": "spatial" + }, + "granularity": "all", + "intervals": [ + "2013-06-01T00:00/2020-01-01T00" + ], + "metric": "tweets", + "queryType": "topN", + "threshold": "10" +} \ No newline at end of file diff --git a/examples/bin/examples/twitter/twitter_realtime.spec b/examples/bin/examples/twitter/twitter_realtime.spec index a452c764d94..8a502ed5516 100644 --- a/examples/bin/examples/twitter/twitter_realtime.spec +++ b/examples/bin/examples/twitter/twitter_realtime.spec @@ -1,119 +1,151 @@ -[ - { +{ + "description": "Ingestion spec for Twitter spritzer. Dimension values taken from io.druid.examples.twitter.TwitterSpritzerFirehoseFactory", + "spec": { "dataSchema": { "dataSource": "twitterstream", - "parser": { - "parseSpec": { - "format": "json", - "timestampSpec": { - "column": "utcdt", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - - ], - "dimensionExclusions": [ - - ], - "spatialDimensions": [ - - ] - } - } + "granularitySpec": { + "queryGranularity": "all", + "segmentGranularity": "hour", + "type": "uniform" }, "metricsSpec": [ { - "type": "count", - "name": "tweets" + "name": "tweets", + "type": "count" }, { - "type": "doubleSum", "fieldName": "follower_count", - "name": "total_follower_count" + "name": "total_follower_count", + "type": "doubleSum" }, { - "type": "doubleSum", "fieldName": "retweet_count", - "name": "total_retweet_count" + "name": "total_retweet_count", + "type": "doubleSum" }, { - "type": "doubleSum", "fieldName": "friends_count", - "name": "total_friends_count" + "name": "total_friends_count", + "type": "doubleSum" }, { - "type": "doubleSum", "fieldName": "statuses_count", - "name": "total_statuses_count" + "name": "total_statuses_count", + "type": "doubleSum" + }, + { + "fieldName": "text", + "name": "text_hll", + "type": "hyperUnique" + }, + { + "fieldName": "user_id", + "name": "user_id_hll", + "type": "hyperUnique" + }, + { + "fieldName": "contributors", + "name": "contributors_hll", + "type": "hyperUnique" + }, + { + "fieldName": "htags", + "name": "htags_hll", + "type": "hyperUnique" }, { - "type": "min", "fieldName": "follower_count", - "name": "min_follower_count" + "name": "min_follower_count", + "type": "min" }, { - "type": "max", "fieldName": "follower_count", - "name": "max_follower_count" + "name": "max_follower_count", + "type": "max" }, { - "type": "min", "fieldName": "friends_count", - "name": "min_friends_count" + "name": "min_friends_count", + "type": "min" }, { - "type": "max", "fieldName": "friends_count", - "name": "max_friends_count" + "name": "max_friends_count", + "type": "max" }, { - "type": "min", "fieldName": "statuses_count", - "name": "min_statuses_count" + "name": "min_statuses_count", + "type": "min" }, { - "type": "max", "fieldName": "statuses_count", - "name": "max_statuses_count" + "name": "max_statuses_count", + "type": "max" }, { - "type": "min", "fieldName": "retweet_count", - "name": "min_retweet_count" + "name": "min_retweet_count", + "type": "min" }, { - "type": "max", "fieldName": "retweet_count", - "name": "max_retweet_count" + "name": "max_retweet_count", + "type": "max" } ], - "granularitySpec": { - "type": "uniform", - "segmentGranularity": "DAY", - "queryGranularity": "NONE" + "parser": { + "parseSpec": { + "dimensionsSpec": { + "dimensions": [ + "text", + "htags", + "contributors", + "lat", + "lon", + "retweet_count", + "follower_count", + "friendscount", + "lang", + "utc_offset", + "statuses_count", + "user_id", + "ts" + ], + "dimensionExclusions": [ + ], + "spatialDimensions": [ + { + "dimName": "geo", + "dims": [ + "lat", + "lon" + ] + } + ] + }, + "format": "json", + "timestampSpec": { + "column": "ts", + "format": "millis" + } + } } }, "ioConfig": { - "type": "realtime", "firehose": { - "type": "twitzer", "maxEventCount": 500000, - "maxRunMinutes": 120 + "maxRunMinutes": 120, + "type": "twitzer" }, - "plumber": { - "type": "realtime" - } + "type": "realtime" }, "tuningConfig": { - "type": "realtime", + "intermediatePersistPeriod": "PT10m", "maxRowsInMemory": 500000, - "intermediatePersistPeriod": "PT2m", - "windowPeriod": "PT3m", - "basePersistDirectory": "\/tmp\/realtime\/basePersist", - "rejectionPolicy": { - "type": "messageTime" - } + "type": "realtime", + "windowPeriod": "PT10m" } - } -] + }, + "type": "index_realtime" +} \ No newline at end of file diff --git a/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java b/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java index 6e338a27939..6744616ae2f 100644 --- a/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java +++ b/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc. * * This program is free software; you can redistribute it and/or * modify it under the terms of the GNU General Public License @@ -22,6 +22,8 @@ package io.druid.examples.twitter; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; import io.druid.data.input.Firehose; @@ -30,6 +32,7 @@ import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.InputRowParser; import twitter4j.ConnectionLifeCycleListener; +import twitter4j.GeoLocation; import twitter4j.HashtagEntity; import twitter4j.StallWarning; import twitter4j.Status; @@ -39,12 +42,13 @@ import twitter4j.TwitterStream; import twitter4j.TwitterStreamFactory; import twitter4j.User; +import javax.annotation.Nullable; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; -import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.TreeMap; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.TimeUnit; @@ -58,25 +62,27 @@ import static java.lang.Thread.sleep; * with timestamps along with ??. * The generated tuples have the form (timestamp, ????) * where the timestamp is from the twitter event. - * + *

* Example spec file: - * + *

* Example query using POST to /druid/v2/?w (where w is an arbitrary parameter and the date and time * is UTC): - * + *

* Notes on twitter.com HTTP (REST) API: v1.0 will be disabled around 2013-03 so v1.1 should be used; * twitter4j 3.0 (not yet released) will support the v1.1 api. * Specifically, we should be using https://stream.twitter.com/1.1/statuses/sample.json * See: http://jira.twitter4j.org/browse/TFJ-186 + *

+ * Notes on JSON parsing: as of twitter4j 2.2.x, the json parser has some bugs (ex: Status.toString() + * can have number format exceptions), so it might be necessary to extract raw json and process it + * separately. If so, set twitter4.jsonStoreEnabled=true and look at DataObjectFactory#getRawJSON(); + * com.fasterxml.jackson.databind.ObjectMapper should be used to parse. * - * Notes on JSON parsing: as of twitter4j 2.2.x, the json parser has some bugs (ex: Status.toString() - * can have number format exceptions), so it might be necessary to extract raw json and process it - * separately. If so, set twitter4.jsonStoreEnabled=true and look at DataObjectFactory#getRawJSON(); - * com.fasterxml.jackson.databind.ObjectMapper should be used to parse. * @author pbaclace */ @JsonTypeName("twitzer") -public class TwitterSpritzerFirehoseFactory implements FirehoseFactory { +public class TwitterSpritzerFirehoseFactory implements FirehoseFactory +{ private static final Logger log = new Logger(TwitterSpritzerFirehoseFactory.class); /** * max events to receive, -1 is infinite, 0 means nothing is delivered; use this to prevent @@ -107,7 +113,8 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory queue = new ArrayBlockingQueue(QUEUE_SIZE); - final LinkedList dimensions = new LinkedList(); final long startMsec = System.currentTimeMillis(); - dimensions.add("htags"); - dimensions.add("lang"); - dimensions.add("utc_offset"); - // // set up Twitter Spritzer // twitterStream = new TwitterStreamFactory().getInstance(); twitterStream.addConnectionLifeCycleListener(connectionLifeCycleListener); - statusListener = new StatusListener() { // This is what really gets called to deliver stuff from twitter4j + statusListener = new StatusListener() + { // This is what really gets called to deliver stuff from twitter4j @Override public void onStatus(Status status) { @@ -160,7 +163,8 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory theMap = new HashMap(2); + private final Map theMap = new TreeMap<>(); // DIY json parsing // private final ObjectMapper omapper = new ObjectMapper(); private boolean maxTimeReached() @@ -253,7 +260,8 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory 0) { - List hashTags = Lists.newArrayListWithExpectedSize(hts.length); - for (HashtagEntity ht : hts) { - hashTags.add(ht.getText()); - } + String text = status.getText(); + theMap.put("text", (null == text) ? "" : text); + theMap.put( + "htags", (hts.length > 0) ? Lists.transform( + Arrays.asList(hts), new Function() + { + @Nullable + @Override + public String apply(HashtagEntity input) + { + return input.getText(); + } + } + ) : ImmutableList.of() + ); - theMap.put("htags", Arrays.asList(hashTags.get(0))); + long[] lcontrobutors = status.getContributors(); + List contributors = new ArrayList<>(); + for (long contrib : lcontrobutors) { + contributors.add(String.format("%d", contrib)); + } + theMap.put("contributors", contributors); + + GeoLocation geoLocation = status.getGeoLocation(); + if (null != geoLocation) { + double lat = status.getGeoLocation().getLatitude(); + double lon = status.getGeoLocation().getLongitude(); + theMap.put("lat", lat); + theMap.put("lon", lon); + } else { + theMap.put("lat", null); + theMap.put("lon", null); } long retweetCount = status.getRetweetCount(); theMap.put("retweet_count", retweetCount); User user = status.getUser(); - if (user != null) { - theMap.put("follower_count", user.getFollowersCount()); - theMap.put("friends_count", user.getFriendsCount()); - theMap.put("lang", user.getLang()); - theMap.put("utc_offset", user.getUtcOffset()); // resolution in seconds, -1 if not available? - theMap.put("statuses_count", user.getStatusesCount()); - } + final boolean hasUser = (null != user); + theMap.put("follower_count", hasUser ? user.getFollowersCount() : 0); + theMap.put("friends_count", hasUser ? user.getFriendsCount() : 0); + theMap.put("lang", hasUser ? user.getLang() : ""); + theMap.put("utc_offset", hasUser ? user.getUtcOffset() : -1); // resolution in seconds, -1 if not available? + theMap.put("statuses_count", hasUser ? user.getStatusesCount() : 0); + theMap.put("user_id", hasUser ? String.format("%d", user.getId()) : ""); + + theMap.put("ts",status.getCreatedAt().getTime()); + + List dimensions = Lists.newArrayList(theMap.keySet()); return new MapBasedInputRow(status.getCreatedAt().getTime(), dimensions, theMap); } diff --git a/pom.xml b/pom.xml index 577ddc38ff6..26b26ad37be 100644 --- a/pom.xml +++ b/pom.xml @@ -94,7 +94,7 @@ com.metamx bytebuffer-collections - 0.1.1 + 0.1.2 com.metamx From 092dfe0309964d1025415588906f2bf14f2d477a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 12 Dec 2014 17:05:40 -0800 Subject: [PATCH 18/71] fix IndexTaskTest tmp dir - Create local firehose files in a clean temp directory to avoid firehose reading other random temp files that start with 'druid' --- .../java/io/druid/indexing/common/task/IndexTaskTest.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 5a486e1cedb..8e40afcc291 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -20,6 +20,7 @@ package io.druid.indexing.common.task; import com.google.common.collect.Lists; +import com.google.common.io.Files; import com.metamx.common.Granularity; import io.druid.data.input.impl.CSVParseSpec; import io.druid.data.input.impl.DimensionsSpec; @@ -57,7 +58,10 @@ public class IndexTaskTest @Test public void testDeterminePartitions() throws Exception { - File tmpFile = File.createTempFile("druid", "index"); + File tmpDir = Files.createTempDir(); + tmpDir.deleteOnExit(); + + File tmpFile = File.createTempFile("druid", "index", tmpDir); tmpFile.deleteOnExit(); PrintWriter writer = new PrintWriter(tmpFile); @@ -97,7 +101,7 @@ public class IndexTaskTest ), new IndexTask.IndexIOConfig( new LocalFirehoseFactory( - tmpFile.getParentFile(), + tmpDir, "druid*", null ) From 020f8653f0d426e083bd8484db5b07d3dabc5bf3 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Sat, 13 Dec 2014 08:01:38 -0800 Subject: [PATCH 19/71] Fix documented default value for druid.coordinator.merge.on. --- docs/content/Coordinator-Config.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Coordinator-Config.md b/docs/content/Coordinator-Config.md index d1bc8fb49b0..40a117608a6 100644 --- a/docs/content/Coordinator-Config.md +++ b/docs/content/Coordinator-Config.md @@ -15,7 +15,7 @@ The coordinator module uses several of the default modules in [Configuration](Co |`druid.coordinator.period`|The run period for the coordinator. The coordinator’s operates by maintaining the current state of the world in memory and periodically looking at the set of segments available and segments being served to make decisions about whether any changes need to be made to the data topology. This property sets the delay between each of these runs.|PT60S| |`druid.coordinator.period.indexingPeriod`|How often to send indexing tasks to the indexing service. Only applies if merge or conversion is turned on.|PT1800S (30 mins)| |`druid.coordinator.startDelay`|The operation of the Coordinator works on the assumption that it has an up-to-date view of the state of the world when it runs, the current ZK interaction code, however, is written in a way that doesn’t allow the Coordinator to know for a fact that it’s done loading the current state of the world. This delay is a hack to give it enough time to believe that it has all the data.|PT300S| -|`druid.coordinator.merge.on`|Boolean flag for whether or not the coordinator should try and merge small segments into a more optimal segment size.|PT300S| +|`druid.coordinator.merge.on`|Boolean flag for whether or not the coordinator should try and merge small segments into a more optimal segment size.|false| |`druid.coordinator.conversion.on`|Boolean flag for converting old segment indexing versions to the latest segment indexing version.|false| |`druid.coordinator.load.timeout`|The timeout duration for when the coordinator assigns a segment to a historical node.|15 minutes| |`druid.manager.segment.pollDuration`|The duration between polls the Coordinator does for updates to the set of active segments. Generally defines the amount of lag time it can take for the coordinator to notice new segments.|PT1M| From bd91a404919d0e2a9c025db8e69a7f7e02f49732 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Sat, 13 Dec 2014 15:22:55 -0800 Subject: [PATCH 20/71] fix task log streaming --- .../io/druid/indexing/overlord/http/OverlordResource.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index 050a5620856..76523504f53 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -412,9 +412,7 @@ public class OverlordResource try { final Optional stream = taskLogStreamer.streamTaskLog(taskid, offset); if (stream.isPresent()) { - try(InputStream istream = stream.get().openStream()) { - return Response.ok(istream).build(); - } + return Response.ok(stream.get().openStream()).build(); } else { return Response.status(Response.Status.NOT_FOUND) .entity( From 88858ea1c3fb462bb878c1379cf3eaa4bec80b27 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 15 Dec 2014 09:42:03 -0800 Subject: [PATCH 21/71] make default handler service announcing --- services/src/main/java/io/druid/cli/CliPeon.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index f777f08b62b..beb206c8eca 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -116,7 +116,7 @@ public class CliPeon extends GuiceRunnable binder, "druid.indexer.task.chathandler.type", Key.get(ChatHandlerProvider.class), - Key.get(NoopChatHandlerProvider.class) + Key.get(ServiceAnnouncingChatHandlerProvider.class) ); final MapBinder handlerProviderBinder = PolyBind.optionBinder( binder, Key.get(ChatHandlerProvider.class) From 3cb7999eb9d7a426561e6e1bcc67a048d209eac3 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 15 Dec 2014 09:52:46 -0800 Subject: [PATCH 22/71] i hate hadoop dependencies --- .../java/io/druid/segment/incremental/IncrementalIndex.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 79a1be0753e..ae89966bfb5 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -258,7 +258,7 @@ public abstract class IncrementalIndex implements Iterable, this.minTimestamp = incrementalIndexSchema.getMinTimestamp(); this.gran = incrementalIndexSchema.getGran(); this.metrics = incrementalIndexSchema.getMetrics(); - this.rowTransformers = Lists.newCopyOnWriteArrayList(); + this.rowTransformers = new CopyOnWriteArrayList<>(); this.deserializeComplexMetrics = deserializeComplexMetrics; final ImmutableList.Builder metricNamesBuilder = ImmutableList.builder(); From 77bf5da317339477f132a1d297e8d9b386aa6203 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 15 Dec 2014 10:07:51 -0800 Subject: [PATCH 23/71] fix typo --- .../java/io/druid/curator/discovery/CuratorServiceUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/curator/discovery/CuratorServiceUtils.java b/server/src/main/java/io/druid/curator/discovery/CuratorServiceUtils.java index 4f23fed5ac1..936581d4b88 100644 --- a/server/src/main/java/io/druid/curator/discovery/CuratorServiceUtils.java +++ b/server/src/main/java/io/druid/curator/discovery/CuratorServiceUtils.java @@ -26,7 +26,7 @@ public class CuratorServiceUtils * curator-x-discovery-server * * This method is marked protected because it should never be used outside of the io.druid.curator.discovery - * packakge. If you are tempted to use this method anywhere else you are most likely doing something wrong. + * package. If you are tempted to use this method anywhere else you are most likely doing something wrong. * Mapping the actual service name to the name used within curator should be left to {@link CuratorServiceAnnouncer} * and {@link ServerDiscoveryFactory} * From 61101bee6ca4f229a74f9f564ec8bd4fda5d5e1a Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 15 Dec 2014 12:57:40 -0800 Subject: [PATCH 24/71] more logs for CIM --- .../inventory/CuratorInventoryManager.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java index 3c4bb2a2c6f..6bbc7beb391 100644 --- a/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java +++ b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java @@ -46,10 +46,10 @@ import java.util.concurrent.atomic.AtomicReference; /** * An InventoryManager watches updates to inventory on Zookeeper (or some other discovery-like service publishing * system). It is built up on two object types: containers and inventory objects. - * + *

* The logic of the InventoryManager just maintains a local cache of the containers and inventory it sees on ZK. It * provides methods for getting at the container objects, which house the actual individual pieces of inventory. - * + *

* A Strategy is provided to the constructor of an Inventory manager, this strategy provides all of the * object-specific logic to serialize, deserialize, compose and alter the container and inventory objects. */ @@ -104,7 +104,8 @@ public class CuratorInventoryManager } catch (Exception e) { synchronized (lock) { - try { stop(); + try { + stop(); } catch (IOException e1) { log.error(e1, "Exception when stopping InventoryManager that couldn't start."); @@ -276,9 +277,9 @@ public class CuratorInventoryManager case INITIALIZED: synchronized (lock) { // must await initialized of all containerholders - for(ContainerHolder holder : containers.values()) { + for (ContainerHolder holder : containers.values()) { synchronized (holder) { - if(!holder.initialized) { + if (!holder.initialized) { uninitializedInventory.add(holder); } } @@ -300,12 +301,12 @@ public class CuratorInventoryManager private void maybeDoneInitializing() { - if(doneInitializing) { + if (doneInitializing) { return; } // only fire if we are done initializing the parent PathChildrenCache - if(containersInitialized && uninitializedInventory.isEmpty()) { + if (containersInitialized && uninitializedInventory.isEmpty()) { doneInitializing = true; strategy.inventoryInitialized(); } @@ -336,6 +337,7 @@ public class CuratorInventoryManager case CHILD_ADDED: { final ChildData child = event.getData(); final String inventoryKey = ZKPaths.getNodeFromPath(child.getPath()); + log.info("CHILD_ADDED[%s] with version[%s]", inventoryKey, event.getData().getStat().getVersion()); final InventoryClass addedInventory = strategy.deserializeInventory(child.getData()); @@ -348,6 +350,7 @@ public class CuratorInventoryManager case CHILD_UPDATED: { final ChildData child = event.getData(); final String inventoryKey = ZKPaths.getNodeFromPath(child.getPath()); + log.info("CHILD_UPDATED[%s] with version[%s]", inventoryKey, event.getData().getStat().getVersion()); final InventoryClass updatedInventory = strategy.deserializeInventory(child.getData()); @@ -361,6 +364,7 @@ public class CuratorInventoryManager case CHILD_REMOVED: { final ChildData child = event.getData(); final String inventoryKey = ZKPaths.getNodeFromPath(child.getPath()); + log.info("CHILD_REMOVED[%s] with version[%s]", inventoryKey, event.getData().getStat().getVersion()); synchronized (holder) { holder.setContainer(strategy.removeInventory(holder.getContainer(), inventoryKey)); From 9c6e32fc3f30d3e8f961df08bcf06b54570523dc Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 15 Dec 2014 13:06:36 -0800 Subject: [PATCH 25/71] address cr --- .../io/druid/curator/inventory/CuratorInventoryManager.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java index 6bbc7beb391..444d46ad386 100644 --- a/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java +++ b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java @@ -337,7 +337,7 @@ public class CuratorInventoryManager case CHILD_ADDED: { final ChildData child = event.getData(); final String inventoryKey = ZKPaths.getNodeFromPath(child.getPath()); - log.info("CHILD_ADDED[%s] with version[%s]", inventoryKey, event.getData().getStat().getVersion()); + log.info("CHILD_ADDED[%s] with version[%s]", child.getPath(), event.getData().getStat().getVersion()); final InventoryClass addedInventory = strategy.deserializeInventory(child.getData()); @@ -350,7 +350,7 @@ public class CuratorInventoryManager case CHILD_UPDATED: { final ChildData child = event.getData(); final String inventoryKey = ZKPaths.getNodeFromPath(child.getPath()); - log.info("CHILD_UPDATED[%s] with version[%s]", inventoryKey, event.getData().getStat().getVersion()); + log.info("CHILD_UPDATED[%s] with version[%s]", child.getPath(), event.getData().getStat().getVersion()); final InventoryClass updatedInventory = strategy.deserializeInventory(child.getData()); @@ -364,7 +364,7 @@ public class CuratorInventoryManager case CHILD_REMOVED: { final ChildData child = event.getData(); final String inventoryKey = ZKPaths.getNodeFromPath(child.getPath()); - log.info("CHILD_REMOVED[%s] with version[%s]", inventoryKey, event.getData().getStat().getVersion()); + log.info("CHILD_REMOVED[%s] with version[%s]", child.getPath(), event.getData().getStat().getVersion()); synchronized (holder) { holder.setContainer(strategy.removeInventory(holder.getContainer(), inventoryKey)); From e166260f807bbd368f96f88fc195ac368f163e7b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 15 Dec 2014 13:19:47 -0800 Subject: [PATCH 26/71] more inventory logs --- .../druid/curator/inventory/CuratorInventoryManager.java | 9 +++++---- .../server/coordination/BatchDataSegmentAnnouncer.java | 1 + 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java index 444d46ad386..aaa4409221c 100644 --- a/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java +++ b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java @@ -263,12 +263,13 @@ public class CuratorInventoryManager final ContainerClass container = strategy.deserializeContainer(child.getData()); - ContainerHolder oldContainer = containers.get(containerKey); - if (oldContainer == null) { + log.info("Container[%s] updated.", child.getPath()); + ContainerHolder holder = containers.get(containerKey); + if (holder == null) { log.warn("Container update[%s], but the old container didn't exist!? Ignoring.", child.getPath()); } else { - synchronized (oldContainer) { - oldContainer.setContainer(strategy.updateContainer(oldContainer.getContainer(), container)); + synchronized (holder) { + holder.setContainer(strategy.updateContainer(holder.getContainer(), container)); } } diff --git a/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncer.java b/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncer.java index d0b24ea5653..620f75c6614 100644 --- a/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncer.java +++ b/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncer.java @@ -121,6 +121,7 @@ public class BatchDataSegmentAnnouncer extends AbstractDataSegmentAnnouncer { final SegmentZNode segmentZNode = segmentLookup.remove(segment); if (segmentZNode == null) { + log.warn("No path to unannounce segment[%s]", segment.getIdentifier()); return; } From e8729523902ffbebacd502bd4ea73099d6224f16 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 15 Dec 2014 14:51:58 -0800 Subject: [PATCH 27/71] fix working path default bug --- .../indexer/HadoopDruidIndexerConfig.java | 48 ++++++++++++------- .../io/druid/indexer/HadoopTuningConfig.java | 5 +- .../main/java/io/druid/indexer/JobHelper.java | 2 +- .../indexing/common/task/HadoopIndexTask.java | 4 +- 4 files changed, 34 insertions(+), 25 deletions(-) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index f7b168d0f3f..8638e53b1d4 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -83,6 +83,8 @@ public class HadoopDruidIndexerConfig public static final Joiner tabJoiner = Joiner.on("\t"); public static final ObjectMapper jsonMapper; + private static final String DEFAULT_WORKING_PATH = "/tmp/druid-indexing"; + static { injector = Initialization.makeInjectorWithModules( GuiceInjectors.makeStartupInjector(), @@ -116,8 +118,8 @@ public class HadoopDruidIndexerConfig { // Eventually PathSpec needs to get rid of its Hadoop dependency, then maybe this can be ingested directly without // the Map<> intermediary - - if(argSpec.containsKey("spec")){ + + if (argSpec.containsKey("spec")) { return HadoopDruidIndexerConfig.jsonMapper.convertValue( argSpec, HadoopDruidIndexerConfig.class @@ -138,8 +140,8 @@ public class HadoopDruidIndexerConfig return fromMap( (Map) HadoopDruidIndexerConfig.jsonMapper.readValue( file, new TypeReference>() - { - } + { + } ) ); } @@ -175,7 +177,7 @@ public class HadoopDruidIndexerConfig private volatile HadoopIngestionSpec schema; private volatile PathSpec pathSpec; - private volatile Map shardSpecLookups = Maps.newHashMap(); + private volatile Map shardSpecLookups = Maps.newHashMap(); private volatile Map hadoopShardSpecLookup = Maps.newHashMap(); private final QueryGranularity rollupGran; @@ -193,17 +195,17 @@ public class HadoopDruidIndexerConfig final ShardSpec actualSpec = entry.getValue().get(0).getActualSpec(); shardSpecLookups.put( entry.getKey(), actualSpec.getLookup( - Lists.transform( - entry.getValue(), new Function() - { - @Override - public ShardSpec apply(HadoopyShardSpec input) - { - return input.getActualSpec(); - } - } + Lists.transform( + entry.getValue(), new Function() + { + @Override + public ShardSpec apply(HadoopyShardSpec input) + { + return input.getActualSpec(); + } + } + ) ) - ) ); for (HadoopyShardSpec hadoopyShardSpec : entry.getValue()) { hadoopShardSpecLookup.put(hadoopyShardSpec.getActualSpec(), hadoopyShardSpec); @@ -212,7 +214,7 @@ public class HadoopDruidIndexerConfig this.rollupGran = schema.getDataSchema().getGranularitySpec().getQueryGranularity(); } - @JsonProperty(value="spec") + @JsonProperty(value = "spec") public HadoopIngestionSpec getSchema() { return schema; @@ -333,7 +335,11 @@ public class HadoopDruidIndexerConfig return Optional.absent(); } - final ShardSpec actualSpec = shardSpecLookups.get(timeBucket.get().getStart()).getShardSpec(rollupGran.truncate(inputRow.getTimestampFromEpoch()), inputRow); + final ShardSpec actualSpec = shardSpecLookups.get(timeBucket.get().getStart()) + .getShardSpec( + rollupGran.truncate(inputRow.getTimestampFromEpoch()), + inputRow + ); final HadoopyShardSpec hadoopyShardSpec = hadoopShardSpecLookup.get(actualSpec); return Optional.of( @@ -403,6 +409,12 @@ public class HadoopDruidIndexerConfig return schema.getTuningConfig().isPersistInHeap(); } + public String getWorkingPath() + { + final String workingPath = schema.getTuningConfig().getWorkingPath(); + return workingPath == null ? DEFAULT_WORKING_PATH : workingPath; + } + /****************************************** Path helper logic ******************************************/ @@ -418,7 +430,7 @@ public class HadoopDruidIndexerConfig return new Path( String.format( "%s/%s/%s", - schema.getTuningConfig().getWorkingPath(), + getWorkingPath(), schema.getDataSchema().getDataSource(), schema.getTuningConfig().getVersion().replace(":", "") ) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java index 9a2787780bc..382d24f0b5b 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java @@ -36,7 +36,6 @@ import java.util.Map; @JsonTypeName("hadoop") public class HadoopTuningConfig implements TuningConfig { - private static final String DEFAULT_WORKING_PATH = "/tmp/druid-indexing"; private static final PartitionsSpec DEFAULT_PARTITIONS_SPEC = HashedPartitionsSpec.makeDefaultHashedPartitionsSpec(); private static final Map> DEFAULT_SHARD_SPECS = ImmutableMap.>of(); private static final int DEFAULT_ROW_FLUSH_BOUNDARY = 80000; @@ -46,7 +45,7 @@ public class HadoopTuningConfig implements TuningConfig public static HadoopTuningConfig makeDefaultTuningConfig() { return new HadoopTuningConfig( - DEFAULT_WORKING_PATH, + null, new DateTime().toString(), DEFAULT_PARTITIONS_SPEC, DEFAULT_SHARD_SPECS, @@ -99,7 +98,7 @@ public class HadoopTuningConfig implements TuningConfig final @JsonProperty("aggregationBufferRatio") Float aggregationBufferRatio ) { - this.workingPath = workingPath == null ? DEFAULT_WORKING_PATH : workingPath; + this.workingPath = workingPath; this.version = version == null ? new DateTime().toString() : version; this.partitionsSpec = partitionsSpec == null ? DEFAULT_PARTITIONS_SPEC : partitionsSpec; this.shardSpecs = shardSpecs == null ? DEFAULT_SHARD_SPECS : shardSpecs; diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java index b1ffabaa1f4..fa0d9b9772d 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java @@ -63,7 +63,7 @@ public class JobHelper final Configuration conf = groupByJob.getConfiguration(); final FileSystem fs = FileSystem.get(conf); - Path distributedClassPath = new Path(config.getSchema().getTuningConfig().getWorkingPath(), "classpath"); + Path distributedClassPath = new Path(config.getWorkingPath(), "classpath"); if (fs instanceof LocalFileSystem) { return; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java index 086aef984b4..e27d70648b1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java @@ -111,13 +111,11 @@ public class HadoopIndexTask extends AbstractTask this.spec = spec; // Some HadoopIngestionSpec stuff doesn't make sense in the context of the indexing service - if (this.spec.getTuningConfig().getWorkingPath() != null) { - log.error("workingPath should be absent in your spec! Ignoring"); - } Preconditions.checkArgument( this.spec.getIOConfig().getSegmentOutputPath() == null, "segmentOutputPath must be absent" ); + Preconditions.checkArgument(this.spec.getTuningConfig().getWorkingPath() == null, "workingPath must be absent"); Preconditions.checkArgument( this.spec.getIOConfig().getMetadataUpdateSpec() == null, "metadataUpdateSpec must be absent" From ac407fb6baffa0264f902fb621a4f4cccfaf05cc Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 15 Dec 2014 15:05:02 -0800 Subject: [PATCH 28/71] clean up defaults --- docs/content/Indexing-Service-Config.md | 4 ++-- .../autoscaling/ResourceManagementSchedulerConfig.java | 2 +- .../overlord/autoscaling/SimpleResourceManagementConfig.java | 2 +- services/src/main/java/io/druid/guice/RealtimeModule.java | 3 +-- 4 files changed, 5 insertions(+), 6 deletions(-) diff --git a/docs/content/Indexing-Service-Config.md b/docs/content/Indexing-Service-Config.md index 795953435d1..70a2f28b541 100644 --- a/docs/content/Indexing-Service-Config.md +++ b/docs/content/Indexing-Service-Config.md @@ -78,9 +78,9 @@ There are additional configs for autoscaling (if it is enabled): |`druid.indexer.autoscale.strategy`|Choices are "noop" or "ec2". Sets the strategy to run when autoscaling is required.|noop| |`druid.indexer.autoscale.doAutoscale`|If set to "true" autoscaling will be enabled.|false| |`druid.indexer.autoscale.provisionPeriod`|How often to check whether or not new middle managers should be added.|PT1M| -|`druid.indexer.autoscale.terminatePeriod`|How often to check when middle managers should be removed.|PT1H| +|`druid.indexer.autoscale.terminatePeriod`|How often to check when middle managers should be removed.|PT5M| |`druid.indexer.autoscale.originTime`|The starting reference timestamp that the terminate period increments upon.|2012-01-01T00:55:00.000Z| -|`druid.indexer.autoscale.workerIdleTimeout`|How long can a worker be idle (not a run task) before it can be considered for termination.|PT10M| +|`druid.indexer.autoscale.workerIdleTimeout`|How long can a worker be idle (not a run task) before it can be considered for termination.|PT90M| |`druid.indexer.autoscale.maxScalingDuration`|How long the overlord will wait around for a middle manager to show up before giving up.|PT15M| |`druid.indexer.autoscale.numEventsToTrack`|The number of autoscaling related events (node creation and termination) to track.|10| |`druid.indexer.autoscale.pendingTaskTimeout`|How long a task can be in "pending" state before the overlord tries to scale up.|PT30S| diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/ResourceManagementSchedulerConfig.java b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/ResourceManagementSchedulerConfig.java index f84c4d1f024..242c20d0f2b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/ResourceManagementSchedulerConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/ResourceManagementSchedulerConfig.java @@ -34,7 +34,7 @@ public class ResourceManagementSchedulerConfig private Period provisionPeriod = new Period("PT1M"); @JsonProperty - private Period terminatePeriod = new Period("PT1H"); + private Period terminatePeriod = new Period("PT5M"); @JsonProperty private DateTime originTime = new DateTime("2012-01-01T00:55:00.000Z"); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/SimpleResourceManagementConfig.java b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/SimpleResourceManagementConfig.java index 85913e00cbc..1eb9479e1f4 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/SimpleResourceManagementConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/SimpleResourceManagementConfig.java @@ -27,7 +27,7 @@ import org.joda.time.Period; public class SimpleResourceManagementConfig { @JsonProperty - private Period workerIdleTimeout = new Period("PT10m"); + private Period workerIdleTimeout = new Period("PT90m"); @JsonProperty private Period maxScalingDuration = new Period("PT15M"); diff --git a/services/src/main/java/io/druid/guice/RealtimeModule.java b/services/src/main/java/io/druid/guice/RealtimeModule.java index 6c3ed06ff5c..f59c2cec435 100644 --- a/services/src/main/java/io/druid/guice/RealtimeModule.java +++ b/services/src/main/java/io/druid/guice/RealtimeModule.java @@ -26,12 +26,11 @@ import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; import io.druid.cli.QueryJettyServerInitializer; import io.druid.metadata.MetadataSegmentPublisher; -import io.druid.metadata.SQLMetadataSegmentPublisher; import io.druid.query.QuerySegmentWalker; -import io.druid.segment.realtime.SegmentPublisher; import io.druid.segment.realtime.FireDepartment; import io.druid.segment.realtime.NoopSegmentPublisher; import io.druid.segment.realtime.RealtimeManager; +import io.druid.segment.realtime.SegmentPublisher; import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.segment.realtime.firehose.ChatHandlerResource; import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; From 54068e8b1d6ddee5546424182b40ed9c161e8dae Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 15 Dec 2014 15:24:59 -0800 Subject: [PATCH 29/71] Remove try-with-resources for log stream in WokerResource --- .../java/io/druid/indexing/overlord/RemoteTaskRunner.java | 2 -- .../java/io/druid/indexing/worker/http/WorkerResource.java | 7 ++++--- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index 1497c7d93e0..355f018ea89 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -32,8 +32,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.io.ByteSource; -import com.google.common.io.ByteStreams; -import com.google.common.io.InputSupplier; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java b/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java index dbecb5361c3..7992fd710c8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java @@ -41,6 +41,7 @@ import javax.ws.rs.Produces; import javax.ws.rs.QueryParam; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import java.io.IOException; import java.io.InputStream; /** @@ -167,10 +168,10 @@ public class WorkerResource final Optional stream = taskRunner.streamTaskLog(taskid, offset); if (stream.isPresent()) { - try (InputStream logStream = stream.get().openStream()) { - return Response.ok(logStream).build(); + try { + return Response.ok(stream.get().openStream()).build(); } - catch (Exception e) { + catch (IOException e) { log.warn(e, "Failed to read log for task: %s", taskid); return Response.serverError().build(); } From 4f0c0c5ae2e3f42a34aa97d5a49fc7bc46d80d1f Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 15 Dec 2014 16:04:19 -0800 Subject: [PATCH 30/71] i hate guice --- services/src/main/java/io/druid/cli/CliPeon.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index beb206c8eca..3ee81cede38 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -125,6 +125,8 @@ public class CliPeon extends GuiceRunnable .to(ServiceAnnouncingChatHandlerProvider.class).in(LazySingleton.class); handlerProviderBinder.addBinding("noop") .to(NoopChatHandlerProvider.class).in(LazySingleton.class); + binder.bind(ServiceAnnouncingChatHandlerProvider.class).in(LazySingleton.class);; + binder.bind(NoopChatHandlerProvider.class).in(LazySingleton.class); binder.bind(TaskToolboxFactory.class).in(LazySingleton.class); From 43d27ddaf07834dc73da43c85ea09a2f9384f0f3 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 15 Dec 2014 16:59:57 -0800 Subject: [PATCH 31/71] update http client and fix logging --- .../src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java | 4 +++- pom.xml | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java index 49ea3654738..467a2e76b55 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java @@ -77,7 +77,9 @@ public class HadoopDruidIndexerJob implements Jobby if (metadataStorageUpdaterJob != null) { jobs.add(metadataStorageUpdaterJob); } else { - log.info("No updaterJobSpec set, not uploading to database"); + log.info( + "No metadataStorageUpdaterJob set in the config. This is cool if you are running a hadoop index task, otherwise nothing will be uploaded to database." + ); } jobs.add( diff --git a/pom.xml b/pom.xml index 375741dedfc..939770ab5de 100644 --- a/pom.xml +++ b/pom.xml @@ -84,7 +84,7 @@ com.metamx http-client - 0.9.7 + 0.9.12 com.metamx From d1fcd4e92e16e4d05deb10af3a3bb0ff4f03d233 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 15 Dec 2014 17:39:19 -0800 Subject: [PATCH 32/71] Add log4j in services --- services/src/main/resources/log4j.xml | 35 +++++++++++++++++++++++++++ 1 file changed, 35 insertions(+) create mode 100644 services/src/main/resources/log4j.xml diff --git a/services/src/main/resources/log4j.xml b/services/src/main/resources/log4j.xml new file mode 100644 index 00000000000..621c7a2ac9a --- /dev/null +++ b/services/src/main/resources/log4j.xml @@ -0,0 +1,35 @@ + + + + + + + + + + + + + + + + + + \ No newline at end of file From 242edd14e55f54a573beee2c0deafabd0a247f4e Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 15 Dec 2014 17:59:32 -0800 Subject: [PATCH 33/71] Add deprecated smile media type for backwards compatability during upgrades --- server/src/main/java/io/druid/server/QueryResource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 318c0d4df15..5f17609c962 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -121,7 +121,7 @@ public class QueryResource @POST @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) - @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE, APPLICATION_SMILE}) public Response doPost( InputStream in, @QueryParam("pretty") String pretty, From e9238161cdfe4fa5063a135652514781d16febd6 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 15 Dec 2014 18:12:44 -0800 Subject: [PATCH 34/71] fix broken test --- .../server/initialization/JettyTest.java | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/server/src/test/java/io/druid/server/initialization/JettyTest.java b/server/src/test/java/io/druid/server/initialization/JettyTest.java index 80a7b7cd935..40b1d4b7c9e 100644 --- a/server/src/test/java/io/druid/server/initialization/JettyTest.java +++ b/server/src/test/java/io/druid/server/initialization/JettyTest.java @@ -23,13 +23,14 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Binder; -import com.google.inject.Inject; import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.servlet.GuiceFilter; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.http.client.HttpClient; +import com.metamx.http.client.HttpClientConfig; +import com.metamx.http.client.HttpClientInit; import com.metamx.http.client.response.InputStreamResponseHandler; import com.metamx.http.client.response.StatusResponseHandler; import com.metamx.http.client.response.StatusResponseHolder; @@ -38,7 +39,6 @@ import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.guice.LifecycleModule; -import io.druid.guice.annotations.Global; import io.druid.guice.annotations.Self; import io.druid.initialization.Initialization; import io.druid.server.DruidNode; @@ -49,12 +49,14 @@ import org.eclipse.jetty.server.handler.HandlerList; import org.eclipse.jetty.servlet.DefaultServlet; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; +import org.joda.time.Duration; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Ignore; import org.junit.Test; +import javax.net.ssl.SSLContext; import javax.servlet.ServletOutputStream; import javax.servlet.http.HttpServletResponse; import javax.ws.rs.GET; @@ -259,10 +261,17 @@ public class JettyTest { HttpClient client; - @Inject - ClientHolder(@Global HttpClient client) + ClientHolder() { - this.client = client; + try { + this.client = HttpClientInit.createClient( + new HttpClientConfig(1, SSLContext.getDefault(), Duration.ZERO), + new Lifecycle() + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } } public HttpClient getClient() From 714317a492aed6e2d5b3915a9fc05a4307ada4b2 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 15 Dec 2014 19:16:37 -0700 Subject: [PATCH 35/71] Revert "Support more AWS credentials providers for S3 storage" --- .../storage/s3/S3StorageDruidModule.java | 77 +++---------------- pom.xml | 2 +- 2 files changed, 13 insertions(+), 66 deletions(-) diff --git a/extensions/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java b/extensions/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java index 792136ae8b9..3d2434365ae 100644 --- a/extensions/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java +++ b/extensions/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java @@ -19,13 +19,7 @@ package io.druid.storage.s3; -import com.amazonaws.AmazonClientException; import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.auth.AWSCredentialsProviderChain; -import com.amazonaws.auth.EnvironmentVariableCredentialsProvider; -import com.amazonaws.auth.InstanceProfileCredentialsProvider; -import com.amazonaws.auth.SystemPropertiesCredentialsProvider; -import com.amazonaws.auth.profile.ProfileCredentialsProvider; import com.fasterxml.jackson.databind.Module; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; @@ -68,18 +62,14 @@ public class S3StorageDruidModule implements DruidModule binder.bind(S3TaskLogs.class).in(LazySingleton.class); } - private static class ConfigDrivenAwsCredentialsConfigProvider implements AWSCredentialsProvider + @Provides + @LazySingleton + public AWSCredentialsProvider getAWSCredentialsProvider(final AWSCredentialsConfig config) { - private AWSCredentialsConfig config; - - public ConfigDrivenAwsCredentialsConfigProvider(AWSCredentialsConfig config) { - this.config = config; - } - - @Override - public com.amazonaws.auth.AWSCredentials getCredentials() - { - if (!Strings.isNullOrEmpty(config.getAccessKey()) && !Strings.isNullOrEmpty(config.getSecretKey())) { + if (!Strings.isNullOrEmpty(config.getAccessKey()) && !Strings.isNullOrEmpty(config.getSecretKey())) { + return new AWSCredentialsProvider() { + @Override + public com.amazonaws.auth.AWSCredentials getCredentials() { return new com.amazonaws.auth.AWSCredentials() { @Override public String getAWSAccessKeyId() { @@ -92,57 +82,14 @@ public class S3StorageDruidModule implements DruidModule } }; } - throw new AmazonClientException("Unable to load AWS credentials from druid AWSCredentialsConfig"); - } - - @Override - public void refresh() {} - } - - private static class LazyFileSessionCredentialsProvider implements AWSCredentialsProvider - { - private AWSCredentialsConfig config; - private FileSessionCredentialsProvider provider; - - public LazyFileSessionCredentialsProvider(AWSCredentialsConfig config) { - this.config = config; - } - - private FileSessionCredentialsProvider getUnderlyingProvider() { - if (provider == null) { - synchronized (config) { - if (provider == null) { - provider = new FileSessionCredentialsProvider(config.getFileSessionCredentials()); - } - } - } - return provider; - } - - @Override - public com.amazonaws.auth.AWSCredentials getCredentials() - { - return getUnderlyingProvider().getCredentials(); - } - @Override - public void refresh() { - getUnderlyingProvider().refresh(); + @Override + public void refresh() {} + }; + } else { + return new FileSessionCredentialsProvider(config.getFileSessionCredentials()); } } - - @Provides - @LazySingleton - public AWSCredentialsProvider getAWSCredentialsProvider(final AWSCredentialsConfig config) - { - return new AWSCredentialsProviderChain( - new ConfigDrivenAwsCredentialsConfigProvider(config), - new LazyFileSessionCredentialsProvider(config), - new EnvironmentVariableCredentialsProvider(), - new SystemPropertiesCredentialsProvider(), - new ProfileCredentialsProvider(), - new InstanceProfileCredentialsProvider()); - } @Provides @LazySingleton diff --git a/pom.xml b/pom.xml index 939770ab5de..dd85ddcd820 100644 --- a/pom.xml +++ b/pom.xml @@ -135,7 +135,7 @@ com.amazonaws aws-java-sdk - 1.8.11 + 1.6.0.1 javax.mail From d1060963275ae1036b67bac7c5c31cf454444832 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 15 Dec 2014 19:27:28 -0700 Subject: [PATCH 36/71] Revert "Revert "Support more AWS credentials providers for S3 storage"" --- .../storage/s3/S3StorageDruidModule.java | 79 ++++++++++++++++--- pom.xml | 2 +- 2 files changed, 67 insertions(+), 14 deletions(-) diff --git a/extensions/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java b/extensions/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java index 3d2434365ae..792136ae8b9 100644 --- a/extensions/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java +++ b/extensions/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java @@ -19,7 +19,13 @@ package io.druid.storage.s3; +import com.amazonaws.AmazonClientException; import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.AWSCredentialsProviderChain; +import com.amazonaws.auth.EnvironmentVariableCredentialsProvider; +import com.amazonaws.auth.InstanceProfileCredentialsProvider; +import com.amazonaws.auth.SystemPropertiesCredentialsProvider; +import com.amazonaws.auth.profile.ProfileCredentialsProvider; import com.fasterxml.jackson.databind.Module; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; @@ -62,14 +68,18 @@ public class S3StorageDruidModule implements DruidModule binder.bind(S3TaskLogs.class).in(LazySingleton.class); } - @Provides - @LazySingleton - public AWSCredentialsProvider getAWSCredentialsProvider(final AWSCredentialsConfig config) + private static class ConfigDrivenAwsCredentialsConfigProvider implements AWSCredentialsProvider { - if (!Strings.isNullOrEmpty(config.getAccessKey()) && !Strings.isNullOrEmpty(config.getSecretKey())) { - return new AWSCredentialsProvider() { - @Override - public com.amazonaws.auth.AWSCredentials getCredentials() { + private AWSCredentialsConfig config; + + public ConfigDrivenAwsCredentialsConfigProvider(AWSCredentialsConfig config) { + this.config = config; + } + + @Override + public com.amazonaws.auth.AWSCredentials getCredentials() + { + if (!Strings.isNullOrEmpty(config.getAccessKey()) && !Strings.isNullOrEmpty(config.getSecretKey())) { return new com.amazonaws.auth.AWSCredentials() { @Override public String getAWSAccessKeyId() { @@ -82,13 +92,56 @@ public class S3StorageDruidModule implements DruidModule } }; } - - @Override - public void refresh() {} - }; - } else { - return new FileSessionCredentialsProvider(config.getFileSessionCredentials()); + throw new AmazonClientException("Unable to load AWS credentials from druid AWSCredentialsConfig"); } + + @Override + public void refresh() {} + } + + private static class LazyFileSessionCredentialsProvider implements AWSCredentialsProvider + { + private AWSCredentialsConfig config; + private FileSessionCredentialsProvider provider; + + public LazyFileSessionCredentialsProvider(AWSCredentialsConfig config) { + this.config = config; + } + + private FileSessionCredentialsProvider getUnderlyingProvider() { + if (provider == null) { + synchronized (config) { + if (provider == null) { + provider = new FileSessionCredentialsProvider(config.getFileSessionCredentials()); + } + } + } + return provider; + } + + @Override + public com.amazonaws.auth.AWSCredentials getCredentials() + { + return getUnderlyingProvider().getCredentials(); + } + + @Override + public void refresh() { + getUnderlyingProvider().refresh(); + } + } + + @Provides + @LazySingleton + public AWSCredentialsProvider getAWSCredentialsProvider(final AWSCredentialsConfig config) + { + return new AWSCredentialsProviderChain( + new ConfigDrivenAwsCredentialsConfigProvider(config), + new LazyFileSessionCredentialsProvider(config), + new EnvironmentVariableCredentialsProvider(), + new SystemPropertiesCredentialsProvider(), + new ProfileCredentialsProvider(), + new InstanceProfileCredentialsProvider()); } @Provides diff --git a/pom.xml b/pom.xml index 26b26ad37be..375741dedfc 100644 --- a/pom.xml +++ b/pom.xml @@ -135,7 +135,7 @@ com.amazonaws aws-java-sdk - 1.6.0.1 + 1.8.11 javax.mail From c0af7b2a5d5d9cf2755f420f91d755a34763626c Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 15 Dec 2014 18:29:15 -0800 Subject: [PATCH 37/71] fix broken ut --- {services => common}/src/main/resources/log4j.xml | 0 pom.xml | 2 +- 2 files changed, 1 insertion(+), 1 deletion(-) rename {services => common}/src/main/resources/log4j.xml (100%) diff --git a/services/src/main/resources/log4j.xml b/common/src/main/resources/log4j.xml similarity index 100% rename from services/src/main/resources/log4j.xml rename to common/src/main/resources/log4j.xml diff --git a/pom.xml b/pom.xml index dd85ddcd820..26b26ad37be 100644 --- a/pom.xml +++ b/pom.xml @@ -84,7 +84,7 @@ com.metamx http-client - 0.9.12 + 0.9.7 com.metamx From b3999bbc6a369886886d26a8881a9884329c1c58 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Dec 2014 11:29:50 -0800 Subject: [PATCH 38/71] update http client and fix log4j dependencies --- common/pom.xml | 300 ++++++++++++++++--------------- extensions/s3-extensions/pom.xml | 166 ++++++++--------- pom.xml | 2 +- 3 files changed, 238 insertions(+), 230 deletions(-) diff --git a/common/pom.xml b/common/pom.xml index cf835d37215..029f61d6892 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -20,157 +20,161 @@ - 4.0.0 - io.druid - druid-common - druid-common - druid-common - - + 4.0.0 io.druid - druid - 0.7.0-SNAPSHOT - + druid-common + druid-common + druid-common - - - com.metamx - java-util - - - io.druid - druid-api - + + io.druid + druid + 0.7.0-SNAPSHOT + - - commons-codec - commons-codec - - - org.apache.commons - commons-dbcp2 - - - commons-pool - commons-pool - 1.6 - - - org.skife.config - config-magic - - - org.hibernate - hibernate-validator - - - javax.validation - validation-api - - - com.google.guava - guava - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.core - jackson-annotations - - - com.fasterxml.jackson.core - jackson-databind - - - com.fasterxml.jackson.datatype - jackson-datatype-guava - - - com.fasterxml.jackson.datatype - jackson-datatype-joda - - - com.fasterxml.jackson.dataformat - jackson-dataformat-smile - - - com.google.inject - guice - - - com.google.inject.extensions - guice-multibindings - - - org.jdbi - jdbi - - - joda-time - joda-time - - - com.google.code.findbugs - jsr305 - - - log4j - log4j - + + + com.metamx + java-util + + + io.druid + druid-api + - - - junit - junit - test - - - org.easymock - easymock - test - - - com.metamx - java-util - test-jar - test - - + + commons-codec + commons-codec + + + org.apache.commons + commons-dbcp2 + + + commons-pool + commons-pool + 1.6 + + + org.skife.config + config-magic + + + org.hibernate + hibernate-validator + + + javax.validation + validation-api + + + com.google.guava + guava + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.datatype + jackson-datatype-guava + + + com.fasterxml.jackson.datatype + jackson-datatype-joda + + + com.fasterxml.jackson.dataformat + jackson-dataformat-smile + + + com.google.inject + guice + + + com.google.inject.extensions + guice-multibindings + + + org.jdbi + jdbi + + + joda-time + joda-time + + + com.google.code.findbugs + jsr305 + + + log4j + log4j + - - - - org.apache.maven.plugins - maven-source-plugin - - - attach-sources - - jar - - - - - - maven-jar-plugin - - - - test-jar - - - - - - - true - true - - - - - - + + + junit + junit + test + + + org.easymock + easymock + test + + + com.metamx + java-util + test-jar + test + + + log4j + log4j + + + + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + maven-jar-plugin + + + + test-jar + + + + + + + true + true + + + + + + diff --git a/extensions/s3-extensions/pom.xml b/extensions/s3-extensions/pom.xml index 8bdbb9f5be0..81533e3b171 100644 --- a/extensions/s3-extensions/pom.xml +++ b/extensions/s3-extensions/pom.xml @@ -20,88 +20,92 @@ - 4.0.0 - io.druid.extensions - druid-s3-extensions - druid-s3-extensions - druid-s3-extensions + 4.0.0 + io.druid.extensions + druid-s3-extensions + druid-s3-extensions + druid-s3-extensions - - io.druid - druid - 0.7.0-SNAPSHOT - ../../pom.xml - + + io.druid + druid + 0.7.0-SNAPSHOT + ../../pom.xml + - - - io.druid - druid-api - - - - net.java.dev.jets3t - jets3t - - - com.amazonaws - aws-java-sdk - - - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - - - com.metamx - emitter - - - commons-io - commons-io - + + + io.druid + druid-api + + + + net.java.dev.jets3t + jets3t + + + com.amazonaws + aws-java-sdk + + + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + + + com.metamx + emitter + + + commons-io + commons-io + - - - junit - junit - test - - - org.easymock - easymock - test - - - - - - org.apache.maven.plugins - maven-source-plugin - - - attach-sources - - jar - - - - - - maven-jar-plugin - - - - true - true - - - - - - + + + junit + junit + test + + + org.easymock + easymock + test + + + log4j + log4j + + + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + maven-jar-plugin + + + + true + true + + + + + + diff --git a/pom.xml b/pom.xml index 375741dedfc..939770ab5de 100644 --- a/pom.xml +++ b/pom.xml @@ -84,7 +84,7 @@ com.metamx http-client - 0.9.7 + 0.9.12 com.metamx From 882874ce602ef3a3045ffdb27bcdef20af78db48 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Dec 2014 11:37:38 -0800 Subject: [PATCH 39/71] address cr --- common/pom.xml | 4 ---- extensions/s3-extensions/pom.xml | 1 + 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/common/pom.xml b/common/pom.xml index 029f61d6892..11f8ae8ee4b 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -137,10 +137,6 @@ test-jar test - - log4j - log4j - diff --git a/extensions/s3-extensions/pom.xml b/extensions/s3-extensions/pom.xml index 81533e3b171..30962948b0a 100644 --- a/extensions/s3-extensions/pom.xml +++ b/extensions/s3-extensions/pom.xml @@ -79,6 +79,7 @@ log4j log4j + test From 76cd7e0f56a1dad626d25bb7a6761b6a9840add0 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Dec 2014 11:45:48 -0800 Subject: [PATCH 40/71] update druid api --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 939770ab5de..2728d7b3023 100644 --- a/pom.xml +++ b/pom.xml @@ -43,7 +43,7 @@ 0.26.9 2.7.0 9.2.5.v20141112 - 0.3.0 + 0.3.1 2.4.4 From a0b2a63d7fe9e3f23a378ca84d17fbed35eccee7 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 16 Dec 2014 18:32:41 -0800 Subject: [PATCH 41/71] Fix deadlock in DirectDruidClient --- .../java/io/druid/client/DirectDruidClient.java | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index 3e562457248..67613773ac9 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -214,15 +214,12 @@ public class DirectDruidClient implements QueryRunner @Override public InputStream nextElement() { - synchronized (done) { - try { - // Ensures more elements are expected via `done` - return queue.take(); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw Throwables.propagate(e); - } + try { + return queue.take(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); } } } From a637a23eaec2d881c6f65860b54e283b28d9762d Mon Sep 17 00:00:00 2001 From: flow Date: Thu, 18 Dec 2014 19:09:13 +0800 Subject: [PATCH 42/71] fix issue #977 --- .../java/io/druid/storage/hdfs/tasklog/HdfsTaskLogs.java | 4 ++-- .../io/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java | 6 ------ 2 files changed, 2 insertions(+), 8 deletions(-) diff --git a/extensions/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogs.java b/extensions/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogs.java index e669b307e75..e8002bac235 100644 --- a/extensions/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogs.java +++ b/extensions/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogs.java @@ -56,7 +56,7 @@ public class HdfsTaskLogs implements TaskLogs final Path path = getTaskLogFileFromId(taskId); log.info("Writing task log to: %s", path); Configuration conf = new Configuration(); - final FileSystem fs = FileSystem.get(conf); + final FileSystem fs = path.getFileSystem(conf); FileUtil.copy(logFile, fs, path, false, conf); log.info("Wrote task log to: %s", path); } @@ -65,7 +65,7 @@ public class HdfsTaskLogs implements TaskLogs public Optional streamTaskLog(final String taskId, final long offset) throws IOException { final Path path = getTaskLogFileFromId(taskId); - final FileSystem fs = FileSystem.get(new Configuration()); + final FileSystem fs = path.getFileSystem(new Configuration()); if (fs.exists(path)) { return Optional.of( new ByteSource() diff --git a/extensions/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java b/extensions/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java index 6e2c67734fb..d8df81cfb50 100644 --- a/extensions/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java +++ b/extensions/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java @@ -27,16 +27,10 @@ import javax.validation.constraints.NotNull; */ public class HdfsTaskLogsConfig { - @JsonProperty @NotNull private String directory; - public HdfsTaskLogsConfig(String directory) - { - this.directory = directory; - } - public String getDirectory() { return directory; From f13eab644ad5f9616d3233e0354bb6ff3e98c91b Mon Sep 17 00:00:00 2001 From: flow Date: Fri, 19 Dec 2014 10:48:39 +0800 Subject: [PATCH 43/71] Keep HdfsTaskLogsConfig creator --- .../io/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/extensions/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java b/extensions/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java index d8df81cfb50..21b427249dd 100644 --- a/extensions/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java +++ b/extensions/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java @@ -18,6 +18,7 @@ */ package io.druid.storage.hdfs.tasklog; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import javax.validation.constraints.NotNull; @@ -31,6 +32,12 @@ public class HdfsTaskLogsConfig @NotNull private String directory; + @JsonCreator + public HdfsTaskLogsConfig(@JsonProperty("directory") String directory) + { + this.directory = directory; + } + public String getDirectory() { return directory; From 7c8d4a7433010e2b9c4c857ac27a87b2f72c3501 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 18 Dec 2014 20:45:26 -0800 Subject: [PATCH 44/71] Use Module instead of generic Object in Guice related items --- .../indexer/HadoopDruidIndexerConfig.java | 2 +- .../java/io/druid/guice/GuiceInjectors.java | 31 +++++++------------ .../druid/initialization/Initialization.java | 2 +- .../initialization/InitializationTest.java | 2 +- .../server/initialization/JettyTest.java | 2 +- .../src/main/java/io/druid/cli/CliBridge.java | 4 +-- .../src/main/java/io/druid/cli/CliBroker.java | 4 +-- .../java/io/druid/cli/CliCoordinator.java | 4 +-- .../main/java/io/druid/cli/CliHistorical.java | 4 +-- .../druid/cli/CliInternalHadoopIndexer.java | 4 +-- .../java/io/druid/cli/CliMiddleManager.java | 4 +-- .../main/java/io/druid/cli/CliOverlord.java | 4 +-- .../src/main/java/io/druid/cli/CliPeon.java | 4 +-- .../main/java/io/druid/cli/CliRealtime.java | 4 +-- .../java/io/druid/cli/CliRealtimeExample.java | 4 +-- .../src/main/java/io/druid/cli/CliRouter.java | 4 +-- .../main/java/io/druid/cli/CreateTables.java | 4 +-- .../main/java/io/druid/cli/GuiceRunnable.java | 3 +- 18 files changed, 41 insertions(+), 49 deletions(-) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index 8638e53b1d4..a7f296f8b79 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -88,7 +88,7 @@ public class HadoopDruidIndexerConfig static { injector = Initialization.makeInjectorWithModules( GuiceInjectors.makeStartupInjector(), - ImmutableList.of( + ImmutableList.of( new Module() { @Override diff --git a/processing/src/main/java/io/druid/guice/GuiceInjectors.java b/processing/src/main/java/io/druid/guice/GuiceInjectors.java index 9a3425c872f..ee527698b40 100644 --- a/processing/src/main/java/io/druid/guice/GuiceInjectors.java +++ b/processing/src/main/java/io/druid/guice/GuiceInjectors.java @@ -19,6 +19,7 @@ package io.druid.guice; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.inject.Binder; import com.google.inject.Guice; @@ -27,15 +28,16 @@ import com.google.inject.Module; import io.druid.jackson.JacksonModule; import java.util.Arrays; +import java.util.Collection; import java.util.List; /** */ public class GuiceInjectors { - public static Injector makeStartupInjector() + public static Collection makeDefaultStartupModules() { - return Guice.createInjector( + return ImmutableList.of( new DruidGuiceExtensions(), new JacksonModule(), new PropertiesModule(Arrays.asList("common.runtime.properties", "runtime.properties")), @@ -52,29 +54,18 @@ public class GuiceInjectors ); } - public static Injector makeStartupInjectorWithModules(Iterable modules) + public static Injector makeStartupInjector() + { + return Guice.createInjector(makeDefaultStartupModules()); + } + + public static Injector makeStartupInjectorWithModules(Iterable modules) { List theModules = Lists.newArrayList(); - theModules.add(new DruidGuiceExtensions()); - theModules.add(new JacksonModule()); - theModules.add(new PropertiesModule(Arrays.asList("common.runtime.properties", "runtime.properties"))); - theModules.add(new ConfigModule()); - theModules.add( - new Module() - { - @Override - public void configure(Binder binder) - { - binder.bind(DruidSecondaryModule.class); - JsonConfigProvider.bind(binder, "druid.extensions", ExtensionsConfig.class); - } - } - ); + theModules.addAll(makeDefaultStartupModules()); for (Module theModule : modules) { theModules.add(theModule); } - - return Guice.createInjector(theModules); } } diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index 058a007701d..4f9597c376b 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -328,7 +328,7 @@ public class Initialization } } - public static Injector makeInjectorWithModules(final Injector baseInjector, Iterable modules) + public static Injector makeInjectorWithModules(final Injector baseInjector, Iterable modules) { final ModuleList defaultModules = new ModuleList(baseInjector); defaultModules.addModules( diff --git a/server/src/test/java/io/druid/initialization/InitializationTest.java b/server/src/test/java/io/druid/initialization/InitializationTest.java index 2dce7260796..bab33ba6c56 100644 --- a/server/src/test/java/io/druid/initialization/InitializationTest.java +++ b/server/src/test/java/io/druid/initialization/InitializationTest.java @@ -102,7 +102,7 @@ public class InitializationTest { Injector startupInjector = GuiceInjectors.makeStartupInjector(); Injector injector = Initialization.makeInjectorWithModules( - startupInjector, ImmutableList.of( + startupInjector, ImmutableList.of( new com.google.inject.Module() { @Override diff --git a/server/src/test/java/io/druid/server/initialization/JettyTest.java b/server/src/test/java/io/druid/server/initialization/JettyTest.java index 40b1d4b7c9e..5b99d2890dc 100644 --- a/server/src/test/java/io/druid/server/initialization/JettyTest.java +++ b/server/src/test/java/io/druid/server/initialization/JettyTest.java @@ -97,7 +97,7 @@ public class JettyTest { setProperties(); Injector injector = Initialization.makeInjectorWithModules( - GuiceInjectors.makeStartupInjector(), ImmutableList.of( + GuiceInjectors.makeStartupInjector(), ImmutableList.of( new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/CliBridge.java b/services/src/main/java/io/druid/cli/CliBridge.java index 808b79bbb48..b49aa1f3bb4 100644 --- a/services/src/main/java/io/druid/cli/CliBridge.java +++ b/services/src/main/java/io/druid/cli/CliBridge.java @@ -62,9 +62,9 @@ public class CliBridge extends ServerRunnable } @Override - protected List getModules() + protected List getModules() { - return ImmutableList.of( + return ImmutableList.of( new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index e5ac925da02..10f5f5529c9 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -72,9 +72,9 @@ public class CliBroker extends ServerRunnable } @Override - protected List getModules() + protected List getModules() { - return ImmutableList.of( + return ImmutableList.of( new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 329348a101b..7fbe40ecee3 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -80,9 +80,9 @@ public class CliCoordinator extends ServerRunnable } @Override - protected List getModules() + protected List getModules() { - return ImmutableList.of( + return ImmutableList.of( new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 79c28b24c54..5da1bfde6a2 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -62,9 +62,9 @@ public class CliHistorical extends ServerRunnable } @Override - protected List getModules() + protected List getModules() { - return ImmutableList.of( + return ImmutableList.of( new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/CliInternalHadoopIndexer.java b/services/src/main/java/io/druid/cli/CliInternalHadoopIndexer.java index 9b310b35b08..56bf59e95c9 100644 --- a/services/src/main/java/io/druid/cli/CliInternalHadoopIndexer.java +++ b/services/src/main/java/io/druid/cli/CliInternalHadoopIndexer.java @@ -66,9 +66,9 @@ public class CliInternalHadoopIndexer extends GuiceRunnable } @Override - protected List getModules() + protected List getModules() { - return ImmutableList.of( + return ImmutableList.of( new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/CliMiddleManager.java b/services/src/main/java/io/druid/cli/CliMiddleManager.java index 6999b262f7a..da4976052f3 100644 --- a/services/src/main/java/io/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/io/druid/cli/CliMiddleManager.java @@ -67,9 +67,9 @@ public class CliMiddleManager extends ServerRunnable } @Override - protected List getModules() + protected List getModules() { - return ImmutableList.of( + return ImmutableList.of( new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index a5351dd8968..792fc4155b7 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -105,9 +105,9 @@ public class CliOverlord extends ServerRunnable } @Override - protected List getModules() + protected List getModules() { - return ImmutableList.of( + return ImmutableList.of( new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index 3ee81cede38..25632cdeb5d 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -101,9 +101,9 @@ public class CliPeon extends GuiceRunnable } @Override - protected List getModules() + protected List getModules() { - return ImmutableList.of( + return ImmutableList.of( new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index a34e3bd0eda..4ac4605edfb 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -45,9 +45,9 @@ public class CliRealtime extends ServerRunnable } @Override - protected List getModules() + protected List getModules() { - return ImmutableList.of( + return ImmutableList.of( new RealtimeModule(), new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/CliRealtimeExample.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java index a9ce071f000..3f1b03d502a 100644 --- a/services/src/main/java/io/druid/cli/CliRealtimeExample.java +++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java @@ -55,9 +55,9 @@ public class CliRealtimeExample extends ServerRunnable } @Override - protected List getModules() + protected List getModules() { - return ImmutableList.of( + return ImmutableList.of( new RealtimeModule(), new Module() { diff --git a/services/src/main/java/io/druid/cli/CliRouter.java b/services/src/main/java/io/druid/cli/CliRouter.java index 550f94e7c47..c80900a59e2 100644 --- a/services/src/main/java/io/druid/cli/CliRouter.java +++ b/services/src/main/java/io/druid/cli/CliRouter.java @@ -64,9 +64,9 @@ public class CliRouter extends ServerRunnable } @Override - protected List getModules() + protected List getModules() { - return ImmutableList.of( + return ImmutableList.of( new JettyHttpClientModule("druid.router.http", Router.class), new Module() { diff --git a/services/src/main/java/io/druid/cli/CreateTables.java b/services/src/main/java/io/druid/cli/CreateTables.java index 1611559fc5e..f9385ca15f9 100644 --- a/services/src/main/java/io/druid/cli/CreateTables.java +++ b/services/src/main/java/io/druid/cli/CreateTables.java @@ -62,9 +62,9 @@ public class CreateTables extends GuiceRunnable } @Override - protected List getModules() + protected List getModules() { - return ImmutableList.of( + return ImmutableList.of( new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/GuiceRunnable.java b/services/src/main/java/io/druid/cli/GuiceRunnable.java index 466104427ed..7f2394d1115 100644 --- a/services/src/main/java/io/druid/cli/GuiceRunnable.java +++ b/services/src/main/java/io/druid/cli/GuiceRunnable.java @@ -22,6 +22,7 @@ package io.druid.cli; import com.google.common.base.Throwables; import com.google.inject.Inject; import com.google.inject.Injector; +import com.google.inject.Module; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import io.druid.initialization.Initialization; @@ -48,7 +49,7 @@ public abstract class GuiceRunnable implements Runnable this.baseInjector = injector; } - protected abstract List getModules(); + protected abstract List getModules(); public Injector makeInjector() { From 971afab36f5517fee0b3639ec545b70da2b95a7e Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 19 Dec 2014 12:52:22 -0800 Subject: [PATCH 45/71] Lengthen CompressionStrategyTest::testKnownSizeConcurrency() to have 2m timeout on its test to account for shared Jenkins build lag --- .../segment/data/CompressionStrategyTest.java | 30 ++++++++----------- 1 file changed, 13 insertions(+), 17 deletions(-) diff --git a/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java b/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java index 3a261a7083f..8c7ce610679 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java @@ -21,6 +21,10 @@ package io.druid.segment.data; import com.google.common.base.Function; import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import io.druid.segment.CompressedPools; import org.junit.Assert; import org.junit.Before; @@ -35,11 +39,13 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.List; import java.util.Random; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -158,26 +164,20 @@ public class CompressionStrategyTest } - @Test + @Test(timeout = 120000) public void testKnownSizeConcurrency() throws InterruptedException, ExecutionException, TimeoutException { final int numThreads = 20; - BlockingQueue queue = new ArrayBlockingQueue<>(numThreads); - ThreadPoolExecutor threadPoolExecutor = new ThreadPoolExecutor( - numThreads, - numThreads, - 100, - TimeUnit.MILLISECONDS, - queue - ); - Collection> results = new ArrayList<>(); + + ListeningExecutorService threadPoolExecutor = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(numThreads)); + List> results = new ArrayList<>(); for (int i = 0; i < numThreads; ++i) { results.add( threadPoolExecutor.submit( - new Callable() + new Runnable() { @Override - public Boolean call() throws Exception + public void run() { ByteBuffer compressed = ByteBuffer.wrap(compressionStrategy.getCompressor().compress(originalData)); ByteBuffer output = ByteBuffer.allocate(originalData.length); @@ -187,15 +187,11 @@ public class CompressionStrategyTest byte[] checkArray = new byte[DATA_SIZER]; output.get(checkArray); Assert.assertArrayEquals("Uncompressed data does not match", originalData, checkArray); - return true; } } ) ); } - threadPoolExecutor.shutdown(); - for (Future result : results) { - Assert.assertTrue((Boolean) result.get(500, TimeUnit.MILLISECONDS)); - } + Futures.allAsList(results).get(); } } From 9cf41b022a4f01e1dc408f8736fc849b6ddb1280 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 19 Dec 2014 15:53:19 -0800 Subject: [PATCH 46/71] add travis-ci config --- .travis.yml | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 .travis.yml diff --git a/.travis.yml b/.travis.yml new file mode 100644 index 00000000000..2fd5140efca --- /dev/null +++ b/.travis.yml @@ -0,0 +1,5 @@ +language: java + +jdk: + - oraclejdk7 + - oraclejdk8 From 27a31693129c893ed09ab997952178d283db88b5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 19 Dec 2014 16:26:28 -0800 Subject: [PATCH 47/71] increase test timeouts --- .../segment/data/CompressionStrategyTest.java | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java b/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java index 8c7ce610679..eda59ed6615 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java @@ -25,16 +25,12 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import io.druid.segment.CompressedPools; import org.junit.Assert; -import org.junit.Before; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.nio.BufferOverflowException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; @@ -44,12 +40,10 @@ import java.util.Random; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; @RunWith(Parameterized.class) public class CompressionStrategyTest @@ -124,8 +118,8 @@ public class CompressionStrategyTest Assert.assertArrayEquals("Uncompressed data does not match", originalData, checkArray); } - @Test - public void testConcurrency() throws InterruptedException, ExecutionException, TimeoutException + @Test(timeout = 60000) + public void testConcurrency() throws Exception { final int numThreads = 20; BlockingQueue queue = new ArrayBlockingQueue<>(numThreads); @@ -159,13 +153,13 @@ public class CompressionStrategyTest } threadPoolExecutor.shutdown(); for (Future result : results) { - Assert.assertTrue((Boolean) result.get(100, TimeUnit.MILLISECONDS)); + Assert.assertTrue((Boolean) result.get()); } } - @Test(timeout = 120000) - public void testKnownSizeConcurrency() throws InterruptedException, ExecutionException, TimeoutException + @Test(timeout = 60000) + public void testKnownSizeConcurrency() throws Exception { final int numThreads = 20; From badc09e85cc6e31120f6f50d63e78058be18b526 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Sun, 21 Dec 2014 09:19:21 -0800 Subject: [PATCH 48/71] Disable checkGzExists in the default GzipFilter. It is unhelpful because we don't actually have any precomputed .gz resources, and checking for them inside jars is expensive. --- .../BaseJettyServerInitializer.java | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/io/druid/server/initialization/BaseJettyServerInitializer.java b/server/src/main/java/io/druid/server/initialization/BaseJettyServerInitializer.java index 22c3b2a7d7f..c08291f86dc 100644 --- a/server/src/main/java/io/druid/server/initialization/BaseJettyServerInitializer.java +++ b/server/src/main/java/io/druid/server/initialization/BaseJettyServerInitializer.java @@ -31,17 +31,26 @@ public abstract class BaseJettyServerInitializer implements JettyServerInitializ public static final String GZIP_METHODS = Joiner.on(",").join(HttpMethod.GET, HttpMethod.POST); - public FilterHolder defaultGzipFilterHolder() { + public FilterHolder defaultGzipFilterHolder() + { final FilterHolder gzipFilterHolder = new FilterHolder(GzipFilter.class); - gzipFilterHolder.setInitParameter("minGzipSize", "0"); - gzipFilterHolder.setInitParameter("methods", GZIP_METHODS); + setDefaultGzipFilterHolderParameters(gzipFilterHolder); return gzipFilterHolder; } - public FilterHolder defaultAsyncGzipFilterHolder() { + public FilterHolder defaultAsyncGzipFilterHolder() + { final FilterHolder gzipFilterHolder = new FilterHolder(AsyncGzipFilter.class); - gzipFilterHolder.setInitParameter("minGzipSize", "0"); - gzipFilterHolder.setInitParameter("methods", GZIP_METHODS); + setDefaultGzipFilterHolderParameters(gzipFilterHolder); return gzipFilterHolder; } + + private static void setDefaultGzipFilterHolderParameters(final FilterHolder filterHolder) + { + filterHolder.setInitParameter("minGzipSize", "0"); + filterHolder.setInitParameter("methods", GZIP_METHODS); + + // We don't actually have any precomputed .gz resources, and checking for them inside jars is expensive. + filterHolder.setInitParameter("checkGzExists", String.valueOf(false)); + } } From 4338af0e3f2ef3f91c3a8cf78e91122c54c89e21 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 22 Dec 2014 12:59:47 -0800 Subject: [PATCH 49/71] Fix NPE in QueryResource on bad query --- server/src/main/java/io/druid/server/QueryResource.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 5f17609c962..16590466893 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -130,7 +130,6 @@ public class QueryResource { final long start = System.currentTimeMillis(); Query query = null; - byte[] requestQuery = null; String queryId = null; final String reqContentType = req.getContentType(); @@ -267,9 +266,10 @@ public class QueryResource ).build(); } catch (Exception e) { + // Input stream has already been consumed by the json object mapper if query == null final String queryString = query == null - ? (isSmile ? "smile_unknown" : new String(requestQuery, Charsets.UTF_8)) + ? "unparsable query" : query.toString(); log.warn(e, "Exception occurred on request [%s]", queryString); From 458236c8fd363ea9aaa09174b174513244d7991c Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 22 Dec 2014 21:53:58 -0800 Subject: [PATCH 50/71] Add simple unit tests for QueryResource --- .../io/druid/server/QueryResourceTest.java | 153 ++++++++++++++++++ 1 file changed, 153 insertions(+) create mode 100644 server/src/test/java/io/druid/server/QueryResourceTest.java diff --git a/server/src/test/java/io/druid/server/QueryResourceTest.java b/server/src/test/java/io/druid/server/QueryResourceTest.java new file mode 100644 index 00000000000..a8b6d52335e --- /dev/null +++ b/server/src/test/java/io/druid/server/QueryResourceTest.java @@ -0,0 +1,153 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.server; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import com.metamx.emitter.service.ServiceEmitter; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QuerySegmentWalker; +import io.druid.query.SegmentDescriptor; +import io.druid.server.initialization.ServerConfig; +import io.druid.server.log.NoopRequestLogger; +import io.druid.server.metrics.NoopServiceEmitter; +import org.easymock.EasyMock; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.util.Map; + +/** + * + */ +public class QueryResourceTest +{ + private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + public static final ServerConfig serverConfig = new ServerConfig(){ + @Override + public int getNumThreads(){ + return 1; + } + @Override + public Period getMaxIdleTime(){ + return Period.seconds(1); + } + }; + private final HttpServletRequest testServletRequest = EasyMock.createMock(HttpServletRequest.class); + public static final QuerySegmentWalker testSegmentWalker = new QuerySegmentWalker() + { + @Override + public QueryRunner getQueryRunnerForIntervals( + Query query, Iterable intervals + ) + { + return new QueryRunner() + { + @Override + public Sequence run( + Query query, Map responseContext + ) + { + return Sequences.empty(); + } + }; + } + + @Override + public QueryRunner getQueryRunnerForSegments( + Query query, Iterable specs + ) + { + return getQueryRunnerForIntervals(null, null); + } + }; + + private static final ServiceEmitter noopServiceEmitter = new NoopServiceEmitter(); + @BeforeClass + public static void staticSetup(){ + com.metamx.emitter.EmittingLogger.registerEmitter(noopServiceEmitter); + } + @Before + public void setup() + { + EasyMock.expect(testServletRequest.getContentType()).andReturn(MediaType.APPLICATION_JSON); + EasyMock.expect(testServletRequest.getRemoteAddr()).andReturn("localhost").anyTimes(); + EasyMock.replay(testServletRequest); + } + private static final String simpleTimeSeriesQuery = "{\n" + + " \"queryType\": \"timeseries\",\n" + + " \"dataSource\": \"mmx_metrics\",\n" + + " \"granularity\": \"hour\",\n" + + " \"intervals\": [\n" + + " \"2014-12-17/2015-12-30\"\n" + + " ],\n" + + " \"aggregations\": [\n" + + " {\n" + + " \"type\": \"count\",\n" + + " \"name\": \"rows\"\n" + + " }\n" + + " ]\n" + + "}"; + @Test + public void testGoodQuery() throws IOException + { + QueryResource queryResource = new QueryResource( + serverConfig, + jsonMapper, + jsonMapper, + testSegmentWalker, + new NoopServiceEmitter(), + new NoopRequestLogger(), + new QueryManager() + ); + Response respone = queryResource.doPost(new ByteArrayInputStream(simpleTimeSeriesQuery.getBytes("UTF-8")), null /*pretty*/, testServletRequest); + Assert.assertNotNull(respone); + } + @Test + public void testBadQuery() throws IOException + { + + QueryResource queryResource = new QueryResource( + serverConfig, + jsonMapper, + jsonMapper, + testSegmentWalker, + new NoopServiceEmitter(), + new NoopRequestLogger(), + new QueryManager() + ); + Response respone = queryResource.doPost(new ByteArrayInputStream("Meka Leka Hi Meka Hiney Ho".getBytes("UTF-8")), null /*pretty*/, testServletRequest); + Assert.assertNotNull(respone); + Assert.assertEquals(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), respone.getStatus()); + } +} From bfb9e1a0932fe65d2fd5e4ddfdea1b80e4c8c340 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 23 Dec 2014 11:17:20 -0800 Subject: [PATCH 51/71] update pom to point to druid-io/druid --- pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 2728d7b3023..413faf5b80d 100644 --- a/pom.xml +++ b/pom.xml @@ -28,9 +28,9 @@ druid druid - scm:git:ssh://git@github.com/metamx/druid.git - scm:git:ssh://git@github.com/metamx/druid.git - http://www.github.com/metamx/druid + scm:git:ssh://git@github.com/druid-io/druid.git + scm:git:ssh://git@github.com/druid-io/druid.git + https://github.com/druid-io/druid druid-0.7.0-SNAPSHOT From 47fa1b3d101622fc2c52f39e79f1171e6cd73f21 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 23 Dec 2014 11:26:52 -0800 Subject: [PATCH 52/71] update docs to point to druid-io/druid --- docs/_includes/page_header.html | 2 +- docs/content/Booting-a-production-cluster.md | 2 +- docs/content/Build-from-source.md | 2 +- docs/content/Examples.md | 6 +++--- docs/content/Realtime.md | 2 +- docs/content/Recommendations.md | 16 ++++++++-------- docs/content/Stand-Alone-With-Riak-CS.md | 2 +- 7 files changed, 16 insertions(+), 16 deletions(-) diff --git a/docs/_includes/page_header.html b/docs/_includes/page_header.html index 623f5e1afc7..f39430ba369 100644 --- a/docs/_includes/page_header.html +++ b/docs/_includes/page_header.html @@ -10,7 +10,7 @@ diff --git a/docs/content/Booting-a-production-cluster.md b/docs/content/Booting-a-production-cluster.md index 2b368d428f1..e19b0eb39e9 100644 --- a/docs/content/Booting-a-production-cluster.md +++ b/docs/content/Booting-a-production-cluster.md @@ -25,7 +25,7 @@ Clone the code from [https://github.com/druid-io/whirr](https://github.com/druid git checkout trunk mvn clean install -Dmaven.test.failure.ignore=true -In order to run the test below, you'll also need two files that available only from a [standard install of Druid](http://druid.io/downloads.html) or the [Druid repo](https://github.com/metamx/druid/tree/master/examples/bin/examples): +In order to run the test below, you'll also need two files that available only from a [standard install of Druid](http://druid.io/downloads.html) or the [Druid repo](https://github.com/druid-io/druid/tree/master/examples/bin/examples): * `druid/examples/bin/examples/wikipedia/wikipedia_realtime.spec` * `druid/examples/bin/examples/indexing/wikipedia_realtime_task.json` diff --git a/docs/content/Build-from-source.md b/docs/content/Build-from-source.md index 6b8f1cc8605..573f6aef2ff 100644 --- a/docs/content/Build-from-source.md +++ b/docs/content/Build-from-source.md @@ -6,7 +6,7 @@ layout: doc_page The other way to setup Druid is from source via git. To do so, run these commands: ``` -git clone git@github.com:metamx/druid.git +git clone git@github.com:druid-io/druid.git cd druid ./build.sh ``` diff --git a/docs/content/Examples.md b/docs/content/Examples.md index 45c5d573892..47ba0f1fc41 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -4,7 +4,7 @@ layout: doc_page Examples ======== -The examples on this page are setup in order to give you a feel for what Druid does in practice. They are quick demos of Druid based on [CliRealtimeExample](https://github.com/metamx/druid/blob/master/services/src/main/java/io/druid/cli/CliRealtimeExample.java). While you wouldn’t run it this way in production you should be able to see how ingestion works and the kind of exploratory queries that are possible. Everything that can be done on your box here can be scaled out to 10’s of billions of events and terabytes of data per day in a production cluster while still giving the snappy responsive exploratory queries. +The examples on this page are setup in order to give you a feel for what Druid does in practice. They are quick demos of Druid based on [CliRealtimeExample](https://github.com/druid-io/druid/blob/master/services/src/main/java/io/druid/cli/CliRealtimeExample.java). While you wouldn’t run it this way in production you should be able to see how ingestion works and the kind of exploratory queries that are possible. Everything that can be done on your box here can be scaled out to 10’s of billions of events and terabytes of data per day in a production cluster while still giving the snappy responsive exploratory queries. Installing Standalone Druid --------------------------- @@ -16,7 +16,7 @@ There are two options for installing standalone Druid. Building from source, and Clone Druid and build it: ``` bash -git clone https://github.com/metamx/druid.git druid +git clone https://github.com/druid-io/druid.git druid cd druid git fetch --tags git checkout druid-0.6.160 @@ -37,7 +37,7 @@ Twitter Example For a full tutorial based on the twitter example, check out this [Twitter Tutorial](Twitter-Tutorial.html). -This Example uses a feature of Twitter that allows for sampling of it’s stream. We sample the Twitter stream via our [TwitterSpritzerFirehoseFactory](https://github.com/metamx/druid/blob/master/examples/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java) class and use it to simulate the kinds of data you might ingest into Druid. Then, with the client part, the sample shows what kinds of analytics explorations you can do during and after the data is loaded. +This Example uses a feature of Twitter that allows for sampling of it’s stream. We sample the Twitter stream via our [TwitterSpritzerFirehoseFactory](https://github.com/druid-io/druid/blob/master/examples/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java) class and use it to simulate the kinds of data you might ingest into Druid. Then, with the client part, the sample shows what kinds of analytics explorations you can do during and after the data is loaded. ### What you’ll learn * See how large amounts of data gets ingested into Druid in real-time diff --git a/docs/content/Realtime.md b/docs/content/Realtime.md index f38e35ec4f3..9ce84f3e68c 100644 --- a/docs/content/Realtime.md +++ b/docs/content/Realtime.md @@ -37,7 +37,7 @@ Extending the code Realtime integration is intended to be extended in two ways: 1. Connect to data streams from varied systems ([Firehose](https://github.com/druid-io/druid-api/blob/master/src/main/java/io/druid/data/input/FirehoseFactory.java)) -2. Adjust the publishing strategy to match your needs ([Plumber](https://github.com/metamx/druid/blob/master/server/src/main/java/io/druid/segment/realtime/plumber/PlumberSchool.java)) +2. Adjust the publishing strategy to match your needs ([Plumber](https://github.com/druid-io/druid/blob/master/server/src/main/java/io/druid/segment/realtime/plumber/PlumberSchool.java)) The expectations are that the former will be very common and something that users of Druid will do on a fairly regular basis. Most users will probably never have to deal with the latter form of customization. Indeed, we hope that all potential use cases can be packaged up as part of Druid proper without requiring proprietary customization. diff --git a/docs/content/Recommendations.md b/docs/content/Recommendations.md index bf764ffe6c2..56022120170 100644 --- a/docs/content/Recommendations.md +++ b/docs/content/Recommendations.md @@ -16,20 +16,20 @@ Druid is not perfect in how it handles mix-cased dimension and metric names. Thi # SSDs SSDs are highly recommended for historical and real-time nodes if you are not running a cluster that is entirely in memory. SSDs can greatly mitigate the time required to page data in and out of memory. - + # Provide Columns Names in Lexicographic Order -Although Druid supports schema-less ingestion of dimensions, because of [https://github.com/metamx/druid/issues/658](https://github.com/metamx/druid/issues/658), you may sometimes get bigger segments than necessary. To ensure segments are as compact as possible, providing dimension names in lexicographic order is recommended. - - +Although Druid supports schema-less ingestion of dimensions, because of [https://github.com/druid-io/druid/issues/658](https://github.com/druid-io/druid/issues/658), you may sometimes get bigger segments than necessary. To ensure segments are as compact as possible, providing dimension names in lexicographic order is recommended. + + # Use Timeseries and TopN Queries Instead of GroupBy Where Possible - -Timeseries and TopN queries are much more optimized and significantly faster than groupBy queries for their designed use cases. Issuing multiple topN or timeseries queries from your application can potentially be more efficient than a single groupBy query. - + +Timeseries and TopN queries are much more optimized and significantly faster than groupBy queries for their designed use cases. Issuing multiple topN or timeseries queries from your application can potentially be more efficient than a single groupBy query. + # Read FAQs You should read common problems people have here: 1) [Ingestion-FAQ](Ingestion-FAQ.html) -2) [Performance-FAQ](Performance-FAQ.html) \ No newline at end of file +2) [Performance-FAQ](Performance-FAQ.html) diff --git a/docs/content/Stand-Alone-With-Riak-CS.md b/docs/content/Stand-Alone-With-Riak-CS.md index 19727ad84c8..2108c2dcb20 100644 --- a/docs/content/Stand-Alone-With-Riak-CS.md +++ b/docs/content/Stand-Alone-With-Riak-CS.md @@ -110,7 +110,7 @@ This just walks through getting the relevant software installed and running. Yo 1. Clone the git repository for druid, checkout a "stable" tag and build - git clone https://github.com/metamx/druid.git druid + git clone https://github.com/druid-io/druid.git druid pushd druid git checkout druid-0.4.12 export LANGUAGE=C From f2439899e7f41ef5b8b2c5e2765ac689b0391774 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 23 Dec 2014 15:07:32 -0800 Subject: [PATCH 53/71] fix bitmap factory serde --- .../io/druid/segment/data/BitmapSerde.java | 6 ++- .../segment/data/BitmapSerdeFactoryTest.java | 48 +++++++++++++++++++ 2 files changed, 53 insertions(+), 1 deletion(-) create mode 100644 processing/src/test/java/io/druid/segment/data/BitmapSerdeFactoryTest.java diff --git a/processing/src/main/java/io/druid/segment/data/BitmapSerde.java b/processing/src/main/java/io/druid/segment/data/BitmapSerde.java index 6e47dd765e9..b68a2bddf1c 100644 --- a/processing/src/main/java/io/druid/segment/data/BitmapSerde.java +++ b/processing/src/main/java/io/druid/segment/data/BitmapSerde.java @@ -19,15 +19,19 @@ package io.druid.segment.data; -import com.metamx.common.ISE; +import com.fasterxml.jackson.annotation.JsonTypeName; public class BitmapSerde { // default bitmap indices for Druid >= 0.7.x + // annotation required so Jackson doesn't get confused + @JsonTypeName("concise") public static class DefaultBitmapSerdeFactory extends ConciseBitmapSerdeFactory {} // default bitmap indices in Druid <= 0.6.x + @JsonTypeName("concise") + // annotation required so Jackson doesn't get confused by subclassing public static class LegacyBitmapSerdeFactory extends ConciseBitmapSerdeFactory {} public static BitmapSerdeFactory createLegacyFactory() diff --git a/processing/src/test/java/io/druid/segment/data/BitmapSerdeFactoryTest.java b/processing/src/test/java/io/druid/segment/data/BitmapSerdeFactoryTest.java new file mode 100644 index 00000000000..4438ab787f0 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/data/BitmapSerdeFactoryTest.java @@ -0,0 +1,48 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment.data; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +public class BitmapSerdeFactoryTest +{ + @Test + public void testSerialization() throws Exception + { + ObjectMapper mapper = new DefaultObjectMapper(); + Assert.assertEquals("{\"type\":\"roaring\"}", mapper.writeValueAsString(new RoaringBitmapSerdeFactory())); + Assert.assertEquals("{\"type\":\"concise\"}", mapper.writeValueAsString(new ConciseBitmapSerdeFactory())); + Assert.assertEquals("{\"type\":\"concise\"}", mapper.writeValueAsString(BitmapSerde.createLegacyFactory())); + Assert.assertEquals("{\"type\":\"concise\"}", mapper.writeValueAsString(new BitmapSerde.DefaultBitmapSerdeFactory())); + Assert.assertEquals("{\"type\":\"concise\"}", mapper.writeValueAsString(new BitmapSerde.LegacyBitmapSerdeFactory())); + } + + @Test + public void testDeserialization() throws Exception + { + ObjectMapper mapper = new DefaultObjectMapper(); + Assert.assertTrue(mapper.readValue("{\"type\":\"roaring\"}", BitmapSerdeFactory.class) instanceof RoaringBitmapSerdeFactory); + Assert.assertTrue(mapper.readValue("{\"type\":\"concise\"}", BitmapSerdeFactory.class) instanceof ConciseBitmapSerdeFactory); + Assert.assertTrue(mapper.readValue("{\"type\":\"BitmapSerde$SomeRandomClass\"}", BitmapSerdeFactory.class) instanceof ConciseBitmapSerdeFactory); + } +} From 574ab3017c5ae112b41be1bfef2e288d4445d6b3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 23 Dec 2014 16:25:38 -0800 Subject: [PATCH 54/71] fix test timeout --- .../segment/realtime/plumber/RealtimePlumberSchoolTest.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index 6d925ecc507..15807388dfa 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -190,7 +190,7 @@ public class RealtimePlumberSchoolTest EasyMock.verify(announcer, segmentPublisher, dataSegmentPusher, serverView, emitter); } - @Test + @Test(timeout = 60000) public void testPersist() throws Exception { final MutableBoolean committed = new MutableBoolean(false); @@ -212,12 +212,8 @@ public class RealtimePlumberSchoolTest } ); - Stopwatch stopwatch = Stopwatch.createStarted(); while (!committed.booleanValue()) { Thread.sleep(100); - if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) { - throw new ISE("Taking too long to set perist value"); - } } plumber.getSinks().clear(); plumber.finishJob(); From 65286a24e0a8116473319834eeea4e6a19a5aa6a Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 1 Dec 2014 13:35:38 -0800 Subject: [PATCH 55/71] Change zk configs to use Jackson injection instead of Skife * Also added generic config testing class JsonConfigTesterBase --- .../guice/IndexingServiceModuleHelper.java | 2 + .../indexing/overlord/RemoteTaskRunner.java | 19 +- .../overlord/RemoteTaskRunnerFactory.java | 5 +- .../druid/indexing/overlord/TaskMaster.java | 5 +- .../worker/WorkerCuratorCoordinator.java | 10 +- .../initialization/IndexerZkConfig.java | 87 +++++++++ .../overlord/RemoteTaskRunnerTest.java | 15 +- .../worker/WorkerTaskMonitorTest.java | 6 +- .../worker/http/WorkerResourceTest.java | 7 +- .../initialization/IndexerZkConfigTest.java | 175 ++++++++++++++++++ pom.xml | 2 +- .../java/io/druid/curator/CuratorConfig.java | 52 ++++-- .../java/io/druid/curator/CuratorModule.java | 7 +- .../java/io/druid/guice/ServerModule.java | 3 +- .../server/bridge/DruidClusterBridge.java | 18 +- .../bridge/DruidClusterBridgeConfig.java | 58 +++--- .../server/initialization/ZkPathsConfig.java | 114 +++++++----- .../client/BatchServerInventoryViewTest.java | 6 +- .../io/druid/curator/CuratorConfigTest.java | 39 ++++ .../io/druid/guice/JsonConfigTesterBase.java | 155 ++++++++++++++++ .../initialization/ZkPathsConfigTest.java | 75 ++++++++ .../server/bridge/DruidClusterBridgeTest.java | 22 +-- .../coordination/ZkCoordinatorTest.java | 2 +- .../BatchDataSegmentAnnouncerTest.java | 2 +- .../coordinator/DruidCoordinatorTest.java | 2 +- .../src/main/java/io/druid/cli/CliBridge.java | 2 +- 26 files changed, 745 insertions(+), 145 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/server/initialization/IndexerZkConfig.java create mode 100644 indexing-service/src/test/java/io/druid/server/initialization/IndexerZkConfigTest.java create mode 100644 server/src/test/java/io/druid/curator/CuratorConfigTest.java create mode 100644 server/src/test/java/io/druid/guice/JsonConfigTesterBase.java create mode 100644 server/src/test/java/io/druid/initialization/ZkPathsConfigTest.java diff --git a/indexing-service/src/main/java/io/druid/guice/IndexingServiceModuleHelper.java b/indexing-service/src/main/java/io/druid/guice/IndexingServiceModuleHelper.java index 72a983d6884..b4f57f3b0ea 100644 --- a/indexing-service/src/main/java/io/druid/guice/IndexingServiceModuleHelper.java +++ b/indexing-service/src/main/java/io/druid/guice/IndexingServiceModuleHelper.java @@ -22,6 +22,7 @@ package io.druid.guice; import com.google.inject.Binder; import io.druid.indexing.overlord.config.ForkingTaskRunnerConfig; import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; +import io.druid.server.initialization.IndexerZkConfig; /** */ @@ -31,5 +32,6 @@ public class IndexingServiceModuleHelper { JsonConfigProvider.bind(binder, "druid.indexer.runner", ForkingTaskRunnerConfig.class); JsonConfigProvider.bind(binder, "druid.indexer.runner", RemoteTaskRunnerConfig.class); + JsonConfigProvider.bind(binder, "druid.zk.paths.indexer", IndexerZkConfig.class); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index 355f018ea89..bc70d153c1b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -51,6 +51,7 @@ import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; import io.druid.indexing.overlord.setup.WorkerSelectStrategy; import io.druid.indexing.worker.TaskAnnouncement; import io.druid.indexing.worker.Worker; +import io.druid.server.initialization.IndexerZkConfig; import io.druid.server.initialization.ZkPathsConfig; import io.druid.tasklogs.TaskLogStreamer; import org.apache.commons.lang.mutable.MutableInt; @@ -102,7 +103,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer private final ObjectMapper jsonMapper; private final RemoteTaskRunnerConfig config; - private final ZkPathsConfig zkPaths; + private final IndexerZkConfig indexerZkConfig; private final CuratorFramework cf; private final PathChildrenCacheFactory pathChildrenCacheFactory; private final PathChildrenCache workerPathCache; @@ -129,7 +130,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer public RemoteTaskRunner( ObjectMapper jsonMapper, RemoteTaskRunnerConfig config, - ZkPathsConfig zkPaths, + IndexerZkConfig indexerZkConfig, CuratorFramework cf, PathChildrenCacheFactory pathChildrenCacheFactory, HttpClient httpClient, @@ -138,10 +139,10 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer { this.jsonMapper = jsonMapper; this.config = config; - this.zkPaths = zkPaths; + this.indexerZkConfig = indexerZkConfig; this.cf = cf; this.pathChildrenCacheFactory = pathChildrenCacheFactory; - this.workerPathCache = pathChildrenCacheFactory.make(cf, zkPaths.getIndexerAnnouncementPath()); + this.workerPathCache = pathChildrenCacheFactory.make(cf, indexerZkConfig.getAnnouncementsPath()); this.httpClient = httpClient; this.strategy = strategy; } @@ -496,7 +497,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer } else { final String workerId = worker.getHost(); log.info("Cleaning up task[%s] on worker[%s]", taskId, workerId); - final String statusPath = JOINER.join(zkPaths.getIndexerStatusPath(), workerId, taskId); + final String statusPath = JOINER.join(indexerZkConfig.getStatus(), workerId, taskId); try { cf.delete().guaranteed().forPath(statusPath); } @@ -582,7 +583,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxZnodeBytes()); } - String taskPath = JOINER.join(zkPaths.getIndexerTaskPath(), theWorker.getHost(), task.getId()); + String taskPath = JOINER.join(indexerZkConfig.getTasksPath(), theWorker.getHost(), task.getId()); if (cf.checkExists().forPath(taskPath) == null) { cf.create() @@ -642,7 +643,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer log.info("Worker[%s] reportin' for duty!", worker.getHost()); try { - final String workerStatusPath = JOINER.join(zkPaths.getIndexerStatusPath(), worker.getHost()); + final String workerStatusPath = JOINER.join(indexerZkConfig.getStatus(), worker.getHost()); final PathChildrenCache statusCache = pathChildrenCacheFactory.make(cf, workerStatusPath); final SettableFuture retVal = SettableFuture.create(); final ZkWorker zkWorker = new ZkWorker( @@ -787,7 +788,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer if (zkWorker != null) { try { List tasksToFail = Lists.newArrayList( - cf.getChildren().forPath(JOINER.join(zkPaths.getIndexerTaskPath(), worker.getHost())) + cf.getChildren().forPath(JOINER.join(indexerZkConfig.getTasksPath(), worker.getHost())) ); log.info("[%s]: Found %d tasks assigned", worker.getHost(), tasksToFail.size()); @@ -805,7 +806,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer for (String assignedTask : tasksToFail) { RemoteTaskRunnerWorkItem taskRunnerWorkItem = runningTasks.remove(assignedTask); if (taskRunnerWorkItem != null) { - String taskPath = JOINER.join(zkPaths.getIndexerTaskPath(), worker.getHost(), assignedTask); + String taskPath = JOINER.join(indexerZkConfig.getTasksPath(), worker.getHost(), assignedTask); if (cf.checkExists().forPath(taskPath) != null) { cf.delete().guaranteed().forPath(taskPath); } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java index e68aa15b431..5c0e1cca09f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java @@ -29,6 +29,7 @@ import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; import io.druid.indexing.overlord.setup.FillCapacityWorkerSelectStrategy; import io.druid.indexing.overlord.setup.WorkerBehaviorConfig; import io.druid.indexing.overlord.setup.WorkerSelectStrategy; +import io.druid.server.initialization.IndexerZkConfig; import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; @@ -38,7 +39,7 @@ public class RemoteTaskRunnerFactory implements TaskRunnerFactory { private final CuratorFramework curator; private final RemoteTaskRunnerConfig remoteTaskRunnerConfig; - private final ZkPathsConfig zkPaths; + private final IndexerZkConfig zkPaths; private final ObjectMapper jsonMapper; private final HttpClient httpClient; private final WorkerSelectStrategy strategy; @@ -47,7 +48,7 @@ public class RemoteTaskRunnerFactory implements TaskRunnerFactory public RemoteTaskRunnerFactory( final CuratorFramework curator, final RemoteTaskRunnerConfig remoteTaskRunnerConfig, - final ZkPathsConfig zkPaths, + final IndexerZkConfig zkPaths, final ObjectMapper jsonMapper, @Global final HttpClient httpClient, final Supplier workerBehaviourConfigSupplier diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java index 61a7b595437..5b4c79a9327 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java @@ -38,6 +38,7 @@ import io.druid.indexing.overlord.config.TaskQueueConfig; import io.druid.indexing.overlord.autoscaling.ResourceManagementScheduler; import io.druid.indexing.overlord.autoscaling.ResourceManagementSchedulerFactory; import io.druid.server.DruidNode; +import io.druid.server.initialization.IndexerZkConfig; import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.leader.LeaderSelector; @@ -74,7 +75,7 @@ public class TaskMaster final TaskStorage taskStorage, final TaskActionClientFactory taskActionClientFactory, @Self final DruidNode node, - final ZkPathsConfig zkPaths, + final IndexerZkConfig zkPaths, final TaskRunnerFactory runnerFactory, final ResourceManagementSchedulerFactory managementSchedulerFactory, final CuratorFramework curator, @@ -85,7 +86,7 @@ public class TaskMaster this.taskActionClientFactory = taskActionClientFactory; this.leaderSelector = new LeaderSelector( curator, - zkPaths.getIndexerLeaderLatchPath(), + zkPaths.getLeaderLatchPath(), new LeaderSelectorListener() { @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java index 32dbf5b85ff..dc215545c7a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java @@ -31,7 +31,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import io.druid.curator.announcement.Announcer; import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; -import io.druid.server.initialization.ZkPathsConfig; +import io.druid.server.initialization.IndexerZkConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.zookeeper.CreateMode; import org.joda.time.DateTime; @@ -63,7 +63,7 @@ public class WorkerCuratorCoordinator @Inject public WorkerCuratorCoordinator( ObjectMapper jsonMapper, - ZkPathsConfig zkPaths, + IndexerZkConfig indexerZkConfig, RemoteTaskRunnerConfig config, CuratorFramework curatorFramework, Worker worker @@ -76,9 +76,9 @@ public class WorkerCuratorCoordinator this.announcer = new Announcer(curatorFramework, MoreExecutors.sameThreadExecutor()); - this.baseAnnouncementsPath = getPath(Arrays.asList(zkPaths.getIndexerAnnouncementPath(), worker.getHost())); - this.baseTaskPath = getPath(Arrays.asList(zkPaths.getIndexerTaskPath(), worker.getHost())); - this.baseStatusPath = getPath(Arrays.asList(zkPaths.getIndexerStatusPath(), worker.getHost())); + this.baseAnnouncementsPath = getPath(Arrays.asList(indexerZkConfig.getAnnouncementsPath(), worker.getHost())); + this.baseTaskPath = getPath(Arrays.asList(indexerZkConfig.getTasksPath(), worker.getHost())); + this.baseStatusPath = getPath(Arrays.asList(indexerZkConfig.getStatus(), worker.getHost())); } @LifecycleStart diff --git a/indexing-service/src/main/java/io/druid/server/initialization/IndexerZkConfig.java b/indexing-service/src/main/java/io/druid/server/initialization/IndexerZkConfig.java new file mode 100644 index 00000000000..6333ea3f0e9 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/server/initialization/IndexerZkConfig.java @@ -0,0 +1,87 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.server.initialization; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.inject.Inject; +import org.apache.curator.utils.ZKPaths; + +/** + * + */ +public class IndexerZkConfig +{ + @Inject + @JsonIgnore + private ZkPathsConfig zkPathsConfig = new ZkPathsConfig(); + @JsonProperty + private String base; + @JsonProperty + private String announcementsPath; + @JsonProperty + private String tasksPath; + @JsonProperty + private String status; + @JsonProperty + private String leaderLatchPath; + + private String defaultIndexerPath(final String subPath) + { + return getZkPathsConfig().defaultPath(ZKPaths.makePath(getBase(), subPath)); + } + + public String getBase() + { + return base == null ? "indexer" : base; + } + + public String getAnnouncementsPath() + { + return announcementsPath == null ? defaultIndexerPath("announcements") : announcementsPath; + } + + public String getTasksPath() + { + return tasksPath == null ? defaultIndexerPath("tasks") : tasksPath; + } + + public String getStatus() + { + return status == null ? defaultIndexerPath("status") : status; + } + + public String getLeaderLatchPath() + { + return leaderLatchPath == null ? defaultIndexerPath("leaderLatchPath") : leaderLatchPath; + } + + public ZkPathsConfig getZkPathsConfig() + { + return zkPathsConfig; + } + + // Setter required for easy debugging + public IndexerZkConfig setZkPathsConfig(ZkPathsConfig zkPathsConfig) + { + this.zkPathsConfig = zkPathsConfig; + return this; + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java index a007c9f4cf2..01bfbca8480 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java @@ -43,6 +43,7 @@ import io.druid.indexing.overlord.setup.FillCapacityWorkerSelectStrategy; import io.druid.indexing.worker.TaskAnnouncement; import io.druid.indexing.worker.Worker; import io.druid.jackson.DefaultObjectMapper; +import io.druid.server.initialization.IndexerZkConfig; import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; @@ -57,6 +58,7 @@ import org.junit.Test; import java.util.Set; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; public class RemoteTaskRunnerTest { @@ -66,6 +68,7 @@ public class RemoteTaskRunnerTest private static final String announcementsPath = String.format("%s/indexer/announcements/worker", basePath); private static final String tasksPath = String.format("%s/indexer/tasks/worker", basePath); private static final String statusPath = String.format("%s/indexer/status/worker", basePath); + private static final int TIMEOUT_SECONDS = 5; private TestingCluster testingCluster; private CuratorFramework cf; @@ -282,7 +285,7 @@ public class RemoteTaskRunnerTest cf.delete().forPath(joiner.join(statusPath, task.getId())); - TaskStatus status = future.get(); + TaskStatus status = future.get(TIMEOUT_SECONDS, TimeUnit.SECONDS); Assert.assertEquals(status.getStatusCode(), TaskStatus.Status.FAILED); } @@ -335,7 +338,7 @@ public class RemoteTaskRunnerTest ListenableFuture future = remoteTaskRunner.run(task); - TaskStatus status = future.get(); + TaskStatus status = future.get(TIMEOUT_SECONDS, TimeUnit.SECONDS); Assert.assertEquals(TaskStatus.Status.SUCCESS, status.getStatusCode()); } @@ -353,7 +356,7 @@ public class RemoteTaskRunnerTest cf.delete().forPath(announcementsPath); - TaskStatus status = future.get(); + TaskStatus status = future.get(TIMEOUT_SECONDS, TimeUnit.SECONDS); Assert.assertEquals(TaskStatus.Status.FAILED, status.getStatusCode()); } @@ -393,14 +396,14 @@ public class RemoteTaskRunnerTest remoteTaskRunner = new RemoteTaskRunner( jsonMapper, config, - new ZkPathsConfig() + new IndexerZkConfig().setZkPathsConfig(new ZkPathsConfig() { @Override - public String getZkBasePath() + public String getBase() { return basePath; } - }, + }), cf, new SimplePathChildrenCacheFactory.Builder().build(), null, diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index 478a77494a3..76b269b855e 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -50,6 +50,7 @@ import io.druid.segment.loading.LocalDataSegmentPuller; import io.druid.segment.loading.OmniSegmentLoader; import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.segment.loading.StorageLocationConfig; +import io.druid.server.initialization.IndexerZkConfig; import io.druid.server.initialization.ZkPathsConfig; import junit.framework.Assert; import org.apache.curator.framework.CuratorFramework; @@ -139,14 +140,15 @@ public class WorkerTaskMonitorTest workerCuratorCoordinator = new WorkerCuratorCoordinator( jsonMapper, + new IndexerZkConfig().setZkPathsConfig( new ZkPathsConfig() { @Override - public String getZkBasePath() + public String getBase() { return basePath; } - }, + }), new TestRemoteTaskRunnerConfig(), cf, worker diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/http/WorkerResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/http/WorkerResourceTest.java index 2a0a1e4a3c3..2b786532e79 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/http/WorkerResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/http/WorkerResourceTest.java @@ -25,6 +25,7 @@ import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; import io.druid.indexing.worker.Worker; import io.druid.indexing.worker.WorkerCuratorCoordinator; import io.druid.jackson.DefaultObjectMapper; +import io.druid.server.initialization.IndexerZkConfig; import io.druid.server.initialization.ZkPathsConfig; import junit.framework.Assert; import org.apache.curator.framework.CuratorFramework; @@ -76,14 +77,14 @@ public class WorkerResourceTest curatorCoordinator = new WorkerCuratorCoordinator( jsonMapper, - new ZkPathsConfig() + new IndexerZkConfig().setZkPathsConfig(new ZkPathsConfig() { @Override - public String getZkBasePath() + public String getBase() { return basePath; } - }, + }), new RemoteTaskRunnerConfig(), cf, worker diff --git a/indexing-service/src/test/java/io/druid/server/initialization/IndexerZkConfigTest.java b/indexing-service/src/test/java/io/druid/server/initialization/IndexerZkConfigTest.java new file mode 100644 index 00000000000..422fc05af83 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/server/initialization/IndexerZkConfigTest.java @@ -0,0 +1,175 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.server.initialization; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.name.Names; +import io.druid.curator.CuratorConfig; +import io.druid.guice.GuiceInjectors; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.JsonConfigurator; +import io.druid.initialization.Initialization; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +/** + * + */ +public class IndexerZkConfigTest +{ + private static final String indexerPropertyString = "test.druid.zk.paths.indexer"; + private static final String zkServiceConfigString = "test.druid.zk.paths"; + private static final Collection clobberableProperties = new ArrayList<>(); + + private static final Module simpleZkConfigModule = new Module() + { + @Override + public void configure(Binder binder) + { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/test"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); + // See IndexingServiceModuleHelper + JsonConfigProvider.bind(binder, indexerPropertyString, IndexerZkConfig.class); + JsonConfigProvider.bind( + binder, zkServiceConfigString, + CuratorConfig.class + ); + } + }; + + private static final Map priorValues = new HashMap<>(); + + @BeforeClass + public static void setup() + { + for (Field field : IndexerZkConfig.class.getDeclaredFields()) { + if (null != field.getAnnotation(JsonProperty.class)) { + clobberableProperties.add(String.format("%s.%s", indexerPropertyString, field.getName())); + } + } + for (Field field : ZkPathsConfig.class.getDeclaredFields()) { + if (null != field.getAnnotation(JsonProperty.class)) { + clobberableProperties.add(String.format("%s.%s", zkServiceConfigString, field.getName())); + } + } + for (String clobberableProperty : clobberableProperties) { + priorValues.put(clobberableProperty, System.getProperty(clobberableProperty)); + } + } + + @AfterClass + public static void cleanup() + { + for (Map.Entry entry : priorValues.entrySet()) { + if (null != entry.getKey() && null != entry.getValue()) { + System.setProperty(entry.getKey(), entry.getValue()); + } + } + } + + private Map propertyValues = new HashMap<>(); + private int assertions = 0; + + @Before + public void setupTest() + { + for (String property : clobberableProperties) { + propertyValues.put(property, UUID.randomUUID().toString()); + } + System.getProperties().putAll(propertyValues); + assertions = 0; + } + + + private void validateEntries(ZkPathsConfig zkPathsConfig) + throws IllegalAccessException, NoSuchMethodException, InvocationTargetException + { + for (Field field : ZkPathsConfig.class.getDeclaredFields()) { + if (null != field.getAnnotation(JsonProperty.class)) { + String property = String.format("%s.%s", zkServiceConfigString, field.getName()); + String getter = String.format( + "get%s%s", + field.getName().substring(0, 1).toUpperCase(), + field.getName().substring(1) + ); + Method method = ZkPathsConfig.class.getDeclaredMethod(getter); + Assert.assertEquals(propertyValues.get(property), method.invoke(zkPathsConfig)); + ++assertions; + } + } + } + + private void validateEntries(IndexerZkConfig indexerZkConfig) + throws IllegalAccessException, NoSuchMethodException, InvocationTargetException + { + for (Field field : IndexerZkConfig.class.getDeclaredFields()) { + if (null != field.getAnnotation(JsonProperty.class)) { + String property = String.format("%s.%s", indexerPropertyString, field.getName()); + String getter = String.format( + "get%s%s", + field.getName().substring(0, 1).toUpperCase(), + field.getName().substring(1) + ); + Method method = IndexerZkConfig.class.getDeclaredMethod(getter); + Assert.assertEquals(propertyValues.get(property), method.invoke(indexerZkConfig)); + ++assertions; + } + } + } + + @Test + public void testSimpleConfig() throws IllegalAccessException, NoSuchMethodException, InvocationTargetException + { + final Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + ImmutableList.of(simpleZkConfigModule) + ); + JsonConfigurator configurator = injector.getBinding(JsonConfigurator.class).getProvider().get(); + + JsonConfigProvider zkPathsConfig = JsonConfigProvider.of(zkServiceConfigString, ZkPathsConfig.class); + zkPathsConfig.inject(System.getProperties(), configurator); + + JsonConfigProvider indexerZkConfig = JsonConfigProvider.of( + indexerPropertyString, + IndexerZkConfig.class + ); + indexerZkConfig.inject(System.getProperties(), configurator); + + validateEntries(indexerZkConfig.get().get()); + validateEntries(zkPathsConfig.get().get()); + Assert.assertEquals(clobberableProperties.size(), assertions); + } +} diff --git a/pom.xml b/pom.xml index 413faf5b80d..6c05af814ff 100644 --- a/pom.xml +++ b/pom.xml @@ -456,7 +456,7 @@ org.easymock easymock - 3.0 + 3.3 test diff --git a/server/src/main/java/io/druid/curator/CuratorConfig.java b/server/src/main/java/io/druid/curator/CuratorConfig.java index 3e8795bfd2c..94401ec7ed6 100644 --- a/server/src/main/java/io/druid/curator/CuratorConfig.java +++ b/server/src/main/java/io/druid/curator/CuratorConfig.java @@ -19,22 +19,50 @@ package io.druid.curator; -import org.skife.config.Config; -import org.skife.config.Default; +import com.fasterxml.jackson.annotation.JsonProperty; +import javax.validation.constraints.Min; /** */ -public abstract class CuratorConfig +public class CuratorConfig { - @Config("druid.zk.service.host") - @Default("localhost") - public abstract String getZkHosts(); + @JsonProperty("host") + private String zkHosts = "localhost"; - @Config("druid.zk.service.sessionTimeoutMs") - @Default("30000") - public abstract int getZkSessionTimeoutMs(); + @JsonProperty("sessionTimeoutMs") + @Min(0) + private int zkSessionTimeoutMs = 30000; - @Config("druid.curator.compress") - @Default("true") - public abstract boolean enableCompression(); + @JsonProperty("compress") + private boolean enableCompression = true; + + public String getZkHosts() + { + return zkHosts; + } + + public void setZkHosts(String zkHosts) + { + this.zkHosts = zkHosts; + } + + public Integer getZkSessionTimeoutMs() + { + return zkSessionTimeoutMs; + } + + public void setZkSessionTimeoutMs(Integer zkSessionTimeoutMs) + { + this.zkSessionTimeoutMs = zkSessionTimeoutMs; + } + + public Boolean getEnableCompression() + { + return enableCompression; + } + + public void setEnableCompression(Boolean enableCompression) + { + this.enableCompression = enableCompression; + } } diff --git a/server/src/main/java/io/druid/curator/CuratorModule.java b/server/src/main/java/io/druid/curator/CuratorModule.java index 94fba357dd2..7d18eda90d3 100644 --- a/server/src/main/java/io/druid/curator/CuratorModule.java +++ b/server/src/main/java/io/druid/curator/CuratorModule.java @@ -25,6 +25,7 @@ import com.google.inject.Provides; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import io.druid.guice.ConfigProvider; +import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; @@ -41,7 +42,9 @@ public class CuratorModule implements Module @Override public void configure(Binder binder) { - ConfigProvider.bind(binder, CuratorConfig.class); + JsonConfigProvider.bind( + binder, "druid.zk.service", + CuratorConfig.class); } @Provides @LazySingleton @@ -52,7 +55,7 @@ public class CuratorModule implements Module .connectString(config.getZkHosts()) .sessionTimeoutMs(config.getZkSessionTimeoutMs()) .retryPolicy(new BoundedExponentialBackoffRetry(1000, 45000, 30)) - .compressionProvider(new PotentiallyGzippedCompressionProvider(config.enableCompression())) + .compressionProvider(new PotentiallyGzippedCompressionProvider(config.getEnableCompression())) .build(); lifecycle.addHandler( diff --git a/server/src/main/java/io/druid/guice/ServerModule.java b/server/src/main/java/io/druid/guice/ServerModule.java index d0ead89825d..3fbfe627edf 100644 --- a/server/src/main/java/io/druid/guice/ServerModule.java +++ b/server/src/main/java/io/druid/guice/ServerModule.java @@ -45,8 +45,7 @@ public class ServerModule implements DruidModule @Override public void configure(Binder binder) { - ConfigProvider.bind(binder, ZkPathsConfig.class); - + JsonConfigProvider.bind(binder, "druid.zk.paths", ZkPathsConfig.class); JsonConfigProvider.bind(binder, "druid", DruidNode.class, Self.class); } diff --git a/server/src/main/java/io/druid/server/bridge/DruidClusterBridge.java b/server/src/main/java/io/druid/server/bridge/DruidClusterBridge.java index b12aa3e5299..37df9000429 100644 --- a/server/src/main/java/io/druid/server/bridge/DruidClusterBridge.java +++ b/server/src/main/java/io/druid/server/bridge/DruidClusterBridge.java @@ -43,6 +43,7 @@ import io.druid.server.DruidNode; import io.druid.server.coordination.AbstractDataSegmentAnnouncer; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.DataSegment; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.leader.LeaderLatch; @@ -80,6 +81,9 @@ public class DruidClusterBridge private final BridgeZkCoordinator bridgeZkCoordinator; private final Announcer announcer; private final ServerInventoryView serverInventoryView; + private final ZkPathsConfig zkPathsConfig; + + private final DruidServerMetadata druidServerMetadata; private final Map segments = Maps.newHashMap(); private final Object lock = new Object(); @@ -91,6 +95,8 @@ public class DruidClusterBridge public DruidClusterBridge( ObjectMapper jsonMapper, DruidClusterBridgeConfig config, + ZkPathsConfig zkPathsConfig, + DruidServerMetadata druidServerMetadata, ScheduledExecutorFactory scheduledExecutorFactory, @Self DruidNode self, CuratorFramework curator, @@ -104,10 +110,12 @@ public class DruidClusterBridge this.jsonMapper = jsonMapper; this.config = config; this.bridgeZkCoordinator = bridgeZkCoordinator; + this.zkPathsConfig = zkPathsConfig; this.announcer = announcer; this.serverInventoryView = serverInventoryView; this.curator = curator; this.leaderLatch = leaderLatch; + this.druidServerMetadata = druidServerMetadata; this.exec = scheduledExecutorFactory.create(1, "Coordinator-Exec--%d"); this.self = self; @@ -212,7 +220,7 @@ public class DruidClusterBridge private LeaderLatch createNewLeaderLatch() { final LeaderLatch newLeaderLatch = new LeaderLatch( - curator, ZKPaths.makePath(config.getConnectorPath(), BRIDGE_OWNER_NODE), self.getHostAndPort() + curator, ZKPaths.makePath(zkPathsConfig.getConnectorPath(), BRIDGE_OWNER_NODE), self.getHostAndPort() ); newLeaderLatch.addListener( @@ -309,13 +317,13 @@ public class DruidClusterBridge self.getHostAndPort(), totalMaxSize, NODE_TYPE, - config.getTier(), - config.getPriority() + druidServerMetadata.getTier(), + druidServerMetadata.getPriority() ); try { - final String path = ZKPaths.makePath(config.getAnnouncementsPath(), self.getHostAndPort()); - log.info("Updating [%s] to have a maxSize of[%,d] bytes", self.getHostAndPort(), totalMaxSize); + final String path = ZKPaths.makePath(zkPathsConfig.getAnnouncementsPath(), self.getHostAndPort()); + log.info("Updating [%s] to have a maxSize of[%,d] bytes", self.getHost(), totalMaxSize); announcer.update(path, jsonMapper.writeValueAsBytes(me)); } catch (Exception e) { diff --git a/server/src/main/java/io/druid/server/bridge/DruidClusterBridgeConfig.java b/server/src/main/java/io/druid/server/bridge/DruidClusterBridgeConfig.java index 5478a375577..4508f368a49 100644 --- a/server/src/main/java/io/druid/server/bridge/DruidClusterBridgeConfig.java +++ b/server/src/main/java/io/druid/server/bridge/DruidClusterBridgeConfig.java @@ -19,34 +19,50 @@ package io.druid.server.bridge; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.inject.Inject; import io.druid.client.DruidServer; -import io.druid.server.initialization.ZkPathsConfig; +import io.druid.client.DruidServerConfig; import org.joda.time.Duration; -import org.skife.config.Config; -import org.skife.config.Default; /** */ -public abstract class DruidClusterBridgeConfig extends ZkPathsConfig +public abstract class DruidClusterBridgeConfig { - @Config("druid.server.tier") - @Default(DruidServer.DEFAULT_TIER) - public abstract String getTier(); + @JsonProperty + private Duration startDelay = new Duration("PT300s"); + @JsonProperty + private Duration period = new Duration("PT60s"); + @JsonProperty + private String brokerServiceName = "broker"; - @Config("druid.bridge.startDelay") - @Default("PT300s") - public abstract Duration getStartDelay(); - - @Config("druid.bridge.period") - @Default("PT60s") - public abstract Duration getPeriod(); - - @Config("druid.bridge.broker.serviceName") - public abstract String getBrokerServiceName(); - - @Config("druid.server.priority") - public int getPriority() + public Duration getStartDelay() { - return DruidServer.DEFAULT_PRIORITY; + return startDelay; + } + + public void setStartDelay(Duration startDelay) + { + this.startDelay = startDelay; + } + + public Duration getPeriod() + { + return period; + } + + public void setPeriod(Duration period) + { + this.period = period; + } + + public String getBrokerServiceName() + { + return brokerServiceName; + } + + public void setBrokerServiceName(String brokerServiceName) + { + this.brokerServiceName = brokerServiceName; } } diff --git a/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java b/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java index 8494b45e257..7fa09926938 100644 --- a/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java +++ b/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java @@ -19,85 +19,105 @@ package io.druid.server.initialization; +import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.curator.utils.ZKPaths; -import org.skife.config.Config; -public abstract class ZkPathsConfig +public class ZkPathsConfig { - @Config("druid.zk.paths.base") - public String getZkBasePath() + @JsonProperty + private + String base = "druid"; + @JsonProperty + private + String propertiesPath; + @JsonProperty + private + String announcementsPath; + @JsonProperty + private + String servedSegmentsPath; + @JsonProperty + private + String liveSegmentsPath; + @JsonProperty + private + String coordinatorPath; + @JsonProperty + private + String loadQueuePath; + @JsonProperty + private + String connectorPath; + + public String getBase() { - return "druid"; + return base; } - @Config("druid.zk.paths.propertiesPath") public String getPropertiesPath() { - return defaultPath("properties"); + return (null == propertiesPath) ? defaultPath("properties") : propertiesPath; } - @Config("druid.zk.paths.announcementsPath") public String getAnnouncementsPath() { - return defaultPath("announcements"); + return (null == announcementsPath) ? defaultPath("announcements") : announcementsPath; } - @Config("druid.zk.paths.servedSegmentsPath") public String getServedSegmentsPath() { - return defaultPath("servedSegments"); + return (null == servedSegmentsPath) ? defaultPath("servedSegments") : servedSegmentsPath; } - @Config("druid.zk.paths.liveSegmentsPath") public String getLiveSegmentsPath() { - return defaultPath("segments"); + return (null == liveSegmentsPath) ? defaultPath("segments") : liveSegmentsPath; } - @Config("druid.zk.paths.loadQueuePath") - public String getLoadQueuePath() - { - return defaultPath("loadQueue"); - } - - @Config("druid.zk.paths.coordinatorPath") public String getCoordinatorPath() { - return defaultPath("coordinator"); + return (null == coordinatorPath) ? defaultPath("coordinator") : coordinatorPath; + } + + public String getLoadQueuePath() + { + return (null == loadQueuePath) ? defaultPath("loadQueue") : loadQueuePath; } - @Config("druid.zk.paths.connectorPath") public String getConnectorPath() { - return defaultPath("connector"); + return (null == connectorPath) ? defaultPath("connector") : connectorPath; } - @Config("druid.zk.paths.indexer.announcementsPath") - public String getIndexerAnnouncementPath() + protected String defaultPath(final String subPath) { - return defaultPath("indexer/announcements"); + return ZKPaths.makePath(getBase(), subPath); } - @Config("druid.zk.paths.indexer.tasksPath") - public String getIndexerTaskPath() - { - return defaultPath("indexer/tasks"); - } - - @Config("druid.zk.paths.indexer.statusPath") - public String getIndexerStatusPath() - { - return defaultPath("indexer/status"); - } - - @Config("druid.zk.paths.indexer.leaderLatchPath") - public String getIndexerLeaderLatchPath() - { - return defaultPath("indexer/leaderLatchPath"); - } - - private String defaultPath(final String subPath) - { - return ZKPaths.makePath(getZkBasePath(), subPath); + @Override + public boolean equals(Object other){ + if(null == other){ + return false; + } + if(this == other){ + return true; + } + if(!(other instanceof ZkPathsConfig)){ + return false; + } + ZkPathsConfig otherConfig = (ZkPathsConfig) other; + if( + this.getBase().equals(otherConfig.getBase()) && + this.getAnnouncementsPath().equals(otherConfig.getAnnouncementsPath()) && + this.getConnectorPath().equals(otherConfig.getConnectorPath()) && + this.getLiveSegmentsPath().equals(otherConfig.getLiveSegmentsPath()) && + this.getCoordinatorPath().equals(otherConfig.getCoordinatorPath()) && + this.getLoadQueuePath().equals(otherConfig.getLoadQueuePath()) && + this.getPropertiesPath().equals(otherConfig.getPropertiesPath()) && + this.getServedSegmentsPath().equals(otherConfig.getServedSegmentsPath()) + ){ + return true; + } + return false; } } diff --git a/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java b/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java index c2ed92d170c..a635e55d688 100644 --- a/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java +++ b/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java @@ -120,7 +120,7 @@ public class BatchServerInventoryViewTest new ZkPathsConfig() { @Override - public String getZkBasePath() + public String getBase() { return testBasePath; } @@ -139,7 +139,7 @@ public class BatchServerInventoryViewTest new ZkPathsConfig() { @Override - public String getZkBasePath() + public String getBase() { return testBasePath; } @@ -155,7 +155,7 @@ public class BatchServerInventoryViewTest new ZkPathsConfig() { @Override - public String getZkBasePath() + public String getBase() { return testBasePath; } diff --git a/server/src/test/java/io/druid/curator/CuratorConfigTest.java b/server/src/test/java/io/druid/curator/CuratorConfigTest.java new file mode 100644 index 00000000000..846d9f37155 --- /dev/null +++ b/server/src/test/java/io/druid/curator/CuratorConfigTest.java @@ -0,0 +1,39 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.curator; + +import io.druid.guice.JsonConfigTesterBase; +import org.junit.Assert; +import org.junit.Test; + +import java.lang.reflect.InvocationTargetException; + +public class CuratorConfigTest extends JsonConfigTesterBase +{ + @Test + public void testHostName() throws IllegalAccessException, NoSuchMethodException, InvocationTargetException + { + propertyValues.put(getPropertyKey("host"),"fooHost"); + testProperties.putAll(propertyValues); + configProvider.inject(testProperties, configurator); + CuratorConfig config = configProvider.get().get(); + Assert.assertEquals("fooHost", config.getZkHosts()); + } +} diff --git a/server/src/test/java/io/druid/guice/JsonConfigTesterBase.java b/server/src/test/java/io/druid/guice/JsonConfigTesterBase.java new file mode 100644 index 00000000000..02779522798 --- /dev/null +++ b/server/src/test/java/io/druid/guice/JsonConfigTesterBase.java @@ -0,0 +1,155 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.guice; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.name.Names; +import io.druid.initialization.Initialization; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.lang.reflect.ParameterizedType; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; + +/** + * + */ +public abstract class JsonConfigTesterBase +{ + + protected static final String configPrefix = "druid.test.prefix"; + protected Injector injector; + protected final Class clazz = (Class) ((ParameterizedType) getClass().getGenericSuperclass()).getActualTypeArguments()[0]; + + protected Map propertyValues = new HashMap<>(); + protected int assertions = 0; + protected Properties testProperties = new Properties(); + + protected static String getPropertyKey(String fieldName){ + return String.format( + "%s.%s", + configPrefix, fieldName + ); + } + protected static String getPropertyKey(Field field) + { + JsonProperty jsonProperty = field.getAnnotation(JsonProperty.class); + if (null != jsonProperty) { + return getPropertyKey( + (jsonProperty.value() == null || jsonProperty.value().isEmpty()) + ? field.getName() + : jsonProperty.value() + ); + } + return null; + } + + private final Module simpleJsonConfigModule = new Module() + { + @Override + public void configure(Binder binder) + { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/test"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); + JsonConfigProvider.bind(binder, configPrefix, clazz); + } + }; + + + protected final void validateEntries(T config) + throws IllegalAccessException, NoSuchMethodException, InvocationTargetException + { + for (Field field : clazz.getDeclaredFields()) { + final String propertyKey = getPropertyKey(field); + if (null != propertyKey) { + field.setAccessible(true); + String getter = String.format( + "get%s%s", + field.getName().substring(0, 1).toUpperCase(), + field.getName().substring(1) + ); + Method method = clazz.getDeclaredMethod(getter); + final String value; + if (null != method) { + value = method.invoke(config).toString(); + } else { + value = field.get(config).toString(); + } + + Assert.assertEquals(propertyValues.get(propertyKey), value); + ++assertions; + } + } + } + + protected JsonConfigurator configurator; + protected JsonConfigProvider configProvider; + + @Before + public void setup() throws IllegalAccessException + { + assertions = 0; + T fakeValues = EasyMock.createNiceMock(clazz); + propertyValues.clear(); + testProperties.clear(); + for (Field field : clazz.getDeclaredFields()) { + final String propertyKey = getPropertyKey(field); + if (null != propertyKey) { + field.setAccessible(true); + if (field.getType().isAssignableFrom(String.class)) { + propertyValues.put(propertyKey, UUID.randomUUID().toString()); + } else { + propertyValues.put(propertyKey, field.get(fakeValues).toString()); + } + } + } + testProperties.putAll(System.getProperties()); + testProperties.putAll(propertyValues); + injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + ImmutableList.of(simpleJsonConfigModule) + ); + configurator = injector.getBinding(JsonConfigurator.class).getProvider().get(); + configProvider = JsonConfigProvider.of(configPrefix, clazz); + } + + @Test + public final void simpleInjectionTest() + throws IllegalAccessException, NoSuchMethodException, InvocationTargetException + { + configProvider.inject(testProperties, configurator); + validateEntries(configProvider.get().get()); + Assert.assertEquals(propertyValues.size(), assertions); + } + + +} diff --git a/server/src/test/java/io/druid/initialization/ZkPathsConfigTest.java b/server/src/test/java/io/druid/initialization/ZkPathsConfigTest.java new file mode 100644 index 00000000000..ee795c605fa --- /dev/null +++ b/server/src/test/java/io/druid/initialization/ZkPathsConfigTest.java @@ -0,0 +1,75 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.initialization; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Key; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.JsonConfigTesterBase; +import io.druid.guice.JsonConfigurator; +import io.druid.guice.annotations.Json; +import io.druid.server.initialization.ZkPathsConfig; +import org.apache.curator.utils.ZKPaths; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.util.UUID; + +/** + * + */ +public class ZkPathsConfigTest extends JsonConfigTesterBase +{ + @Test + public void testOverrideBaseOnlyConfig() + throws IllegalAccessException, NoSuchMethodException, InvocationTargetException, IOException + { + JsonConfigurator configurator = injector.getBinding(JsonConfigurator.class).getProvider().get(); + + JsonConfigProvider zkPathsConfig = JsonConfigProvider.of(configPrefix, ZkPathsConfig.class); + testProperties.clear(); + String base = UUID.randomUUID().toString(); + testProperties.put(String.format("%s.base", configPrefix), base); + zkPathsConfig.inject(testProperties, configurator); + + propertyValues.clear(); + propertyValues.put(String.format("%s.base", configPrefix), base); + propertyValues.put(String.format("%s.propertiesPath", configPrefix), ZKPaths.makePath(base, "properties")); + propertyValues.put(String.format("%s.announcementsPath", configPrefix), ZKPaths.makePath(base, "announcements")); + propertyValues.put(String.format("%s.servedSegmentsPath", configPrefix), ZKPaths.makePath(base, "servedSegments")); + propertyValues.put(String.format("%s.liveSegmentsPath", configPrefix), ZKPaths.makePath(base, "segments")); + propertyValues.put(String.format("%s.coordinatorPath", configPrefix), ZKPaths.makePath(base, "coordinator")); + propertyValues.put(String.format("%s.loadQueuePath", configPrefix), ZKPaths.makePath(base, "loadQueue")); + propertyValues.put(String.format("%s.connectorPath", configPrefix), ZKPaths.makePath(base, "connector")); + + ZkPathsConfig zkPathsConfigObj = zkPathsConfig.get().get(); + validateEntries(zkPathsConfigObj); + Assert.assertEquals(propertyValues.size(), assertions); + + ObjectMapper jsonMapper = injector.getProvider(Key.get(ObjectMapper.class, Json.class)).get(); + String jsonVersion = jsonMapper.writeValueAsString(zkPathsConfigObj); + + ZkPathsConfig zkPathsConfigObjDeSer = jsonMapper.readValue(jsonVersion, ZkPathsConfig.class); + + Assert.assertEquals(zkPathsConfigObj, zkPathsConfigObjDeSer); + } +} diff --git a/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java b/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java index 7440367a562..a5306307629 100644 --- a/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java +++ b/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java @@ -90,12 +90,6 @@ public class DruidClusterBridgeTest ObjectMapper jsonMapper = new DefaultObjectMapper(); DruidClusterBridgeConfig config = new DruidClusterBridgeConfig() { - @Override - public String getTier() - { - return DruidServer.DEFAULT_TIER; - } - @Override public Duration getStartDelay() { @@ -107,18 +101,6 @@ public class DruidClusterBridgeTest { return new Duration(Long.MAX_VALUE); } - - @Override - public String getBrokerServiceName() - { - return "testz0rz"; - } - - @Override - public int getPriority() - { - return 0; - } }; ScheduledExecutorFactory factory = ScheduledExecutors.createFactory(new Lifecycle()); @@ -134,7 +116,7 @@ public class DruidClusterBridgeTest ZkPathsConfig zkPathsConfig = new ZkPathsConfig() { @Override - public String getZkBasePath() + public String getBase() { return "/druid"; } @@ -195,6 +177,8 @@ public class DruidClusterBridgeTest DruidClusterBridge bridge = new DruidClusterBridge( jsonMapper, config, + zkPathsConfig, + metadata, factory, me, localCf, diff --git a/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java b/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java index e6216082e50..cb49380d8b5 100644 --- a/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java @@ -101,7 +101,7 @@ public class ZkCoordinatorTest extends CuratorTestBase final ZkPathsConfig zkPaths = new ZkPathsConfig() { @Override - public String getZkBasePath() + public String getBase() { return "/druid"; } diff --git a/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java b/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java index d4d2f8d7aaa..2ba2b0ba135 100644 --- a/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java +++ b/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java @@ -107,7 +107,7 @@ public class BatchDataSegmentAnnouncerTest new ZkPathsConfig() { @Override - public String getZkBasePath() + public String getBase() { return testBasePath; } diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java index 42eb36be50b..bcc418e590e 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java @@ -99,7 +99,7 @@ public class DruidCoordinatorTest { @Override - public String getZkBasePath() + public String getBase() { return ""; } diff --git a/services/src/main/java/io/druid/cli/CliBridge.java b/services/src/main/java/io/druid/cli/CliBridge.java index 808b79bbb48..861150ba4a1 100644 --- a/services/src/main/java/io/druid/cli/CliBridge.java +++ b/services/src/main/java/io/druid/cli/CliBridge.java @@ -109,7 +109,7 @@ public class CliBridge extends ServerRunnable .retryPolicy(new BoundedExponentialBackoffRetry(1000, 45000, 30)) .compressionProvider( new PotentiallyGzippedCompressionProvider( - bridgeCuratorConfig.enableCompression() + bridgeCuratorConfig.getEnableCompression() ) ) .build(); From 071943a367d1dcc21897d80b228ccb4beab53e47 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 2 Jan 2015 11:37:03 -0800 Subject: [PATCH 56/71] fix LZF compression with buffers exceeding LZF chunk size --- pom.xml | 2 +- .../java/io/druid/segment/CompressedPools.java | 16 ++++++++-------- .../segment/data/CompressedObjectStrategy.java | 12 ++++++------ 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/pom.xml b/pom.xml index 413faf5b80d..b82343b5ff1 100644 --- a/pom.xml +++ b/pom.xml @@ -158,7 +158,7 @@ com.ning compress-lzf - 0.8.4 + 1.0.3 io.airlift diff --git a/processing/src/main/java/io/druid/segment/CompressedPools.java b/processing/src/main/java/io/druid/segment/CompressedPools.java index 2c9236396a2..2824ac3aabb 100644 --- a/processing/src/main/java/io/druid/segment/CompressedPools.java +++ b/processing/src/main/java/io/druid/segment/CompressedPools.java @@ -21,7 +21,7 @@ package io.druid.segment; import com.google.common.base.Supplier; import com.metamx.common.logger.Logger; -import com.ning.compress.lzf.ChunkEncoder; +import com.ning.compress.BufferRecycler; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidPool; @@ -36,23 +36,23 @@ public class CompressedPools private static final Logger log = new Logger(CompressedPools.class); public static final int BUFFER_SIZE = 0x10000; - private static final StupidPool chunkEncoderPool = new StupidPool( - new Supplier() + private static final StupidPool bufferRecyclerPool = new StupidPool( + new Supplier() { private final AtomicLong counter = new AtomicLong(0); @Override - public ChunkEncoder get() + public BufferRecycler get() { - log.info("Allocating new chunkEncoder[%,d]", counter.incrementAndGet()); - return new ChunkEncoder(BUFFER_SIZE); + log.info("Allocating new bufferRecycler[%,d]", counter.incrementAndGet()); + return new BufferRecycler(); } } ); - public static ResourceHolder getChunkEncoder() + public static ResourceHolder getBufferRecycler() { - return chunkEncoderPool.take(); + return bufferRecyclerPool.take(); } private static final StupidPool outputBytesPool = new StupidPool( diff --git a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java index b646fd800c2..9144f1a3b2b 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java @@ -21,9 +21,10 @@ package io.druid.segment.data; import com.google.common.collect.Maps; import com.metamx.common.logger.Logger; +import com.ning.compress.BufferRecycler; import com.ning.compress.lzf.ChunkEncoder; -import com.ning.compress.lzf.LZFChunk; import com.ning.compress.lzf.LZFDecoder; +import com.ning.compress.lzf.LZFEncoder; import io.druid.collections.ResourceHolder; import io.druid.segment.CompressedPools; import net.jpountz.lz4.LZ4Factory; @@ -190,7 +191,7 @@ public class CompressedObjectStrategy implements ObjectStrateg out.flip(); } catch (IOException e) { - log.error(e, "IOException thrown while closing ChunkEncoder."); + log.error(e, "Error decompressing data"); } } @@ -209,12 +210,11 @@ public class CompressedObjectStrategy implements ObjectStrateg public byte[] compress(byte[] bytes) { - try (final ResourceHolder encoder = CompressedPools.getChunkEncoder()) { - final LZFChunk chunk = encoder.get().encodeChunk(bytes, 0, bytes.length); - return chunk.getData(); + try (final ResourceHolder bufferRecycler = CompressedPools.getBufferRecycler()) { + return LZFEncoder.encode(bytes, 0, bytes.length, bufferRecycler.get()); } catch (IOException e) { - log.error(e, "IOException thrown while closing ChunkEncoder."); + log.error(e, "Error compressing data"); } // IOException should be on ResourceHolder.close(), not encodeChunk, so this *should* never happen return null; From f2f9cbeca8f3a14078f5786c8505b6f256405439 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 2 Jan 2015 13:41:32 -0800 Subject: [PATCH 57/71] throw error rather than returning garbage results --- .../java/io/druid/segment/data/CompressedObjectStrategy.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java index 9144f1a3b2b..17703f10104 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java @@ -19,6 +19,7 @@ package io.druid.segment.data; +import com.google.common.base.Throwables; import com.google.common.collect.Maps; import com.metamx.common.logger.Logger; import com.ning.compress.BufferRecycler; @@ -215,9 +216,8 @@ public class CompressedObjectStrategy implements ObjectStrateg } catch (IOException e) { log.error(e, "Error compressing data"); + throw Throwables.propagate(e); } - // IOException should be on ResourceHolder.close(), not encodeChunk, so this *should* never happen - return null; } } From 3fc6cf918d44488dceb90b75cb4ee2afc1c90a21 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 2 Jan 2015 13:41:52 -0800 Subject: [PATCH 58/71] add test for large chunks --- .../CompressedFloatsIndexedSupplierTest.java | 85 ++++++++++---- .../CompressedLongsIndexedSupplierTest.java | 105 ++++++++++++++---- 2 files changed, 151 insertions(+), 39 deletions(-) diff --git a/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java index c1a55700b05..5e5a00985e3 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java @@ -21,6 +21,7 @@ package io.druid.segment.data; import com.google.common.io.Closeables; import com.google.common.primitives.Floats; +import io.druid.segment.CompressedPools; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -34,6 +35,9 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.FloatBuffer; import java.nio.channels.Channels; +import java.util.Arrays; +import java.util.Collections; +import java.util.Random; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; @@ -87,6 +91,11 @@ public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest 0.0f, 0.1f, 0.2f, 0.3f, 0.4f, 0.5f, 0.6f, 0.7f, 0.8f, 0.9f, 0.10f, 0.11f, 0.12f, 0.13f, 0.14f, 0.15f, 0.16f }; + makeWithSerde(chunkSize); + } + + private void makeWithSerde(int chunkSize) throws IOException + { ByteArrayOutputStream baos = new ByteArrayOutputStream(); final CompressedFloatsIndexedSupplier theSupplier = CompressedFloatsIndexedSupplier.fromFloatBuffer( FloatBuffer.wrap(vals), chunkSize, ByteOrder.nativeOrder(), compressionStrategy @@ -100,28 +109,53 @@ public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest indexed = supplier.get(); } + private void setupLargeChunks(final int chunkSize, final int totalSize) throws IOException + { + vals = new float[totalSize]; + Random rand = new Random(0); + for(int i = 0; i < vals.length; ++i) { + vals[i] = (float)rand.nextGaussian(); + } + + makeWithSerde(chunkSize); + } + @Test public void testSanity() throws Exception { setupSimple(5); - Assert.assertEquals(4, supplier.getBaseFloatBuffers().size()); - - Assert.assertEquals(vals.length, indexed.size()); - for (int i = 0; i < indexed.size(); ++i) { - Assert.assertEquals(vals[i], indexed.get(i), 0.0); - } + assertIndexMatchesVals(); // test powers of 2 setupSimple(2); - Assert.assertEquals(9, supplier.getBaseFloatBuffers().size()); + assertIndexMatchesVals(); + } - Assert.assertEquals(vals.length, indexed.size()); - for (int i = 0; i < indexed.size(); ++i) { - Assert.assertEquals(vals[i], indexed.get(i), 0.0); - } + @Test + public void testLargeChunks() throws Exception + { + final int maxChunkSize = CompressedPools.BUFFER_SIZE / Floats.BYTES; + setupLargeChunks(maxChunkSize, 10 * maxChunkSize); + Assert.assertEquals(10, supplier.getBaseFloatBuffers().size()); + assertIndexMatchesVals(); + + setupLargeChunks(maxChunkSize, 10 * maxChunkSize + 1); + Assert.assertEquals(11, supplier.getBaseFloatBuffers().size()); + assertIndexMatchesVals(); + + setupLargeChunks(maxChunkSize - 1, 10 * (maxChunkSize - 1) + 1); + Assert.assertEquals(11, supplier.getBaseFloatBuffers().size()); + assertIndexMatchesVals(); + } + + @Test(expected = IllegalArgumentException.class) + public void testChunkTooBig() throws Exception + { + final int maxChunkSize = CompressedPools.BUFFER_SIZE / Floats.BYTES; + setupLargeChunks(maxChunkSize + 1, 10 * (maxChunkSize + 1)); } @Test @@ -149,20 +183,14 @@ public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest Assert.assertEquals(4, supplier.getBaseFloatBuffers().size()); - Assert.assertEquals(vals.length, indexed.size()); - for (int i = 0; i < indexed.size(); ++i) { - Assert.assertEquals(vals[i], indexed.get(i), 0.0); - } + assertIndexMatchesVals(); // test powers of 2 setupSimpleWithSerde(2); Assert.assertEquals(9, supplier.getBaseFloatBuffers().size()); - Assert.assertEquals(vals.length, indexed.size()); - for (int i = 0; i < indexed.size(); ++i) { - Assert.assertEquals(vals[i], indexed.get(i), 0.0); - } + assertIndexMatchesVals(); } @Test @@ -296,4 +324,23 @@ public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest Assert.assertEquals(vals[i + startIndex], filled[i], 0.0); } } + + private void assertIndexMatchesVals() + { + Assert.assertEquals(vals.length, indexed.size()); + + // sequential access + int[] indices = new int[vals.length]; + for (int i = 0; i < indexed.size(); ++i) { + Assert.assertEquals(vals[i], indexed.get(i), 0.0); + indices[i] = i; + } + + Collections.shuffle(Arrays.asList(indices)); + // random access + for (int i = 0; i < indexed.size(); ++i) { + int k = indices[i]; + Assert.assertEquals(vals[k], indexed.get(k), 0.0); + } + } } diff --git a/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java index b251134ddc7..d8bedf1d988 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java @@ -21,6 +21,7 @@ package io.druid.segment.data; import com.google.common.primitives.Longs; import com.metamx.common.guava.CloseQuietly; +import io.druid.segment.CompressedPools; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -34,6 +35,9 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.LongBuffer; import java.nio.channels.Channels; +import java.util.Arrays; +import java.util.Collections; +import java.util.Random; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; @@ -65,13 +69,13 @@ public class CompressedLongsIndexedSupplierTest extends CompressionStrategyTest CloseQuietly.close(indexed); } - private void setupSimple() + private void setupSimple(final int chunkSize) { vals = new long[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 16}; supplier = CompressedLongsIndexedSupplier.fromLongBuffer( LongBuffer.wrap(vals), - 5, + chunkSize, ByteOrder.nativeOrder(), compressionStrategy ); @@ -79,13 +83,18 @@ public class CompressedLongsIndexedSupplierTest extends CompressionStrategyTest indexed = supplier.get(); } - private void setupSimpleWithSerde() throws IOException + private void setupSimpleWithSerde(final int chunkSize) throws IOException { vals = new long[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 16}; + makeWithSerde(chunkSize); + } + + private void makeWithSerde(final int chunkSize) throws IOException + { ByteArrayOutputStream baos = new ByteArrayOutputStream(); final CompressedLongsIndexedSupplier theSupplier = CompressedLongsIndexedSupplier.fromLongBuffer( - LongBuffer.wrap(vals), 5, ByteOrder.nativeOrder(), compressionStrategy + LongBuffer.wrap(vals), chunkSize, ByteOrder.nativeOrder(), compressionStrategy ); theSupplier.writeToChannel(Channels.newChannel(baos)); @@ -96,23 +105,64 @@ public class CompressedLongsIndexedSupplierTest extends CompressionStrategyTest indexed = supplier.get(); } + private void setupLargeChunks(final int chunkSize, final int totalSize) throws IOException + { + vals = new long[totalSize]; + Random rand = new Random(0); + for(int i = 0; i < vals.length; ++i) { + vals[i] = rand.nextLong(); + } + + makeWithSerde(chunkSize); + } + @Test public void testSanity() throws Exception { - setupSimple(); + setupSimple(5); Assert.assertEquals(4, supplier.getBaseLongBuffers().size()); + assertIndexMatchesVals(); - Assert.assertEquals(vals.length, indexed.size()); - for (int i = 0; i < indexed.size(); ++i) { - Assert.assertEquals(vals[i], indexed.get(i)); - } + // test powers of 2 + setupSimple(4); + Assert.assertEquals(4, supplier.getBaseLongBuffers().size()); + assertIndexMatchesVals(); + + setupSimple(32); + Assert.assertEquals(1, supplier.getBaseLongBuffers().size()); + assertIndexMatchesVals(); + } + + @Test + public void testLargeChunks() throws Exception + { + final int maxChunkSize = CompressedPools.BUFFER_SIZE / Longs.BYTES; + + setupLargeChunks(maxChunkSize, 10 * maxChunkSize); + Assert.assertEquals(10, supplier.getBaseLongBuffers().size()); + assertIndexMatchesVals(); + + setupLargeChunks(maxChunkSize, 10 * maxChunkSize + 1); + Assert.assertEquals(11, supplier.getBaseLongBuffers().size()); + assertIndexMatchesVals(); + + setupLargeChunks(maxChunkSize - 1, 10 * (maxChunkSize - 1) + 1); + Assert.assertEquals(11, supplier.getBaseLongBuffers().size()); + assertIndexMatchesVals(); + } + + @Test(expected = IllegalArgumentException.class) + public void testChunkTooBig() throws Exception + { + final int maxChunkSize = CompressedPools.BUFFER_SIZE / Longs.BYTES; + setupLargeChunks(maxChunkSize + 1, 10 * (maxChunkSize + 1)); } @Test public void testBulkFill() throws Exception { - setupSimple(); + setupSimple(5); tryFill(0, 15); tryFill(3, 6); @@ -123,27 +173,23 @@ public class CompressedLongsIndexedSupplierTest extends CompressionStrategyTest @Test(expected = IndexOutOfBoundsException.class) public void testBulkFillTooMuch() throws Exception { - setupSimple(); + setupSimple(5); tryFill(7, 10); } @Test public void testSanityWithSerde() throws Exception { - setupSimpleWithSerde(); + setupSimpleWithSerde(5); Assert.assertEquals(4, supplier.getBaseLongBuffers().size()); - - Assert.assertEquals(vals.length, indexed.size()); - for (int i = 0; i < indexed.size(); ++i) { - Assert.assertEquals(vals[i], indexed.get(i)); - } + assertIndexMatchesVals(); } @Test public void testBulkFillWithSerde() throws Exception { - setupSimpleWithSerde(); + setupSimpleWithSerde(5); tryFill(0, 15); tryFill(3, 6); @@ -154,7 +200,7 @@ public class CompressedLongsIndexedSupplierTest extends CompressionStrategyTest @Test(expected = IndexOutOfBoundsException.class) public void testBulkFillTooMuchWithSerde() throws Exception { - setupSimpleWithSerde(); + setupSimpleWithSerde(5); tryFill(7, 10); } @@ -163,7 +209,7 @@ public class CompressedLongsIndexedSupplierTest extends CompressionStrategyTest @Test public void testConcurrentThreadReads() throws Exception { - setupSimple(); + setupSimple(5); final AtomicReference reason = new AtomicReference("none"); @@ -271,4 +317,23 @@ public class CompressedLongsIndexedSupplierTest extends CompressionStrategyTest Assert.assertEquals(vals[i + startIndex], filled[i]); } } + + private void assertIndexMatchesVals() + { + Assert.assertEquals(vals.length, indexed.size()); + + // sequential access + int[] indices = new int[vals.length]; + for (int i = 0; i < indexed.size(); ++i) { + Assert.assertEquals(vals[i], indexed.get(i), 0.0); + indices[i] = i; + } + + Collections.shuffle(Arrays.asList(indices)); + // random access + for (int i = 0; i < indexed.size(); ++i) { + int k = indices[i]; + Assert.assertEquals(vals[k], indexed.get(k), 0.0); + } + } } From 81fdc791e09340e41a024f77c3280a4cf72b7490 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 2 Jan 2015 14:05:57 -0800 Subject: [PATCH 59/71] Add memory hinting and concurrency helpers in CachingClusteredClient --- .../java/io/druid/client/CachingClusteredClient.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 0fe5bf4b97c..71eb5492d93 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -64,12 +64,12 @@ import io.druid.server.coordination.DruidServerMetadata; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineLookup; import io.druid.timeline.TimelineObjectHolder; -import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.PartitionChunk; import org.joda.time.Interval; import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -191,8 +191,8 @@ public class CachingClusteredClient implements QueryRunner final byte[] queryCacheKey; - if ( (populateCache || useCache) // implies strategy != null - && !isBySegment ) // explicit bySegment queries are never cached + if ((populateCache || useCache) // implies strategy != null + && !isBySegment) // explicit bySegment queries are never cached { queryCacheKey = strategy.computeCacheKey(query); } else { @@ -400,7 +400,7 @@ public class CachingClusteredClient implements QueryRunner String.format("%s_%s", value.getSegmentId(), value.getInterval()) ); - final List cacheData = Lists.newLinkedList(); + final Collection cacheData = new ConcurrentLinkedQueue<>(); return Sequences.withEffect( Sequences.map( @@ -446,6 +446,9 @@ public class CachingClusteredClient implements QueryRunner try { Futures.allAsList(cacheFutures).get(); cachePopulator.populate(cacheData); + // Help out GC by making sure all references are gone + cacheFutures.clear(); + cacheData.clear(); } catch (Exception e) { log.error(e, "Error populating cache"); From f1375b0bfb52edb676d75e504a09d3ab15a0b6c6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 2 Jan 2015 17:26:59 -0800 Subject: [PATCH 60/71] workaround to pass down bitmap type to map-reduce tasks --- .../druid/indexer/HadoopDruidIndexerConfig.java | 5 +++++ .../java/io/druid/indexer/IndexGeneratorJob.java | 16 ++++++++++++++++ 2 files changed, 21 insertions(+) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index 8638e53b1d4..bd245e4f3e6 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -67,6 +67,7 @@ import java.io.IOException; import java.nio.charset.Charset; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Set; import java.util.SortedSet; @@ -83,6 +84,9 @@ public class HadoopDruidIndexerConfig public static final Joiner tabJoiner = Joiner.on("\t"); public static final ObjectMapper jsonMapper; + // workaround to pass down druid.processing.bitmap.type, see IndexGeneratorJob.run() + protected static final Properties properties; + private static final String DEFAULT_WORKING_PATH = "/tmp/druid-indexing"; static { @@ -102,6 +106,7 @@ public class HadoopDruidIndexerConfig ) ); jsonMapper = injector.getInstance(ObjectMapper.class); + properties = injector.getInstance(Properties.class); } public static enum IndexJobCounters diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index bc8ca44dd33..5b1b71bb0d9 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -21,6 +21,7 @@ package io.druid.indexer; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; +import com.google.common.base.Strings; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -44,6 +45,8 @@ import io.druid.segment.LoggingProgressIndicator; import io.druid.segment.ProgressIndicator; import io.druid.segment.QueryableIndex; import io.druid.segment.SegmentUtils; +import io.druid.segment.data.BitmapSerde; +import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.OffheapIncrementalIndex; @@ -180,6 +183,19 @@ public class IndexGeneratorJob implements Jobby config.addInputPaths(job); config.addJobProperties(job); + + // hack to get druid.processing.bitmap property passed down to hadoop job. + // once IndexIO doesn't rely on globally injected properties, we can move this into the HadoopTuningConfig. + final String bitmapProperty = "druid.processing.bitmap.type"; + final String bitmapType = HadoopDruidIndexerConfig.properties.getProperty(bitmapProperty); + if(bitmapType != null) { + for(String property : new String[] {"mapreduce.reduce.java.opts", "mapreduce.map.java.opts"}) { + // prepend property to allow overriding using hadoop.xxx properties by JobHelper.injectSystemProperties above + String value = Strings.nullToEmpty(job.getConfiguration().get(property)); + job.getConfiguration().set(property, String.format("-D%s=%s %s", bitmapProperty, bitmapType, value)); + } + } + config.intoConfiguration(job); JobHelper.setupClasspath(config, job); From c20142f2a905ecd9cfc0c6433abd01947dff2d4a Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 5 Jan 2015 13:27:29 -0800 Subject: [PATCH 61/71] Add some missing versions for maven plugins --- pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pom.xml b/pom.xml index b82343b5ff1..dbf595f0207 100644 --- a/pom.xml +++ b/pom.xml @@ -593,6 +593,14 @@ true + + maven-source-plugin + 2.4 + + + maven-javadoc-plugin + 2.10.1 + From f6fbb733b87492813eb2d067d1b96afb92bbea18 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 5 Jan 2015 13:47:25 -0800 Subject: [PATCH 62/71] Added a few places where tests were using Object instead of Module --- .../io/druid/server/initialization/IndexerZkConfigTest.java | 2 +- server/src/test/java/io/druid/guice/JsonConfigTesterBase.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/test/java/io/druid/server/initialization/IndexerZkConfigTest.java b/indexing-service/src/test/java/io/druid/server/initialization/IndexerZkConfigTest.java index 422fc05af83..d19d7b6490d 100644 --- a/indexing-service/src/test/java/io/druid/server/initialization/IndexerZkConfigTest.java +++ b/indexing-service/src/test/java/io/druid/server/initialization/IndexerZkConfigTest.java @@ -155,7 +155,7 @@ public class IndexerZkConfigTest { final Injector injector = Initialization.makeInjectorWithModules( GuiceInjectors.makeStartupInjector(), - ImmutableList.of(simpleZkConfigModule) + ImmutableList.of(simpleZkConfigModule) ); JsonConfigurator configurator = injector.getBinding(JsonConfigurator.class).getProvider().get(); diff --git a/server/src/test/java/io/druid/guice/JsonConfigTesterBase.java b/server/src/test/java/io/druid/guice/JsonConfigTesterBase.java index 02779522798..4aeb37659bc 100644 --- a/server/src/test/java/io/druid/guice/JsonConfigTesterBase.java +++ b/server/src/test/java/io/druid/guice/JsonConfigTesterBase.java @@ -136,7 +136,7 @@ public abstract class JsonConfigTesterBase testProperties.putAll(propertyValues); injector = Initialization.makeInjectorWithModules( GuiceInjectors.makeStartupInjector(), - ImmutableList.of(simpleJsonConfigModule) + ImmutableList.of(simpleJsonConfigModule) ); configurator = injector.getBinding(JsonConfigurator.class).getProvider().get(); configProvider = JsonConfigProvider.of(configPrefix, clazz); From dde21a16e1b29b446b219084635f8fc5edecd332 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 7 Jan 2015 09:55:33 -0800 Subject: [PATCH 63/71] Improve maxPendingPersists docs. --- docs/content/Plumber.md | 2 +- docs/content/Realtime-ingestion.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/content/Plumber.md b/docs/content/Plumber.md index 82ed7f69141..d80091d2b80 100644 --- a/docs/content/Plumber.md +++ b/docs/content/Plumber.md @@ -13,7 +13,7 @@ The following can be configured on the plumber: * `windowPeriod` is the amount of lag time to allow events. This is configured with a 10 minute window, meaning that any event more than 10 minutes ago will be thrown away and not included in the segment generated by the realtime server. * `basePersistDirectory` is the directory to put things that need persistence. The plumber is responsible for the actual intermediate persists and this tells it where to store those persists. -* `maxPendingPersists` is how many persists a plumber can do concurrently without starting to block. +* `maxPendingPersists` is the maximum number of persists that can be pending, but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. * `segmentGranularity` specifies the granularity of the segment, or the amount of time a segment will represent. * `rejectionPolicy` controls how data sets the data acceptance policy for creating and handing off segments. The following policies are available: * `serverTime` – The recommended policy for "current time" data, it is optimal for current data that is generated and ingested in real time. Uses `windowPeriod` to accept only those events that are inside the window looking forward and back. diff --git a/docs/content/Realtime-ingestion.md b/docs/content/Realtime-ingestion.md index e996c49ce54..23afede2759 100644 --- a/docs/content/Realtime-ingestion.md +++ b/docs/content/Realtime-ingestion.md @@ -134,13 +134,13 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |Field|Type|Description|Required| |-----|----|-----------|--------| |type|String|This should always be 'realtime'.|no| -|maxRowsInMemory|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size.|no (default == 5 million)| +|maxRowsInMemory|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 5 million)| |windowPeriod|ISO 8601 Period String|The amount of lag time to allow events. This is configured with a 10 minute window, meaning that any event more than 10 minutes ago will be thrown away and not included in the segment generated by the realtime server.|no (default == PT10m)| |intermediatePersistPeriod|ISO8601 Period String|The period that determines the rate at which intermediate persists occur. These persists determine how often commits happen against the incoming realtime stream. If the realtime data loading process is interrupted at time T, it should be restarted to re-read data that arrived at T minus this period.|no (default == PT10m)| |basePersistDirectory|String|The directory to put things that need persistence. The plumber is responsible for the actual intermediate persists and this tells it where to store those persists.|no (default == java tmp dir)| |versioningPolicy|Object|How to version segments.|no (default == based on segment start time)| |rejectionPolicy|Object|Controls how data sets the data acceptance policy for creating and handing off segments. More on this below.|no (default=='serverTime')| -|maxPendingPersists|Integer|How many persists a plumber can do concurrently without starting to block.|no (default == 0)| +|maxPendingPersists|Integer|Maximum number of persists that can be pending, but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0; meaning one persist can be running concurrently with ingestion, and none can be queued up)| |shardSpec|Object|This describes the shard that is represented by this server. This must be specified properly in order to have multiple realtime nodes indexing the same data stream in a sharded fashion.|no (default == 'NoneShardSpec'| #### Rejection Policy From c1295202ffc5bdd49128840b1ce653f5f33cab37 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 5 Jan 2015 17:55:51 -0800 Subject: [PATCH 64/71] Allow background caching run after the futures are finished --- .../druid/client/CachingClusteredClient.java | 37 ++++++++++++------- 1 file changed, 24 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 71eb5492d93..14da6225f9a 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -31,6 +31,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; +import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; @@ -411,7 +412,7 @@ public class CachingClusteredClient implements QueryRunner @Override public T apply(final T input) { - if(cachePopulator != null) { + if (cachePopulator != null) { // only compute cache data if populating cache cacheFutures.add( backgroundExecutorService.submit( @@ -443,21 +444,31 @@ public class CachingClusteredClient implements QueryRunner public void run() { if (cachePopulator != null) { - try { - Futures.allAsList(cacheFutures).get(); - cachePopulator.populate(cacheData); - // Help out GC by making sure all references are gone - cacheFutures.clear(); - cacheData.clear(); - } - catch (Exception e) { - log.error(e, "Error populating cache"); - throw Throwables.propagate(e); - } + Futures.addCallback( + Futures.allAsList(cacheFutures), + new FutureCallback>() + { + @Override + public void onSuccess(List objects) + { + cachePopulator.populate(cacheData); + // Help out GC by making sure all references are gone + cacheFutures.clear(); + cacheData.clear(); + } + + @Override + public void onFailure(Throwable throwable) + { + log.error(throwable, "Background caching failed"); + } + }, + backgroundExecutorService + ); } } }, - backgroundExecutorService + MoreExecutors.sameThreadExecutor() );// End withEffect } } From f69e0591b06ae095a9271f3fda24792436fb66f3 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Thu, 6 Nov 2014 23:53:24 +0530 Subject: [PATCH 65/71] working integration tests --- integration-tests/README.md | 90 + integration-tests/docker/Dockerfile | 71 + integration-tests/docker/broker.conf | 29 + integration-tests/docker/coordinator.conf | 23 + integration-tests/docker/historical.conf | 27 + .../docker/metadata-storage.conf | 6 + integration-tests/docker/middlemanager.conf | 29 + integration-tests/docker/overlord.conf | 25 + integration-tests/docker/router.conf | 20 + integration-tests/docker/sample-data.sql | 5 + integration-tests/docker/supervisord.conf | 6 + integration-tests/docker/zookeeper.conf | 5 + integration-tests/pom.xml | 151 ++ integration-tests/run_cluster.sh | 49 + .../druid/testing/DockerConfigProvider.java | 67 + .../testing/IntegrationTestingConfig.java | 33 + .../IntegrationTestingConfigProvider.java | 32 + .../CoordinatorResourceTestClient.java | 146 ++ .../EventReceiverFirehoseTestClient.java | 135 + .../clients/OverlordResourceTestClient.java | 213 ++ .../clients/QueryResourceTestClient.java | 98 + .../testing/clients/TaskResponseObject.java | 68 + .../druid/testing/guice/DruidTestModule.java | 54 + .../testing/guice/DruidTestModuleFactory.java | 63 + .../utils/FromFileTestQueryHelper.java | 79 + .../testing/utils/QueryResultVerifier.java | 49 + .../druid/testing/utils/QueryWithResults.java | 64 + .../io/druid/testing/utils/RetryUtil.java | 76 + .../testing/utils/ServerDiscoveryUtil.java | 64 + .../org/testng/DruidTestRunnerFactory.java | 150 ++ .../src/main/java/org/testng/TestNG.java | 2252 +++++++++++++++++ .../java/org/testng/remote/RemoteTestNG.java | 336 +++ .../tests/indexer/AbstractIndexerTest.java | 85 + .../io/druid/tests/indexer/ITIndexerTest.java | 81 + .../indexer/ITRealtimeIndexTaskTest.java | 142 ++ .../druid/tests/indexer/ITUnionQueryTest.java | 172 ++ .../druid/tests/query/ITTwitterQueryTest.java | 65 + .../tests/query/ITWikipediaQueryTest.java | 65 + .../src/test/resources/indexer/select.query | 19 + .../resources/indexer/select_reindex.query | 19 + .../test/resources/indexer/union_queries.json | 564 +++++ .../resources/indexer/union_select_query.json | 121 + .../indexer/wikipedia_index_data.json | 5 + .../indexer/wikipedia_index_queries.json | 16 + .../indexer/wikipedia_index_task.json | 59 + .../wikipedia_realtime_index_task.json | 71 + .../indexer/wikipedia_reindex_task.json | 62 + .../queries/twitterstream_queries.json | 780 ++++++ .../queries/wikipedia_editstream_queries.json | 1063 ++++++++ .../src/test/resources/testng.xml | 10 + integration-tests/stop_cluster.sh | 5 + pom.xml | 6 + 52 files changed, 7925 insertions(+) create mode 100644 integration-tests/README.md create mode 100644 integration-tests/docker/Dockerfile create mode 100644 integration-tests/docker/broker.conf create mode 100644 integration-tests/docker/coordinator.conf create mode 100644 integration-tests/docker/historical.conf create mode 100644 integration-tests/docker/metadata-storage.conf create mode 100644 integration-tests/docker/middlemanager.conf create mode 100644 integration-tests/docker/overlord.conf create mode 100644 integration-tests/docker/router.conf create mode 100644 integration-tests/docker/sample-data.sql create mode 100644 integration-tests/docker/supervisord.conf create mode 100644 integration-tests/docker/zookeeper.conf create mode 100644 integration-tests/pom.xml create mode 100755 integration-tests/run_cluster.sh create mode 100644 integration-tests/src/main/java/io/druid/testing/DockerConfigProvider.java create mode 100644 integration-tests/src/main/java/io/druid/testing/IntegrationTestingConfig.java create mode 100644 integration-tests/src/main/java/io/druid/testing/IntegrationTestingConfigProvider.java create mode 100644 integration-tests/src/main/java/io/druid/testing/clients/CoordinatorResourceTestClient.java create mode 100644 integration-tests/src/main/java/io/druid/testing/clients/EventReceiverFirehoseTestClient.java create mode 100644 integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java create mode 100644 integration-tests/src/main/java/io/druid/testing/clients/QueryResourceTestClient.java create mode 100644 integration-tests/src/main/java/io/druid/testing/clients/TaskResponseObject.java create mode 100644 integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java create mode 100644 integration-tests/src/main/java/io/druid/testing/guice/DruidTestModuleFactory.java create mode 100644 integration-tests/src/main/java/io/druid/testing/utils/FromFileTestQueryHelper.java create mode 100644 integration-tests/src/main/java/io/druid/testing/utils/QueryResultVerifier.java create mode 100644 integration-tests/src/main/java/io/druid/testing/utils/QueryWithResults.java create mode 100644 integration-tests/src/main/java/io/druid/testing/utils/RetryUtil.java create mode 100644 integration-tests/src/main/java/io/druid/testing/utils/ServerDiscoveryUtil.java create mode 100644 integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java create mode 100644 integration-tests/src/main/java/org/testng/TestNG.java create mode 100644 integration-tests/src/main/java/org/testng/remote/RemoteTestNG.java create mode 100644 integration-tests/src/test/java/io/druid/tests/indexer/AbstractIndexerTest.java create mode 100644 integration-tests/src/test/java/io/druid/tests/indexer/ITIndexerTest.java create mode 100644 integration-tests/src/test/java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java create mode 100644 integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java create mode 100644 integration-tests/src/test/java/io/druid/tests/query/ITTwitterQueryTest.java create mode 100644 integration-tests/src/test/java/io/druid/tests/query/ITWikipediaQueryTest.java create mode 100644 integration-tests/src/test/resources/indexer/select.query create mode 100644 integration-tests/src/test/resources/indexer/select_reindex.query create mode 100644 integration-tests/src/test/resources/indexer/union_queries.json create mode 100644 integration-tests/src/test/resources/indexer/union_select_query.json create mode 100644 integration-tests/src/test/resources/indexer/wikipedia_index_data.json create mode 100644 integration-tests/src/test/resources/indexer/wikipedia_index_queries.json create mode 100644 integration-tests/src/test/resources/indexer/wikipedia_index_task.json create mode 100644 integration-tests/src/test/resources/indexer/wikipedia_realtime_index_task.json create mode 100644 integration-tests/src/test/resources/indexer/wikipedia_reindex_task.json create mode 100644 integration-tests/src/test/resources/queries/twitterstream_queries.json create mode 100644 integration-tests/src/test/resources/queries/wikipedia_editstream_queries.json create mode 100644 integration-tests/src/test/resources/testng.xml create mode 100755 integration-tests/stop_cluster.sh diff --git a/integration-tests/README.md b/integration-tests/README.md new file mode 100644 index 00000000000..abeae63a0aa --- /dev/null +++ b/integration-tests/README.md @@ -0,0 +1,90 @@ +Integration Testing +========================= + +## Installing Docker and Running + +Please refer to instructions at [https://github.com/druid-io/docker-druid/blob/master/docker-install.md](https://github.com/druid-io/docker-druid/blob/master/docker-install.md) + +Instead of running +``` +boot2docker init +``` + +run instead +``` +boot2docker init -m 6000 +``` + +Make sure that you have at least 6GB of memory available before you run the tests. + +Set the docker ip via: +``` +export DOCKER_IP=$(boot2docker ip 2>/dev/null) +``` + +Verify that docker is running by issuing the following command: + +``` +docker info +``` + +Running Integration tests +========================= + +## Running tests using mvn + +To run all the tests using mvn run the following command - +''''' + mvn verify -P integration-tests +''''' + +To run only a single test using mvn run following command - +''''' + mvn verify -P integration-tests -Dit.test= +''''' + + +Writing a New Test +=============== + +## What should we cover in integration tests + +For every end-user functionality provided by druid we should have an integration-test verifying the correctness. + +## Rules to be followed while writing a new integration test + +### Every Integration Test must follow these rules + +1) Name of the test must start with a prefix "IT" +2) A test should be independent of other tests +3) Tests are to be written in TestNG style ([http://testng.org/doc/documentation-main.html#methods](http://testng.org/doc/documentation-main.html#methods)) +4) If a test loads some data it is the responsibility of the test to clean up the data from the cluster + +### How to use Guice Dependency Injection in a test + +A test can access different helper and utility classes provided by test-framework in order to access Coordinator,Broker etc.. +To mark a test be able to use Guice Dependency Injection - +Annotate the test class with the below annotation + + ''''''' + @Guice(moduleFactory = DruidTestModuleFactory.class) + ''''''' +This will tell the test framework that the test class needs to be constructed using guice. + +### Helper Classes provided + +1) IntegrationTestingConfig - configuration of the test +2) CoordinatorResourceTestClient - httpclient for coordinator endpoints +3) OverlordResourceTestClient - httpclient for indexer endpoints +4) QueryResourceTestClient - httpclient for broker endpoints + +### Static Utility classes + +1) RetryUtil - provides methods to retry an operation until it succeeds for configurable no. of times +2) FromFileTestQueryHelper - reads queries with expected results from file and executes them and verifies the results using ResultVerifier + +Refer ITIndexerTest as an example on how to use dependency Injection + +TODOS +======================= +1) Remove the patch for TestNG after resolution of Surefire-622 diff --git a/integration-tests/docker/Dockerfile b/integration-tests/docker/Dockerfile new file mode 100644 index 00000000000..8f03b95cd0d --- /dev/null +++ b/integration-tests/docker/Dockerfile @@ -0,0 +1,71 @@ +FROM ubuntu:14.04 + +# Add Java 7 repository +RUN apt-get update +RUN apt-get install -y software-properties-common +RUN apt-add-repository -y ppa:webupd8team/java +RUN apt-get update + +# Oracle Java 7 +RUN echo oracle-java-7-installer shared/accepted-oracle-license-v1-1 select true | /usr/bin/debconf-set-selections +RUN apt-get install -y oracle-java7-installer +RUN apt-get install -y oracle-java7-set-default + +# MySQL (Metadata store) +RUN apt-get install -y mysql-server + +# Supervisor +RUN apt-get install -y supervisor + +# Maven +RUN wget -q -O - http://www.us.apache.org/dist/maven/maven-3/3.2.5/binaries/apache-maven-3.2.5-bin.tar.gz | tar -xzf - -C /usr/local +RUN ln -s /usr/local/apache-maven-3.2.1 /usr/local/apache-maven +RUN ln -s /usr/local/apache-maven/bin/mvn /usr/local/bin/mvn + +# Zookeeper +RUN wget -q -O - http://www.us.apache.org/dist/zookeeper/zookeeper-3.4.6/zookeeper-3.4.6.tar.gz | tar -xzf - -C /usr/local +RUN cp /usr/local/zookeeper-3.4.6/conf/zoo_sample.cfg /usr/local/zookeeper-3.4.6/conf/zoo.cfg +RUN ln -s /usr/local/zookeeper-3.4.6 /usr/local/zookeeper + +# git +RUN apt-get install -y git + +# Druid system user +RUN adduser --system --group --no-create-home druid +RUN mkdir -p /var/lib/druid +RUN chown druid:druid /var/lib/druid + +# Add druid jars +ADD lib/* /usr/local/druid/lib/ + +WORKDIR / + +# Setup metadata store +RUN /etc/init.d/mysql start && echo "GRANT ALL ON druid.* TO 'druid'@'%' IDENTIFIED BY 'diurd'; CREATE database druid;" | mysql -u root && /etc/init.d/mysql stop + +# Add sample data +RUN /etc/init.d/mysql start && java -Ddruid.metadata.storage.type=mysql -cp "/usr/local/druid/lib/*" io.druid.cli.Main tools metadata-init --connectURI="jdbc:mysql://localhost:3306/druid" --user=druid --password=diurd && /etc/init.d/mysql stop +ADD sample-data.sql sample-data.sql +RUN /etc/init.d/mysql start && cat sample-data.sql | mysql -u root druid && /etc/init.d/mysql stop + +# Setup supervisord +ADD supervisord.conf /etc/supervisor/conf.d/supervisord.conf + +# Clean up +RUN apt-get clean && rm -rf /tmp/* /var/tmp/* + +# Expose ports: +# - 8081: HTTP (coordinator) +# - 8082: HTTP (broker) +# - 8083: HTTP (historical) +# - 3306: MySQL +# - 2181 2888 3888: ZooKeeper +# - 8100 8101 8102 8103 8104 : peon ports +EXPOSE 8081 +EXPOSE 8082 +EXPOSE 8083 +EXPOSE 3306 +EXPOSE 2181 2888 3888 +EXPOSE 8100 8101 8102 8103 8104 +WORKDIR /var/lib/druid +ENTRYPOINT export HOST_IP="$(resolveip -s $HOSTNAME)" && exec /usr/bin/supervisord -c /etc/supervisor/conf.d/supervisord.conf diff --git a/integration-tests/docker/broker.conf b/integration-tests/docker/broker.conf new file mode 100644 index 00000000000..63b6af12e3c --- /dev/null +++ b/integration-tests/docker/broker.conf @@ -0,0 +1,29 @@ +[program:druid-broker] +command=java + -server + -Xmx1g + -Xms1g + -XX:NewSize=500m + -XX:MaxNewSize=500m + -XX:+UseConcMarkSweepGC + -XX:+PrintGCDetails + -XX:+PrintGCTimeStamps + -Duser.timezone=UTC + -Dfile.encoding=UTF-8 + -Ddruid.host=%(ENV_HOST_IP)s + -Ddruid.zk.service.host=druid-zookeeper + -Ddruid.processing.buffer.sizeBytes=75000000 + -Ddruid.server.http.numThreads=100 + -Ddruid.processing.numThreads=1 + -Ddruid.broker.http.numConnections=30 + -Ddruid.broker.http.readTimeout=PT5M + -Ddruid.broker.cache.useCache=true + -Ddruid.broker.cache.populateCache=true + -Ddruid.cache.type=local + -Ddruid.cache.sizeInBytes=40000000 + -cp /usr/local/druid/lib/* + io.druid.cli.Main server broker +redirect_stderr=true +autorestart=false +priority=100 +stdout_logfile=/shared/logs/broker.log diff --git a/integration-tests/docker/coordinator.conf b/integration-tests/docker/coordinator.conf new file mode 100644 index 00000000000..60cf6b97362 --- /dev/null +++ b/integration-tests/docker/coordinator.conf @@ -0,0 +1,23 @@ +[program:druid-coordinator] +command=java + -server + -Xmx128m + -Xms128m + -XX:+UseConcMarkSweepGC + -XX:+PrintGCDetails + -XX:+PrintGCTimeStamps + -Duser.timezone=UTC + -Dfile.encoding=UTF-8 + -Ddruid.host=%(ENV_HOST_IP)s + -Ddruid.metadata.storage.type=mysql + -Ddruid.metadata.storage.connector.connectURI=jdbc:mysql://druid-metadata-storage/druid + -Ddruid.metadata.storage.connector.user=druid + -Ddruid.metadata.storage.connector.password=diurd + -Ddruid.zk.service.host=druid-zookeeper + -Ddruid.coordinator.startDelay=PT5S + -cp /usr/local/druid/lib/* + io.druid.cli.Main server coordinator +redirect_stderr=true +priority=100 +autorestart=false +stdout_logfile=/shared/logs/coordinator.log diff --git a/integration-tests/docker/historical.conf b/integration-tests/docker/historical.conf new file mode 100644 index 00000000000..ab02bbc664a --- /dev/null +++ b/integration-tests/docker/historical.conf @@ -0,0 +1,27 @@ +[program:druid-historical] +command=java + -server + -Xmx1500m + -Xms1500m + -XX:NewSize=750m + -XX:MaxNewSize=750m + -XX:+UseConcMarkSweepGC + -XX:+PrintGCDetails + -XX:+PrintGCTimeStamps + -Duser.timezone=UTC + -Dfile.encoding=UTF-8 + -Ddruid.host=%(ENV_HOST_IP)s + -Ddruid.zk.service.host=druid-zookeeper + -Ddruid.s3.accessKey=AKIAIMKECRUYKDQGR6YQ + -Ddruid.s3.secretKey=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 + -Ddruid.processing.buffer.sizeBytes=75000000 + -Ddruid.processing.numThreads=3 + -Ddruid.server.http.numThreads=100 + -Ddruid.segmentCache.locations="[{\"path\":\"/shared/druid/indexCache\",\"maxSize\":5000000000}]" + -Ddruid.server.maxSize=5000000000 + -cp /usr/local/druid/lib/* + io.druid.cli.Main server historical +redirect_stderr=true +priority=100 +autorestart=false +stdout_logfile=/shared/logs/historical.log diff --git a/integration-tests/docker/metadata-storage.conf b/integration-tests/docker/metadata-storage.conf new file mode 100644 index 00000000000..eb60e214665 --- /dev/null +++ b/integration-tests/docker/metadata-storage.conf @@ -0,0 +1,6 @@ +[program:mysql] +command=/usr/bin/pidproxy /var/run/mysqld/mysqld.pid /usr/bin/mysqld_safe + --bind-address=0.0.0.0 +user=mysql +priority=0 +stdout_logfile=/shared/logs/mysql.log diff --git a/integration-tests/docker/middlemanager.conf b/integration-tests/docker/middlemanager.conf new file mode 100644 index 00000000000..cf1436182e7 --- /dev/null +++ b/integration-tests/docker/middlemanager.conf @@ -0,0 +1,29 @@ +[program:druid-middlemanager] +command=java + -server + -Xmx64m + -Xms64m + -XX:+UseConcMarkSweepGC + -XX:+PrintGCDetails + -XX:+PrintGCTimeStamps + -Duser.timezone=UTC + -Dfile.encoding=UTF-8 + -Ddruid.host=%(ENV_HOST_IP)s + -Ddruid.zk.service.host=druid-zookeeper + -Ddruid.indexer.logs.directory=/shared/tasklogs + -Ddruid.storage.storageDirectory=/shared/storage + -Ddruid.indexer.runner.javaOpts=-server -Xmx256m -Xms256m -XX:NewSize=128m -XX:MaxNewSize=128m -XX:+UseConcMarkSweepGC -XX:+PrintGCDetails -XX:+PrintGCTimeStamps + -Ddruid.indexer.fork.property.druid.processing.buffer.sizeBytes=75000000 + -Ddruid.indexer.fork.property.druid.processing.numThreads=1 + -Ddruid.indexer.fork.server.http.numThreads=100 + -Ddruid.s3.accessKey=AKIAIMKECRUYKDQGR6YQ + -Ddruid.s3.secretKey=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 + -Ddruid.worker.ip=%(ENV_HOST_IP)s + -Ddruid.selectors.indexing.serviceName=druid:overlord + -Ddruid.indexer.task.chathandler.type=announce + -cp /usr/local/druid/lib/* + io.druid.cli.Main server middleManager +redirect_stderr=true +priority=100 +autorestart=false +stdout_logfile=/shared/logs/middlemanager.log diff --git a/integration-tests/docker/overlord.conf b/integration-tests/docker/overlord.conf new file mode 100644 index 00000000000..a0d436c5a02 --- /dev/null +++ b/integration-tests/docker/overlord.conf @@ -0,0 +1,25 @@ +[program:druid-overlord] +command=java + -server + -Xmx128m + -Xms128m + -XX:+UseConcMarkSweepGC + -XX:+PrintGCDetails + -XX:+PrintGCTimeStamps + -Duser.timezone=UTC + -Dfile.encoding=UTF-8 + -Ddruid.host=%(ENV_HOST_IP)s + -Ddruid.metadata.storage.type=mysql + -Ddruid.metadata.storage.connector.connectURI=jdbc:mysql://druid-metadata-storage/druid + -Ddruid.metadata.storage.connector.user=druid + -Ddruid.metadata.storage.connector.password=diurd + -Ddruid.zk.service.host=druid-zookeeper + -Ddruid.indexer.storage.type=metadata + -Ddruid.indexer.logs.directory=/shared/tasklogs + -Ddruid.indexer.runner.type=remote + -cp /usr/local/druid/lib/* + io.druid.cli.Main server overlord +redirect_stderr=true +priority=100 +autorestart=false +stdout_logfile=/shared/logs/overlord.log diff --git a/integration-tests/docker/router.conf b/integration-tests/docker/router.conf new file mode 100644 index 00000000000..ddd8121f6c9 --- /dev/null +++ b/integration-tests/docker/router.conf @@ -0,0 +1,20 @@ +[program:druid-router] +command=java + -server + -Xmx1g + -XX:+UseConcMarkSweepGC + -XX:+PrintGCDetails + -XX:+PrintGCTimeStamps + -Duser.timezone=UTC + -Dfile.encoding=UTF-8 + -Ddruid.host=%(ENV_HOST_IP)s + -Ddruid.zk.service.host=druid-zookeeper + -Ddruid.computation.buffer.size=75000000 + -Ddruid.server.http.numThreads=100 + -Ddruid.processing.numThreads=1 + -cp /usr/local/druid/lib/* + io.druid.cli.Main server router +redirect_stderr=true +priority=100 +autorestart=false +stdout_logfile=/shared/logs/router.log diff --git a/integration-tests/docker/sample-data.sql b/integration-tests/docker/sample-data.sql new file mode 100644 index 00000000000..8221ced23e1 --- /dev/null +++ b/integration-tests/docker/sample-data.sql @@ -0,0 +1,5 @@ +INSERT INTO druid_segments (id,dataSource,created_date,start,end,partitioned,version,used,payload) VALUES ('twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9','twitterstream','2013-05-13T01:08:18.192Z','2013-01-01T00:00:00.000Z','2013-01-02T00:00:00.000Z',0,'2013-01-02T04:13:41.980Z_v9',1,'{\"dataSource\":\"twitterstream\",\"interval\":\"2013-01-01T00:00:00.000Z/2013-01-02T00:00:00.000Z\",\"version\":\"2013-01-02T04:13:41.980Z_v9\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/twitterstream/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/2013-01-02T04:13:41.980Z_v9/0/index.zip\"},\"dimensions\":\"has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name\",\"metrics\":\"count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":445235220,\"identifier\":\"twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9\"}'); +INSERT INTO druid_segments (id,dataSource,created_date,start,end,partitioned,version,used,payload) VALUES ('twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9','twitterstream','2013-05-13T00:03:28.640Z','2013-01-02T00:00:00.000Z','2013-01-03T00:00:00.000Z',0,'2013-01-03T03:44:58.791Z_v9',1,'{\"dataSource\":\"twitterstream\",\"interval\":\"2013-01-02T00:00:00.000Z/2013-01-03T00:00:00.000Z\",\"version\":\"2013-01-03T03:44:58.791Z_v9\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/twitterstream/2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z/2013-01-03T03:44:58.791Z_v9/0/index.zip\"},\"dimensions\":\"has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name\",\"metrics\":\"count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":435325540,\"identifier\":\"twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9\"}'); +INSERT INTO druid_segments (id,dataSource,created_date,start,end,partitioned,version,used,payload) VALUES ('twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9','twitterstream','2013-05-13T00:03:48.807Z','2013-01-03T00:00:00.000Z','2013-01-04T00:00:00.000Z',0,'2013-01-04T04:09:13.590Z_v9',1,'{\"dataSource\":\"twitterstream\",\"interval\":\"2013-01-03T00:00:00.000Z/2013-01-04T00:00:00.000Z\",\"version\":\"2013-01-04T04:09:13.590Z_v9\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/twitterstream/2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z/2013-01-04T04:09:13.590Z_v9/0/index.zip\"},\"dimensions\":\"has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name\",\"metrics\":\"count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":411651320,\"identifier\":\"twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9\"}'); +INSERT INTO druid_segments (id,dataSource,created_date,start,end,partitioned,version,used,payload) VALUES ('wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9','wikipedia_editstream','2013-03-15T20:49:52.348Z','2012-12-29T00:00:00.000Z','2013-01-10T08:00:00.000Z',0,'2013-01-10T08:13:47.830Z_v9',1,'{\"dataSource\":\"wikipedia_editstream\",\"interval\":\"2012-12-29T00:00:00.000Z/2013-01-10T08:00:00.000Z\",\"version\":\"2013-01-10T08:13:47.830Z_v9\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/wikipedia_editstream/2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z/2013-01-10T08:13:47.830Z_v9/0/index.zip\"},\"dimensions\":\"anonymous,area_code,city,continent_code,country_name,dma_code,geo,language,namespace,network,newpage,page,postal_code,region_lookup,robot,unpatrolled,user\",\"metrics\":\"added,count,deleted,delta,delta_hist,unique_users,variation\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":446027801,\"identifier\":\"wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\"}'); +INSERT INTO druid_segments (id, dataSource, created_date, start, end, partitioned, version, used, payload) VALUES ('wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z', 'wikipedia', '2013-08-08T21:26:23.799Z', '2013-08-01T00:00:00.000Z', '2013-08-02T00:00:00.000Z', '0', '2013-08-08T21:22:48.989Z', '1', '{\"dataSource\":\"wikipedia\",\"interval\":\"2013-08-01T00:00:00.000Z/2013-08-02T00:00:00.000Z\",\"version\":\"2013-08-08T21:22:48.989Z\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/wikipedia/20130801T000000.000Z_20130802T000000.000Z/2013-08-08T21_22_48.989Z/0/index.zip\"},\"dimensions\":\"dma_code,continent_code,geo,area_code,robot,country_name,network,city,namespace,anonymous,unpatrolled,page,postal_code,language,newpage,user,region_lookup\",\"metrics\":\"count,delta,variation,added,deleted\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":24664730,\"identifier\":\"wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z\"}'); diff --git a/integration-tests/docker/supervisord.conf b/integration-tests/docker/supervisord.conf new file mode 100644 index 00000000000..99ab6b4b5cb --- /dev/null +++ b/integration-tests/docker/supervisord.conf @@ -0,0 +1,6 @@ +[supervisord] +nodaemon=true + +[include] +files = /usr/lib/druid/conf/*.conf + diff --git a/integration-tests/docker/zookeeper.conf b/integration-tests/docker/zookeeper.conf new file mode 100644 index 00000000000..1ee5247817f --- /dev/null +++ b/integration-tests/docker/zookeeper.conf @@ -0,0 +1,5 @@ +[program:zookeeper] +command=/usr/local/zookeeper/bin/zkServer.sh start-foreground +user=daemon +priority=0 +stdout_logfile=/shared/logs/zookeeper.log diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml new file mode 100644 index 00000000000..50c5e1f2ed3 --- /dev/null +++ b/integration-tests/pom.xml @@ -0,0 +1,151 @@ + + + + + 4.0.0 + io.druid + druid-integration-tests + druid-integration-tests + druid-integration-tests + + io.druid + druid + 0.7.0-SNAPSHOT + + + + + io.druid + druid-common + ${project.parent.version} + + + io.druid.extensions + druid-s3-extensions + ${project.parent.version} + + + io.druid.extensions + druid-histogram + ${project.parent.version} + + + io.druid.extensions + mysql-metadata-storage + ${project.parent.version} + + + io.druid + druid-services + ${project.parent.version} + + + io.druid + druid-server + ${project.parent.version} + + + + + org.testng + testng + + + org.easymock + easymock + test + + + + + + + maven-surefire-plugin + + + **/IT*.java + + + UTC + + + + + + + + integration-tests + + + + org.codehaus.mojo + exec-maven-plugin + 1.2.1 + + + build-and-start-druid-cluster + + exec + + pre-integration-test + + ${project.basedir}/run_cluster.sh + + + + stop-druid-cluster + + exec + + post-integration-test + + ${project.basedir}/stop_cluster.sh + + + + + + maven-failsafe-plugin + + + integration-tests + integration-test + + integration-test + verify + + + + + -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Dtestrunfactory=org.testng.DruidTestRunnerFactory + -Ddruid.test.config.dockerIp=${env.DOCKER_IP} -Ddruid.zk.service.host=${env.DOCKER_IP} + + + src/test/resources/testng.xml + + + + + + + + + diff --git a/integration-tests/run_cluster.sh b/integration-tests/run_cluster.sh new file mode 100755 index 00000000000..eca4e65e60e --- /dev/null +++ b/integration-tests/run_cluster.sh @@ -0,0 +1,49 @@ +# cleanup +for node in druid-historical druid-coordinator druid-overlord druid-router druid-broker druid-middlemanager druid-zookeeper druid-metadata-storage; +do +docker stop $node +docker rm $node +done + +# environment variables +DIR=$(cd $(dirname $0) && pwd) +DOCKERDIR=$DIR/docker +SHARED_DIR=${HOME}/shared +SUPERVISORDIR=/usr/lib/druid/conf +RESOURCEDIR=$DIR/src/test/resources + +# Make directories if they dont exist +mkdir -p $SHARED_DIR/logs +mkdir -p $SHARED_DIR/tasklogs + +# install druid jars +rm -rf $SHARED_DIR/docker +cp -R docker $SHARED_DIR/docker +mvn dependency:copy-dependencies -DoutputDirectory=$SHARED_DIR/docker/lib + +# Build Druid Cluster Image +docker build -t druid/cluster $SHARED_DIR/docker + +# Start zookeeper +docker run -d --name druid-zookeeper -p 2181:2181 -v $SHARED_DIR:/shared -v $DOCKERDIR/zookeeper.conf:$SUPERVISORDIR/zookeeper.conf druid/cluster + +# Start MYSQL +docker run -d --name druid-metadata-storage -v $SHARED_DIR:/shared -v $DOCKERDIR/metadata-storage.conf:$SUPERVISORDIR/metadata-storage.conf druid/cluster + +# Start Overlord +docker run -d --name druid-overlord -p 8090:8090 -v $SHARED_DIR:/shared -v $DOCKERDIR/overlord.conf:$SUPERVISORDIR/overlord.conf --link druid-metadata-storage:druid-metadata-storage --link druid-zookeeper:druid-zookeeper druid/cluster + +# Start Coordinator +docker run -d --name druid-coordinator -p 8081:8081 -v $SHARED_DIR:/shared -v $DOCKERDIR/coordinator.conf:$SUPERVISORDIR/coordinator.conf --link druid-overlord:druid-overlord --link druid-metadata-storage:druid-metadata-storage --link druid-zookeeper:druid-zookeeper druid/cluster + +# Start Historical +docker run -d --name druid-historical -v $SHARED_DIR:/shared -v $DOCKERDIR/historical.conf:$SUPERVISORDIR/historical.conf --link druid-zookeeper:druid-zookeeper druid/cluster + +# Start Middlemanger +docker run -d --name druid-middlemanager -p 8100:8100 -p 8101:8101 -p 8102:8102 -p 8103:8103 -p 8104:8104 -p 8105:8105 -v $RESOURCEDIR:/resources -v $SHARED_DIR:/shared -v $DOCKERDIR/middlemanager.conf:$SUPERVISORDIR/middlemanager.conf --link druid-zookeeper:druid-zookeeper --link druid-overlord:druid-overlord druid/cluster + +# Start Broker +docker run -d --name druid-broker -v $SHARED_DIR:/shared -v $DOCKERDIR/broker.conf:$SUPERVISORDIR/broker.conf --link druid-zookeeper:druid-zookeeper --link druid-middlemanager:druid-middlemanager --link druid-historical:druid-historical druid/cluster + +# Start Router +docker run -d --name druid-router -p 8888:8888 -v $SHARED_DIR:/shared -v $DOCKERDIR/router.conf:$SUPERVISORDIR/router.conf --link druid-zookeeper:druid-zookeeper --link druid-coordinator:druid-coordinator --link druid-broker:druid-broker druid/cluster diff --git a/integration-tests/src/main/java/io/druid/testing/DockerConfigProvider.java b/integration-tests/src/main/java/io/druid/testing/DockerConfigProvider.java new file mode 100644 index 00000000000..a11e9b082ff --- /dev/null +++ b/integration-tests/src/main/java/io/druid/testing/DockerConfigProvider.java @@ -0,0 +1,67 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.testing; + + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.api.client.repackaged.com.google.common.base.Throwables; +import org.apache.commons.io.IOUtils; + +import javax.validation.constraints.NotNull; +import java.util.List; + +public class DockerConfigProvider implements IntegrationTestingConfigProvider +{ + + @JsonProperty + @NotNull + private String dockerIp; + + @Override + public IntegrationTestingConfig get() + { + return new IntegrationTestingConfig() + { + @Override + public String getCoordinatorHost() + { + return dockerIp+":8081"; + } + + @Override + public String getIndexerHost() + { + return dockerIp+":8090"; + } + + @Override + public String getRouterHost() + { + return dockerIp+ ":8888"; + } + + @Override + public String getMiddleManagerHost() + { + return dockerIp; + } + }; + } +} diff --git a/integration-tests/src/main/java/io/druid/testing/IntegrationTestingConfig.java b/integration-tests/src/main/java/io/druid/testing/IntegrationTestingConfig.java new file mode 100644 index 00000000000..dc94fb8ea1b --- /dev/null +++ b/integration-tests/src/main/java/io/druid/testing/IntegrationTestingConfig.java @@ -0,0 +1,33 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.testing; + +/** + */ +public interface IntegrationTestingConfig +{ + public String getCoordinatorHost(); + + public String getIndexerHost(); + + public String getRouterHost(); + + public String getMiddleManagerHost(); +} diff --git a/integration-tests/src/main/java/io/druid/testing/IntegrationTestingConfigProvider.java b/integration-tests/src/main/java/io/druid/testing/IntegrationTestingConfigProvider.java new file mode 100644 index 00000000000..7de9d8fce7d --- /dev/null +++ b/integration-tests/src/main/java/io/druid/testing/IntegrationTestingConfigProvider.java @@ -0,0 +1,32 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.testing; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.inject.Provider; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DockerConfigProvider.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "docker", value = DockerConfigProvider.class) +}) +public interface IntegrationTestingConfigProvider extends Provider +{ +} diff --git a/integration-tests/src/main/java/io/druid/testing/clients/CoordinatorResourceTestClient.java b/integration-tests/src/main/java/io/druid/testing/clients/CoordinatorResourceTestClient.java new file mode 100644 index 00000000000..cc0ac53f35a --- /dev/null +++ b/integration-tests/src/main/java/io/druid/testing/clients/CoordinatorResourceTestClient.java @@ -0,0 +1,146 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.testing.clients; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Charsets; +import com.google.common.base.Throwables; +import com.google.inject.Inject; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.RequestBuilder; +import com.metamx.http.client.response.StatusResponseHandler; +import com.metamx.http.client.response.StatusResponseHolder; +import io.druid.guice.annotations.Global; +import io.druid.testing.IntegrationTestingConfig; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.joda.time.Interval; + +import java.net.URL; +import java.net.URLEncoder; +import java.util.Map; + +public class CoordinatorResourceTestClient +{ + private final static Logger LOG = new Logger(CoordinatorResourceTestClient.class); + private final ObjectMapper jsonMapper; + private final HttpClient httpClient; + private final String coordinator; + private final StatusResponseHandler responseHandler; + + @Inject + CoordinatorResourceTestClient( + ObjectMapper jsonMapper, + @Global HttpClient httpClient, IntegrationTestingConfig config + ) + { + this.jsonMapper = jsonMapper; + this.httpClient = httpClient; + this.coordinator = config.getCoordinatorHost(); + this.responseHandler = new StatusResponseHandler(Charsets.UTF_8); + } + + private String getCoordinatorURL() + { + return String.format( + "http://%s/druid/coordinator/v1/", + coordinator + ); + } + + private Map getLoadStatus() + { + Map status = null; + try { + StatusResponseHolder response = makeRequest(HttpMethod.GET, getCoordinatorURL() + "loadstatus?simple"); + + status = jsonMapper.readValue( + response.getContent(), new TypeReference>() + { + } + ); + } + catch (Exception e) { + Throwables.propagate(e); + } + return status; + } + + public boolean areSegmentsLoaded(String dataSource) + { + final Map status = getLoadStatus(); + return (status.containsKey(dataSource) && status.get(dataSource) == 0); + } + + public void unloadSegmentsForDataSource(String dataSource, Interval interval) + { + killDataSource(dataSource, false, interval); + } + + public void deleteSegmentsDataSource(String dataSource, Interval interval) + { + killDataSource(dataSource, true, interval); + } + + private void killDataSource(String dataSource, boolean kill, Interval interval) + { + try { + makeRequest( + HttpMethod.DELETE, + String.format( + "%sdatasources/%s?kill=%s&interval=%s", + getCoordinatorURL(), + dataSource, kill, URLEncoder.encode(interval.toString(), "UTF-8") + ) + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + private StatusResponseHolder makeRequest(HttpMethod method, String url) + { + try { + StatusResponseHolder response = new RequestBuilder( + this.httpClient, + method, new URL(url) + ) + .go(responseHandler) + .get(); + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error while making request to url[%s] status[%s] content[%s]", + url, + response.getStatus(), + response.getContent() + ); + } + return response; + } + catch (Exception e) { + LOG.error(e, "Exception while sending request"); + throw Throwables.propagate(e); + } + } +} diff --git a/integration-tests/src/main/java/io/druid/testing/clients/EventReceiverFirehoseTestClient.java b/integration-tests/src/main/java/io/druid/testing/clients/EventReceiverFirehoseTestClient.java new file mode 100644 index 00000000000..ee212cdf112 --- /dev/null +++ b/integration-tests/src/main/java/io/druid/testing/clients/EventReceiverFirehoseTestClient.java @@ -0,0 +1,135 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.testing.clients; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.client.util.Charsets; +import com.google.common.base.Throwables; +import com.metamx.common.ISE; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.response.StatusResponseHandler; +import com.metamx.http.client.response.StatusResponseHolder; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; + +import javax.ws.rs.core.MediaType; +import java.io.BufferedReader; +import java.io.InputStreamReader; +import java.net.URL; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Map; + +public class EventReceiverFirehoseTestClient +{ + private final String host; + private final StatusResponseHandler responseHandler; + private final ObjectMapper jsonMapper; + private final HttpClient httpClient; + private final String chatID; + + public EventReceiverFirehoseTestClient(String host, String chatID, ObjectMapper jsonMapper, HttpClient httpClient) + { + this.host = host; + this.jsonMapper = jsonMapper; + this.responseHandler = new StatusResponseHandler(Charsets.UTF_8); + this.httpClient = httpClient; + this.chatID = chatID; + } + + private String getURL() + { + return String.format( + "http://%s/druid/worker/v1/chat/%s/push-events/", + host, + chatID + ); + } + + /** + * post events from the collection and return the count of events accepted + * + * @param events Collection of events to be posted + * + * @return + */ + public int postEvents(Collection> events) + { + try { + StatusResponseHolder response = httpClient.post(new URL(getURL())) + .setContent( + MediaType.APPLICATION_JSON, + this.jsonMapper.writeValueAsBytes(events) + ) + .go(responseHandler) + .get(); + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error while posting events to url[%s] status[%s] content[%s]", + getURL(), + response.getStatus(), + response.getContent() + ); + } + Map responseData = jsonMapper.readValue( + response.getContent(), new TypeReference>() + { + } + ); + return responseData.get("eventCount"); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public int postEventsFromFile(String file) + { + try { + BufferedReader reader = new BufferedReader( + new InputStreamReader( + EventReceiverFirehoseTestClient.class.getResourceAsStream( + file + ) + ) + ); + String s; + Collection> events = new ArrayList>(); + while ((s = reader.readLine()) != null) { + events.add( + (Map) this.jsonMapper.readValue( + s, new TypeReference>() + { + } + ) + ); + } + int eventsPosted = postEvents(events); + if (eventsPosted != events.size()) { + throw new ISE("All events not posted, expected : %d actual : %d", events.size(), eventsPosted); + } + return eventsPosted; + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + } +} diff --git a/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java b/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java new file mode 100644 index 00000000000..af97b8b0bcf --- /dev/null +++ b/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java @@ -0,0 +1,213 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.testing.clients; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Charsets; +import com.google.common.base.Throwables; +import com.google.inject.Inject; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.response.StatusResponseHandler; +import com.metamx.http.client.response.StatusResponseHolder; +import io.druid.guice.annotations.Global; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.task.Task; +import io.druid.testing.IntegrationTestingConfig; +import io.druid.testing.utils.RetryUtil; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; + +import java.net.URL; +import java.net.URLEncoder; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Callable; + +public class OverlordResourceTestClient +{ + private final static Logger LOG = new Logger(OverlordResourceTestClient.class); + private final ObjectMapper jsonMapper; + private final HttpClient httpClient; + private final String indexer; + private final StatusResponseHandler responseHandler; + + @Inject + OverlordResourceTestClient( + ObjectMapper jsonMapper, + @Global HttpClient httpClient, IntegrationTestingConfig config + ) + { + this.jsonMapper = jsonMapper; + this.httpClient = httpClient; + this.indexer = config.getIndexerHost(); + this.responseHandler = new StatusResponseHandler(Charsets.UTF_8); + } + + private String getIndexerURL() + { + return String.format( + "http://%s/druid/indexer/v1/", + indexer + ); + } + + public String submitTask(Task task) + { + try { + return submitTask(this.jsonMapper.writeValueAsString(task)); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public String submitTask(String task) + { + try { + StatusResponseHolder response = httpClient.post(new URL(getIndexerURL() + "task")) + .setContent( + "application/json", + task.getBytes() + ) + .go(responseHandler) + .get(); + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error while submitting task to indexer response [%s %s]", + response.getStatus(), + response.getContent() + ); + } + Map responseData = jsonMapper.readValue( + response.getContent(), new TypeReference>() + { + } + ); + String taskID = responseData.get("task"); + LOG.info("Submitted task with TaskID[%s]", taskID); + return taskID; + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public TaskStatus.Status getTaskStatus(String taskID) + { + try { + StatusResponseHolder response = makeRequest( + String.format( + "%stask/%s/status", + getIndexerURL(), + URLEncoder.encode(taskID, "UTF-8") + ) + ); + + LOG.info("Index status response" + response.getContent()); + Map responseData = jsonMapper.readValue( + response.getContent(), new TypeReference>() + { + } + ); + //TODO: figure out a better way to parse the response... + String status = (String) ((Map) responseData.get("status")).get("status"); + return TaskStatus.Status.valueOf(status); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public List getRunningTasks() + { + return getTasks("runningTasks"); + } + + public List getWaitingTasks() + { + return getTasks("waitingTasks"); + } + + public List getPendingTasks() + { + return getTasks("pendingTasks"); + } + + private List getTasks(String identifier) + { + try { + StatusResponseHolder response = makeRequest( + String.format("%s%s", getIndexerURL(), identifier) + ); + LOG.info("Tasks %s response %s", identifier, response.getContent()); + return jsonMapper.readValue( + response.getContent(), new TypeReference>() + { + } + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public void waitUntilTaskCompletes(final String taskID) + { + RetryUtil.retryUntil( + new Callable() + { + @Override + public Boolean call() throws Exception + { + TaskStatus.Status status = getTaskStatus(taskID); + if (status == TaskStatus.Status.FAILED) { + throw new ISE("Indexer task FAILED"); + } + return status == TaskStatus.Status.SUCCESS; + } + }, + true, + 60000, + 10, + "Index Task to complete" + ); + } + + private StatusResponseHolder makeRequest(String url) + { + try { + StatusResponseHolder response = this.httpClient + .get(new URL(url)) + .go(responseHandler) + .get(); + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE("Error while making request to indexer [%s %s]", response.getStatus(), response.getContent()); + } + return response; + } + catch (Exception e) { + LOG.error(e, "Exception while sending request"); + throw Throwables.propagate(e); + } + } + +} diff --git a/integration-tests/src/main/java/io/druid/testing/clients/QueryResourceTestClient.java b/integration-tests/src/main/java/io/druid/testing/clients/QueryResourceTestClient.java new file mode 100644 index 00000000000..4ba1d6ca7a5 --- /dev/null +++ b/integration-tests/src/main/java/io/druid/testing/clients/QueryResourceTestClient.java @@ -0,0 +1,98 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.testing.clients; + + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Charsets; +import com.google.common.base.Throwables; +import com.google.inject.Inject; +import com.metamx.common.ISE; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.response.StatusResponseHandler; +import com.metamx.http.client.response.StatusResponseHolder; +import io.druid.guice.annotations.Global; +import io.druid.query.Query; +import io.druid.testing.IntegrationTestingConfig; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; + +import java.net.URL; +import java.util.List; +import java.util.Map; + +public class QueryResourceTestClient +{ + private final ObjectMapper jsonMapper; + private final HttpClient httpClient; + private final String router; + private final StatusResponseHandler responseHandler; + + @Inject + QueryResourceTestClient( + ObjectMapper jsonMapper, + @Global HttpClient httpClient, + IntegrationTestingConfig config + ) + { + this.jsonMapper = jsonMapper; + this.httpClient = httpClient; + this.router = config.getRouterHost(); + this.responseHandler = new StatusResponseHandler(Charsets.UTF_8); + } + + private String getBrokerURL() + { + return String.format( + "http://%s/druid/v2/", + router + ); + } + + public List> query(Query query) + { + try { + StatusResponseHolder response = httpClient.post(new URL(getBrokerURL())) + .setContent( + "application/json", + jsonMapper.writeValueAsBytes(query) + ) + .go(responseHandler) + .get(); + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error while querying[%s] status[%s] content[%s]", + getBrokerURL(), + response.getStatus(), + response.getContent() + ); + } + + return jsonMapper.readValue( + response.getContent(), new TypeReference>>() + { + } + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/integration-tests/src/main/java/io/druid/testing/clients/TaskResponseObject.java b/integration-tests/src/main/java/io/druid/testing/clients/TaskResponseObject.java new file mode 100644 index 00000000000..8b138505a6e --- /dev/null +++ b/integration-tests/src/main/java/io/druid/testing/clients/TaskResponseObject.java @@ -0,0 +1,68 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.testing.clients; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.indexing.common.TaskStatus; +import org.joda.time.DateTime; + +public class TaskResponseObject +{ + + private final String id; + private final DateTime createdTime; + private final DateTime queueInsertionTime; + private final TaskStatus status; + + @JsonCreator + private TaskResponseObject( + @JsonProperty("id") String id, + @JsonProperty("createdTime") DateTime createdTime, + @JsonProperty("queueInsertionTime") DateTime queueInsertionTime, + @JsonProperty("status") TaskStatus status + ) + { + this.id = id; + this.createdTime = createdTime; + this.queueInsertionTime = queueInsertionTime; + this.status = status; + } + + public String getId() + { + return id; + } + + public DateTime getCreatedTime() + { + return createdTime; + } + + public DateTime getQueueInsertionTime() + { + return queueInsertionTime; + } + + public TaskStatus getStatus() + { + return status; + } +} diff --git a/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java b/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java new file mode 100644 index 00000000000..95dd35d3274 --- /dev/null +++ b/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java @@ -0,0 +1,54 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.testing.guice; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.emitter.core.LoggingEmitter; +import com.metamx.emitter.core.LoggingEmitterConfig; +import com.metamx.emitter.service.ServiceEmitter; +import io.druid.guice.ConfigProvider; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.LazySingleton; +import io.druid.guice.ManageLifecycle; +import io.druid.testing.IntegrationTestingConfig; +import io.druid.testing.IntegrationTestingConfigProvider; + +/** + */ +public class DruidTestModule implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(IntegrationTestingConfig.class).toProvider(IntegrationTestingConfigProvider.class).in(ManageLifecycle.class); + JsonConfigProvider.bind(binder, "druid.test.config", IntegrationTestingConfigProvider.class); + } + + @Provides + @LazySingleton + public ServiceEmitter getServiceEmitter(Supplier config, ObjectMapper jsonMapper) + { + return new ServiceEmitter("", "", new LoggingEmitter(config.get(), jsonMapper)); + } +} diff --git a/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModuleFactory.java b/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModuleFactory.java new file mode 100644 index 00000000000..062a87c130f --- /dev/null +++ b/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModuleFactory.java @@ -0,0 +1,63 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.testing.guice; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Injector; +import com.google.inject.Module; +import io.druid.guice.GuiceInjectors; +import io.druid.guice.IndexingServiceFirehoseModule; +import io.druid.initialization.Initialization; +import org.testng.IModuleFactory; +import org.testng.ITestContext; + +import java.util.Collections; +import java.util.List; + +public class DruidTestModuleFactory implements IModuleFactory +{ + private static final Module module = new DruidTestModule(); + private static final Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + getModules() + ); + + public static Injector getInjector() + { + return injector; + } + + private static List getModules() + { + return ImmutableList.of( + new DruidTestModule(), + new IndexingServiceFirehoseModule() + ); + } + + @Override + public Module createModule(ITestContext context, Class testClass) + { + context.addGuiceModule(DruidTestModule.class, module); + context.addInjector(Collections.singletonList(module), injector); + return module; + } + +} diff --git a/integration-tests/src/main/java/io/druid/testing/utils/FromFileTestQueryHelper.java b/integration-tests/src/main/java/io/druid/testing/utils/FromFileTestQueryHelper.java new file mode 100644 index 00000000000..4c815d68843 --- /dev/null +++ b/integration-tests/src/main/java/io/druid/testing/utils/FromFileTestQueryHelper.java @@ -0,0 +1,79 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.testing.utils; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import io.druid.testing.clients.QueryResourceTestClient; + +import java.util.List; +import java.util.Map; + +public class FromFileTestQueryHelper +{ + public static Logger LOG = new Logger(FromFileTestQueryHelper.class); + private final QueryResourceTestClient queryClient; + private final ObjectMapper jsonMapper; + + @Inject + FromFileTestQueryHelper(ObjectMapper jsonMapper, QueryResourceTestClient queryClient) + { + this.jsonMapper = jsonMapper; + this.queryClient = queryClient; + } + + public void testQueriesFromFile(String filePath, int timesToRun) throws Exception + { + LOG.info("Starting query tests for [%s]", filePath); + List queries = + jsonMapper.readValue( + FromFileTestQueryHelper.class.getResourceAsStream(filePath), + new TypeReference>() + { + } + ); + for (int i = 0; i < timesToRun; i++) { + LOG.info("Starting Iteration " + i); + + boolean failed = false; + for (QueryWithResults queryWithResult : queries) { + LOG.info("Running Query " + queryWithResult.getQuery().getType()); + List> result = queryClient.query(queryWithResult.getQuery()); + if (!QueryResultVerifier.compareResults(result, queryWithResult.getExpectedResults())) { + LOG.error( + "Failed while executing %s actualResults : %s", + queryWithResult, + jsonMapper.writeValueAsString(result) + ); + failed = true; + } else { + LOG.info("Results Verified for Query " + queryWithResult.getQuery().getType()); + } + } + + if (failed) { + throw new ISE("one or more twitter queries failed"); + } + } + } +} diff --git a/integration-tests/src/main/java/io/druid/testing/utils/QueryResultVerifier.java b/integration-tests/src/main/java/io/druid/testing/utils/QueryResultVerifier.java new file mode 100644 index 00000000000..fee833f5b29 --- /dev/null +++ b/integration-tests/src/main/java/io/druid/testing/utils/QueryResultVerifier.java @@ -0,0 +1,49 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.testing.utils; + +import java.util.Iterator; +import java.util.Map; + +public class QueryResultVerifier +{ + public static boolean compareResults( + Iterable> actual, + Iterable> expected + ) + { + Iterator> actualIter = actual.iterator(); + Iterator> expectedIter = expected.iterator(); + + while (actualIter.hasNext() && expectedIter.hasNext()) { + Map actualRes = actualIter.next(); + Map expRes = expectedIter.next(); + + if (!actualRes.equals(expRes)) { + return false; + } + } + + if (actualIter.hasNext() || expectedIter.hasNext()) { + return false; + } + return true; + } +} diff --git a/integration-tests/src/main/java/io/druid/testing/utils/QueryWithResults.java b/integration-tests/src/main/java/io/druid/testing/utils/QueryWithResults.java new file mode 100644 index 00000000000..2ad56d3263b --- /dev/null +++ b/integration-tests/src/main/java/io/druid/testing/utils/QueryWithResults.java @@ -0,0 +1,64 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.testing.utils; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.query.Query; + +import java.util.List; +import java.util.Map; + +public class QueryWithResults +{ + private final Query query; + private final List> expectedResults; + + @JsonCreator + public QueryWithResults( + @JsonProperty("query") Query query, + @JsonProperty("expectedResults") List> expectedResults + ) + { + this.query = query; + this.expectedResults = expectedResults; + } + + @JsonProperty + public Query getQuery() + { + return query; + } + + @JsonProperty + public List> getExpectedResults() + { + return expectedResults; + } + + @Override + public String toString() + { + return "QueryWithResults{" + + "query=" + query + + ", expectedResults=" + expectedResults + + '}'; + } +} \ No newline at end of file diff --git a/integration-tests/src/main/java/io/druid/testing/utils/RetryUtil.java b/integration-tests/src/main/java/io/druid/testing/utils/RetryUtil.java new file mode 100644 index 00000000000..0c8451bf039 --- /dev/null +++ b/integration-tests/src/main/java/io/druid/testing/utils/RetryUtil.java @@ -0,0 +1,76 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.testing.utils; + +import com.google.common.base.Throwables; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; + +import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; + +public class RetryUtil +{ + + private static final Logger LOG = new Logger(RetryUtil.class); + + public static int DEFAULT_RETRY_COUNT = 10; + + public static long DEFAULT_RETRY_SLEEP = TimeUnit.SECONDS.toMillis(30); + + public static void retryUntilTrue(Callable callable, String task) + { + retryUntil(callable, true, DEFAULT_RETRY_SLEEP, DEFAULT_RETRY_COUNT, task); + } + + public static void retryUntilFalse(Callable callable, String task) + { + retryUntil(callable, false, DEFAULT_RETRY_SLEEP, DEFAULT_RETRY_COUNT, task); + } + + public static void retryUntil( + Callable callable, + boolean expectedValue, + long delayInMillis, + int retryCount, + String taskMessage + ) + { + try { + int currentTry = 0; + while (callable.call() != expectedValue) { + if (currentTry > retryCount) { + throw new ISE("Max number of retries[%d] exceeded for Task[%s]. Failing.", retryCount, taskMessage); + } + LOG.info( + "Attempt[%d]: Task %s still not complete. Next retry in %d ms", + currentTry, taskMessage, delayInMillis + ); + Thread.sleep(delayInMillis); + + currentTry++; + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + +} diff --git a/integration-tests/src/main/java/io/druid/testing/utils/ServerDiscoveryUtil.java b/integration-tests/src/main/java/io/druid/testing/utils/ServerDiscoveryUtil.java new file mode 100644 index 00000000000..e675a2a35e9 --- /dev/null +++ b/integration-tests/src/main/java/io/druid/testing/utils/ServerDiscoveryUtil.java @@ -0,0 +1,64 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.testing.utils; + +import com.metamx.common.logger.Logger; +import io.druid.client.selector.Server; +import io.druid.curator.discovery.ServerDiscoverySelector; + +import java.util.concurrent.Callable; + +public class ServerDiscoveryUtil +{ + + private static final Logger LOG = new Logger(ServerDiscoveryUtil.class); + + public static boolean isInstanceReady(ServerDiscoverySelector serviceProvider) + { + try { + Server instance = serviceProvider.pick(); + if (instance == null) { + LOG.warn("Unable to find a host"); + return false; + } + } + catch (Exception e) { + LOG.error(e, "Caught exception waiting for host"); + return false; + } + return true; + } + + public static void waitUntilInstanceReady(final ServerDiscoverySelector serviceProvider, String instanceType) + { + RetryUtil.retryUntilTrue( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return isInstanceReady(serviceProvider); + } + }, + String.format("Instance %s to get ready", instanceType) + ); + } + +} diff --git a/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java b/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java new file mode 100644 index 00000000000..6d0f3684938 --- /dev/null +++ b/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java @@ -0,0 +1,150 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package org.testng; + +import com.google.api.client.repackaged.com.google.common.base.Throwables; +import com.google.api.client.util.Charsets; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.common.logger.Logger; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.response.StatusResponseHandler; +import com.metamx.http.client.response.StatusResponseHolder; +import io.druid.guice.annotations.Global; +import io.druid.testing.IntegrationTestingConfig; +import io.druid.testing.guice.DruidTestModuleFactory; +import io.druid.testing.utils.RetryUtil; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.testng.internal.IConfiguration; +import org.testng.internal.annotations.IAnnotationFinder; +import org.testng.xml.XmlTest; + +import java.net.URL; +import java.util.List; +import java.util.concurrent.Callable; + +public class DruidTestRunnerFactory implements ITestRunnerFactory +{ + private static final Logger LOG = new Logger(DruidTestRunnerFactory.class); + + @Override + public TestRunner newTestRunner( + ISuite suite, XmlTest test, List listeners + ) + { + IConfiguration configuration = TestNG.getDefault().getConfiguration(); + String outputDirectory = suite.getOutputDirectory(); + IAnnotationFinder annotationFinder = configuration.getAnnotationFinder(); + Boolean skipFailedInvocationCounts = suite.getXmlSuite().skipFailedInvocationCounts(); + return new DruidTestRunner( + configuration, + suite, + test, + outputDirectory, + annotationFinder, + skipFailedInvocationCounts, + listeners + ); + } + + private static class DruidTestRunner extends TestRunner + { + + protected DruidTestRunner( + IConfiguration configuration, + ISuite suite, + XmlTest test, + String outputDirectory, + IAnnotationFinder finder, + boolean skipFailedInvocationCounts, + List invokedMethodListeners + ) + { + super(configuration, suite, test, outputDirectory, finder, skipFailedInvocationCounts, invokedMethodListeners); + } + + @Override + public void run() + { + Injector injector = DruidTestModuleFactory.getInjector(); + IntegrationTestingConfig config = injector.getInstance(IntegrationTestingConfig.class); + HttpClient client = injector.getInstance(Key.get(HttpClient.class, Global.class)); + ; + waitUntilInstanceReady(client, config.getCoordinatorHost()); + waitUntilInstanceReady(client, config.getIndexerHost()); + waitUntilInstanceReady(client, config.getRouterHost()); + Lifecycle lifecycle = injector.getInstance(Lifecycle.class); + try { + lifecycle.start(); + runTests(); + } + catch (Exception e) { + e.printStackTrace(); + throw Throwables.propagate(e); + } + finally { + lifecycle.stop(); + } + + } + + private void runTests() + { + super.run(); + } + + public void waitUntilInstanceReady(final HttpClient client, final String host) + { + final StatusResponseHandler handler = new StatusResponseHandler(Charsets.UTF_8); + RetryUtil.retryUntilTrue( + new Callable() + { + @Override + public Boolean call() throws Exception + { + try { + StatusResponseHolder response = client.get( + new URL( + String.format( + "http://%s/status", + host + ) + ) + ) + .go(handler) + .get(); + System.out.println(response.getStatus() + response.getContent()); + if (response.getStatus().equals(HttpResponseStatus.OK)) { + return true; + } else { + return false; + } + } + catch (Throwable e) { + e.printStackTrace(); + return false; + } + } + }, "Waiting for instance to be ready: [" + host + "]" + ); + } + } +} diff --git a/integration-tests/src/main/java/org/testng/TestNG.java b/integration-tests/src/main/java/org/testng/TestNG.java new file mode 100644 index 00000000000..c02793e5486 --- /dev/null +++ b/integration-tests/src/main/java/org/testng/TestNG.java @@ -0,0 +1,2252 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package org.testng; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.ParameterException; +import org.testng.annotations.ITestAnnotation; +import org.testng.collections.Lists; +import org.testng.collections.Maps; +import org.testng.internal.ClassHelper; +import org.testng.internal.Configuration; +import org.testng.internal.DynamicGraph; +import org.testng.internal.IConfiguration; +import org.testng.internal.IResultListener2; +import org.testng.internal.OverrideProcessor; +import org.testng.internal.SuiteRunnerMap; +import org.testng.internal.Utils; +import org.testng.internal.Version; +import org.testng.internal.annotations.DefaultAnnotationTransformer; +import org.testng.internal.annotations.IAnnotationFinder; +import org.testng.internal.annotations.JDK15AnnotationFinder; +import org.testng.internal.annotations.Sets; +import org.testng.internal.thread.graph.GraphThreadPoolExecutor; +import org.testng.internal.thread.graph.IThreadWorkerFactory; +import org.testng.internal.thread.graph.SuiteWorkerFactory; +import org.testng.junit.JUnitTestFinder; +import org.testng.log4testng.Logger; +import org.testng.remote.SuiteDispatcher; +import org.testng.remote.SuiteSlave; +import org.testng.reporters.EmailableReporter; +import org.testng.reporters.EmailableReporter2; +import org.testng.reporters.FailedReporter; +import org.testng.reporters.JUnitReportReporter; +import org.testng.reporters.SuiteHTMLReporter; +import org.testng.reporters.VerboseReporter; +import org.testng.reporters.XMLReporter; +import org.testng.reporters.jq.Main; +import org.testng.xml.Parser; +import org.testng.xml.XmlClass; +import org.testng.xml.XmlInclude; +import org.testng.xml.XmlMethodSelector; +import org.testng.xml.XmlSuite; +import org.testng.xml.XmlTest; +import org.xml.sax.SAXException; + +import javax.xml.parsers.ParserConfigurationException; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Enumeration; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.jar.JarEntry; +import java.util.jar.JarFile; + +import static org.testng.internal.Utils.defaultIfStringEmpty; +import static org.testng.internal.Utils.isStringEmpty; +import static org.testng.internal.Utils.isStringNotEmpty; + +/** + * Class copied from TestNG library ver 6.8.7 to apply a workaround for http://jira.codehaus.org/browse/SUREFIRE-622 + * To Locate the PATCHED AREA search for keyword "PATCH" in this class file + *

+ *

+ * This class is the main entry point for running tests in the TestNG framework. + * Users can create their own TestNG object and invoke it in many different + * ways: + *

    + *
  • On an existing testng.xml + *
  • On a synthetic testng.xml, created entirely from Java + *
  • By directly setting the test classes + *
+ * You can also define which groups to include or exclude, assign parameters, etc... + *

+ * The command line parameters are: + *

    + *
  • -d outputdir: specify the output directory
  • + *
  • -testclass class_name: specifies one or several class names
  • + *
  • -testjar jar_name: specifies the jar containing the tests
  • + *
  • -sourcedir src1;src2: ; separated list of source directories + * (used only when javadoc annotations are used)
  • + *
  • -target
  • + *
  • -groups
  • + *
  • -testrunfactory
  • + *
  • -listener
  • + *
+ *

+ * Please consult documentation for more details. + *

+ * FIXME: should support more than simple paths for suite xmls + * + * @author Cedric Beust + * @author Alex Popescu + * @see #usage() + */ +public class TestNG +{ + + /** + * This class' log4testng Logger. + */ + private static final Logger LOGGER = Logger.getLogger(TestNG.class); + + /** + * The default name for a suite launched from the command line + */ + public static final String DEFAULT_COMMAND_LINE_SUITE_NAME = "Command line suite"; + + /** + * The default name for a test launched from the command line + */ + public static final String DEFAULT_COMMAND_LINE_TEST_NAME = "Command line test"; + + /** + * The default name of the result's output directory (keep public, used by Eclipse). + */ + public static final String DEFAULT_OUTPUTDIR = "test-output"; + + /** + * System properties + */ + public static final String SHOW_TESTNG_STACK_FRAMES = "testng.show.stack.frames"; + public static final String TEST_CLASSPATH = "testng.test.classpath"; + + private static TestNG m_instance; + + private static JCommander m_jCommander; + + private List m_commandLineMethods; + protected List m_suites = Lists.newArrayList(); + private List m_cmdlineSuites; + private String m_outputDir = DEFAULT_OUTPUTDIR; + + private String[] m_includedGroups; + private String[] m_excludedGroups; + + private Boolean m_isJUnit = XmlSuite.DEFAULT_JUNIT; + private Boolean m_isMixed = XmlSuite.DEFAULT_MIXED; + protected boolean m_useDefaultListeners = true; + + private ITestRunnerFactory m_testRunnerFactory; + + // These listeners can be overridden from the command line + private List m_testListeners = Lists.newArrayList(); + private List m_suiteListeners = Lists.newArrayList(); + private Set m_reporters = Sets.newHashSet(); + + protected static final int HAS_FAILURE = 1; + protected static final int HAS_SKIPPED = 2; + protected static final int HAS_FSP = 4; + protected static final int HAS_NO_TEST = 8; + + public static final Integer DEFAULT_VERBOSE = 1; + + private int m_status; + private boolean m_hasTests = false; + + private String m_slavefileName = null; + private String m_masterfileName = null; + + // Command line suite parameters + private int m_threadCount; + private boolean m_useThreadCount; + private String m_parallelMode; + private boolean m_useParallelMode; + private String m_configFailurePolicy; + private Class[] m_commandLineTestClasses; + + private String m_defaultSuiteName = DEFAULT_COMMAND_LINE_SUITE_NAME; + private String m_defaultTestName = DEFAULT_COMMAND_LINE_TEST_NAME; + + private Map m_methodDescriptors = Maps.newHashMap(); + + private ITestObjectFactory m_objectFactory; + + private List m_invokedMethodListeners = Lists.newArrayList(); + + private Integer m_dataProviderThreadCount = null; + + private String m_jarPath; + /** + * The path of the testng.xml file inside the jar file + */ + private String m_xmlPathInJar = CommandLineArgs.XML_PATH_IN_JAR_DEFAULT; + + private List m_stringSuites = Lists.newArrayList(); + + private IHookable m_hookable; + private IConfigurable m_configurable; + + protected long m_end; + protected long m_start; + + private List m_executionListeners = Lists.newArrayList(); + + private boolean m_isInitialized = false; + + /** + * Default constructor. Setting also usage of default listeners/reporters. + */ + public TestNG() + { + init(true); + } + + /** + * Used by maven2 to have 0 output of any kind come out + * of testng. + * + * @param useDefaultListeners Whether or not any default reports + * should be added to tests. + */ + public TestNG(boolean useDefaultListeners) + { + init(useDefaultListeners); + } + + private void init(boolean useDefaultListeners) + { + m_instance = this; + + m_useDefaultListeners = useDefaultListeners; + m_configuration = new Configuration(); + } + + public int getStatus() + { + return m_status; + } + + private void setStatus(int status) + { + m_status |= status; + } + + /** + * Sets the output directory where the reports will be created. + * + * @param outputdir The directory. + */ + public void setOutputDirectory(final String outputdir) + { + if (isStringNotEmpty(outputdir)) { + m_outputDir = outputdir; + } + } + + /** + * If this method is passed true before run(), the default listeners + * will not be used. + *

    + *
  • org.testng.reporters.TestHTMLReporter + *
  • org.testng.reporters.JUnitXMLReporter + *
  • org.testng.reporters.XMLReporter + *
+ * + * @see org.testng.reporters.TestHTMLReporter + * @see org.testng.reporters.JUnitXMLReporter + * @see org.testng.reporters.XMLReporter + */ + public void setUseDefaultListeners(boolean useDefaultListeners) + { + m_useDefaultListeners = useDefaultListeners; + } + + /** + * Sets a jar containing a testng.xml file. + * + * @param jarPath + */ + public void setTestJar(String jarPath) + { + m_jarPath = jarPath; + } + + /** + * Sets the path to the XML file in the test jar file. + */ + public void setXmlPathInJar(String xmlPathInJar) + { + m_xmlPathInJar = xmlPathInJar; + } + + public void initializeSuitesAndJarFile() + { + // The Eclipse plug-in (RemoteTestNG) might have invoked this method already + // so don't initialize suites twice. + if (m_isInitialized) { + return; + } + + m_isInitialized = true; + if (m_suites.size() > 0) { + //to parse the suite files (), if any + for (XmlSuite s : m_suites) { + + for (String suiteFile : s.getSuiteFiles()) { + try { + Collection childSuites = getParser(suiteFile).parse(); + for (XmlSuite cSuite : childSuites) { + cSuite.setParentSuite(s); + s.getChildSuites().add(cSuite); + } + } + catch (FileNotFoundException e) { + e.printStackTrace(System.out); + } + catch (ParserConfigurationException e) { + e.printStackTrace(System.out); + } + catch (SAXException e) { + e.printStackTrace(System.out); + } + catch (IOException e) { + e.printStackTrace(System.out); + } + } + + } + return; + } + + // + // Parse the suites that were passed on the command line + // + for (String suitePath : m_stringSuites) { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("suiteXmlPath: \"" + suitePath + "\""); + } + try { + Collection allSuites = getParser(suitePath).parse(); + + for (XmlSuite s : allSuites) { + // If test names were specified, only run these test names + if (m_testNames != null) { + m_suites.add(extractTestNames(s, m_testNames)); + } else { + m_suites.add(s); + } + } + } + catch (FileNotFoundException e) { + e.printStackTrace(System.out); + } + catch (IOException e) { + e.printStackTrace(System.out); + } + catch (ParserConfigurationException e) { + e.printStackTrace(System.out); + } + catch (SAXException e) { + e.printStackTrace(System.out); + } + catch (Exception ex) { + // Probably a Yaml exception, unnest it + Throwable t = ex; + while (t.getCause() != null) { + t = t.getCause(); + } +// t.printStackTrace(); + if (t instanceof TestNGException) { + throw (TestNGException) t; + } else { + throw new TestNGException(t); + } + } + } + + // + // jar path + // + // If suites were passed on the command line, they take precedence over the suite file + // inside that jar path + if (m_jarPath != null && m_stringSuites.size() > 0) { + StringBuilder suites = new StringBuilder(); + for (String s : m_stringSuites) { + suites.append(s); + } + Utils.log( + "TestNG", 2, "Ignoring the XML file inside " + m_jarPath + " and using " + + suites + " instead" + ); + return; + } + if (isStringEmpty(m_jarPath)) { + return; + } + + // We have a jar file and no XML file was specified: try to find an XML file inside the jar + File jarFile = new File(m_jarPath); + + try { + + Utils.log("TestNG", 2, "Trying to open jar file:" + jarFile); + + JarFile jf = new JarFile(jarFile); +// System.out.println(" result: " + jf); + Enumeration entries = jf.entries(); + List classes = Lists.newArrayList(); + boolean foundTestngXml = false; + while (entries.hasMoreElements()) { + JarEntry je = entries.nextElement(); + if (je.getName().equals(m_xmlPathInJar)) { + Parser parser = getParser(jf.getInputStream(je)); + m_suites.addAll(parser.parse()); + foundTestngXml = true; + break; + } else if (je.getName().endsWith(".class")) { + int n = je.getName().length() - ".class".length(); + classes.add(je.getName().replace("/", ".").substring(0, n)); + } + } + if (!foundTestngXml) { + Utils.log( + "TestNG", 1, + "Couldn't find the " + m_xmlPathInJar + " in the jar file, running all the classes" + ); + XmlSuite xmlSuite = new XmlSuite(); + xmlSuite.setVerbose(0); + xmlSuite.setName("Jar suite"); + XmlTest xmlTest = new XmlTest(xmlSuite); + List xmlClasses = Lists.newArrayList(); + for (String cls : classes) { + XmlClass xmlClass = new XmlClass(cls); + xmlClasses.add(xmlClass); + } + xmlTest.setXmlClasses(xmlClasses); + m_suites.add(xmlSuite); + } + } + catch (ParserConfigurationException ex) { + ex.printStackTrace(); + } + catch (SAXException ex) { + ex.printStackTrace(); + } + catch (IOException ex) { + ex.printStackTrace(); + } + } + + private Parser getParser(String path) + { + Parser result = new Parser(path); + initProcessor(result); + return result; + } + + private Parser getParser(InputStream is) + { + Parser result = new Parser(is); + initProcessor(result); + return result; + } + + private void initProcessor(Parser result) + { + result.setPostProcessor(new OverrideProcessor(m_includedGroups, m_excludedGroups)); + } + + /** + * If the XmlSuite contains at least one test named as testNames, return + * an XmlSuite that's made only of these tests, otherwise, return the + * original suite. + */ + private static XmlSuite extractTestNames(XmlSuite s, List testNames) + { + List tests = Lists.newArrayList(); + for (XmlTest xt : s.getTests()) { + for (String tn : testNames) { + if (xt.getName().equals(tn)) { + tests.add(xt); + } + } + } + + if (tests.size() == 0) { + return s; + } else { + XmlSuite result = (XmlSuite) s.clone(); + result.getTests().clear(); + result.getTests().addAll(tests); + return result; + } + } + + /** + * Define the number of threads in the thread pool. + */ + public void setThreadCount(int threadCount) + { + if (threadCount < 1) { + exitWithError("Cannot use a threadCount parameter less than 1; 1 > " + threadCount); + } + + m_threadCount = threadCount; + m_useThreadCount = true; + } + + /** + * Define whether this run will be run in parallel mode. + */ + public void setParallel(String parallel) + { + m_parallelMode = parallel; + m_useParallelMode = true; + } + + public void setCommandLineSuite(XmlSuite suite) + { + m_cmdlineSuites = Lists.newArrayList(); + m_cmdlineSuites.add(suite); + m_suites.add(suite); + } + + /** + * Set the test classes to be run by this TestNG object. This method + * will create a dummy suite that will wrap these classes called + * "Command Line Test". + *

+ * If used together with threadCount, parallel, groups, excludedGroups than this one must be set first. + * + * @param classes An array of classes that contain TestNG annotations. + */ + public void setTestClasses(Class[] classes) + { + m_suites.clear(); + m_commandLineTestClasses = classes; + } + + /** + * Given a string com.example.Foo.f1, return an array where [0] is the class and [1] + * is the method. + */ + private String[] splitMethod(String m) + { + int index = m.lastIndexOf("."); + if (index < 0) { + throw new TestNGException( + "Bad format for command line method:" + m + + ", expected ." + ); + } + + return new String[]{m.substring(0, index), m.substring(index + 1).replaceAll("\\*", "\\.\\*")}; + } + + /** + * @param commandLineMethods a string with the form "com.example.Foo.f1,com.example.Bar.f2" + * + * @return a list of XmlSuite objects that represent the list of classes and methods passed + * in parameter. + */ + private List createCommandLineSuitesForMethods(List commandLineMethods) + { + // + // Create the tag + // + Set classes = Sets.newHashSet(); + for (String m : commandLineMethods) { + Class c = ClassHelper.forName(splitMethod(m)[0]); + if (c != null) { + classes.add(c); + } + } + + List result = createCommandLineSuitesForClasses(classes.toArray(new Class[0])); + + // + // Add the method tags + // + List xmlClasses = Lists.newArrayList(); + for (XmlSuite s : result) { + for (XmlTest t : s.getTests()) { + xmlClasses.addAll(t.getClasses()); + } + } + + for (XmlClass xc : xmlClasses) { + for (String m : commandLineMethods) { + String[] split = splitMethod(m); + String className = split[0]; + if (xc.getName().equals(className)) { + XmlInclude includedMethod = new XmlInclude(split[1]); + xc.getIncludedMethods().add(includedMethod); + } + } + } + + return result; + } + + private List createCommandLineSuitesForClasses(Class[] classes) + { + // + // See if any of the classes has an xmlSuite or xmlTest attribute. + // If it does, create the appropriate XmlSuite, otherwise, create + // the default one + // + XmlClass[] xmlClasses = Utils.classesToXmlClasses(classes); + Map suites = Maps.newHashMap(); + IAnnotationFinder finder = m_configuration.getAnnotationFinder(); + + for (int i = 0; i < classes.length; i++) { + Class c = classes[i]; + ITestAnnotation test = finder.findAnnotation(c, ITestAnnotation.class); + String suiteName = getDefaultSuiteName(); + String testName = getDefaultTestName(); + boolean isJUnit = false; + if (test != null) { + suiteName = defaultIfStringEmpty(test.getSuiteName(), suiteName); + testName = defaultIfStringEmpty(test.getTestName(), testName); + } else { + if (m_isMixed && JUnitTestFinder.isJUnitTest(c)) { + isJUnit = true; + testName = c.getName(); + } + } + XmlSuite xmlSuite = suites.get(suiteName); + if (xmlSuite == null) { + xmlSuite = new XmlSuite(); + xmlSuite.setName(suiteName); + suites.put(suiteName, xmlSuite); + } + + if (m_dataProviderThreadCount != null) { + xmlSuite.setDataProviderThreadCount(m_dataProviderThreadCount); + } + XmlTest xmlTest = null; + for (XmlTest xt : xmlSuite.getTests()) { + if (xt.getName().equals(testName)) { + xmlTest = xt; + break; + } + } + + if (xmlTest == null) { + xmlTest = new XmlTest(xmlSuite); + xmlTest.setName(testName); + xmlTest.setJUnit(isJUnit); + } + + xmlTest.getXmlClasses().add(xmlClasses[i]); + } + + return new ArrayList(suites.values()); + } + + public void addMethodSelector(String className, int priority) + { + m_methodDescriptors.put(className, priority); + } + + /** + * Set the suites file names to be run by this TestNG object. This method tries to load and + * parse the specified TestNG suite xml files. If a file is missing, it is ignored. + * + * @param suites A list of paths to one more XML files defining the tests. For example: + *

+ *

+   *                             TestNG tng = new TestNG();
+   *                             List suites = Lists.newArrayList();
+   *                             suites.add("c:/tests/testng1.xml");
+   *                             suites.add("c:/tests/testng2.xml");
+   *                             tng.setTestSuites(suites);
+   *                             tng.run();
+   *                             
+ */ + public void setTestSuites(List suites) + { + m_stringSuites = suites; + } + + /** + * Specifies the XmlSuite objects to run. + * + * @param suites + * + * @see org.testng.xml.XmlSuite + */ + public void setXmlSuites(List suites) + { + m_suites = suites; + } + + /** + * Define which groups will be excluded from this run. + * + * @param groups A list of group names separated by a comma. + */ + public void setExcludedGroups(String groups) + { + m_excludedGroups = Utils.split(groups, ","); + } + + + /** + * Define which groups will be included from this run. + * + * @param groups A list of group names separated by a comma. + */ + public void setGroups(String groups) + { + m_includedGroups = Utils.split(groups, ","); + } + + + private void setTestRunnerFactoryClass(Class testRunnerFactoryClass) + { + setTestRunnerFactory((ITestRunnerFactory) ClassHelper.newInstance(testRunnerFactoryClass)); + } + + + protected void setTestRunnerFactory(ITestRunnerFactory itrf) + { + m_testRunnerFactory = itrf; + } + + public void setObjectFactory(Class c) + { + m_objectFactory = (ITestObjectFactory) ClassHelper.newInstance(c); + } + + public void setObjectFactory(ITestObjectFactory factory) + { + m_objectFactory = factory; + } + + /** + * Define which listeners to user for this run. + * + * @param classes A list of classes, which must be either ISuiteListener, + * ITestListener or IReporter + */ + public void setListenerClasses(List classes) + { + for (Class cls : classes) { + addListener(ClassHelper.newInstance(cls)); + } + } + + public void addListener(Object listener) + { + if (!(listener instanceof ITestNGListener)) { + exitWithError( + "Listener " + listener + + " must be one of ITestListener, ISuiteListener, IReporter, " + + " IAnnotationTransformer, IMethodInterceptor or IInvokedMethodListener" + ); + } else { + if (listener instanceof ISuiteListener) { + addListener((ISuiteListener) listener); + } + if (listener instanceof ITestListener) { + addListener((ITestListener) listener); + } + if (listener instanceof IReporter) { + addListener((IReporter) listener); + } + if (listener instanceof IAnnotationTransformer) { + setAnnotationTransformer((IAnnotationTransformer) listener); + } + if (listener instanceof IMethodInterceptor) { + setMethodInterceptor((IMethodInterceptor) listener); + } + if (listener instanceof IInvokedMethodListener) { + addInvokedMethodListener((IInvokedMethodListener) listener); + } + if (listener instanceof IHookable) { + setHookable((IHookable) listener); + } + if (listener instanceof IConfigurable) { + setConfigurable((IConfigurable) listener); + } + if (listener instanceof IExecutionListener) { + addExecutionListener((IExecutionListener) listener); + } + if (listener instanceof IConfigurationListener) { + getConfiguration().addConfigurationListener((IConfigurationListener) listener); + } + } + } + + public void addListener(IInvokedMethodListener listener) + { + m_invokedMethodListeners.add(listener); + } + + public void addListener(ISuiteListener listener) + { + if (null != listener) { + m_suiteListeners.add(listener); + } + } + + public void addListener(ITestListener listener) + { + if (null != listener) { + m_testListeners.add(listener); + } + } + + public void addListener(IReporter listener) + { + if (null != listener) { + m_reporters.add(listener); + } + } + + public void addInvokedMethodListener(IInvokedMethodListener listener) + { + m_invokedMethodListeners.add(listener); + } + + public Set getReporters() + { + return m_reporters; + } + + public List getTestListeners() + { + return m_testListeners; + } + + public List getSuiteListeners() + { + return m_suiteListeners; + } + + /** + * If m_verbose gets set, it will override the verbose setting in testng.xml + */ + private Integer m_verbose = null; + + private final IAnnotationTransformer m_defaultAnnoProcessor = new DefaultAnnotationTransformer(); + private IAnnotationTransformer m_annotationTransformer = m_defaultAnnoProcessor; + + private Boolean m_skipFailedInvocationCounts = false; + + private IMethodInterceptor m_methodInterceptor = null; + + /** + * The list of test names to run from the given suite + */ + private List m_testNames; + + private Integer m_suiteThreadPoolSize = CommandLineArgs.SUITE_THREAD_POOL_SIZE_DEFAULT; + + private boolean m_randomizeSuites = Boolean.FALSE; + + private boolean m_preserveOrder = false; + private Boolean m_groupByInstances; + + private IConfiguration m_configuration; + + /** + * Sets the level of verbosity. This value will override the value specified + * in the test suites. + * + * @param verbose the verbosity level (0 to 10 where 10 is most detailed) + * Actually, this is a lie: you can specify -1 and this will put TestNG + * in debug mode (no longer slicing off stack traces and all). + */ + public void setVerbose(int verbose) + { + m_verbose = verbose; + } + + private void initializeCommandLineSuites() + { + if (m_commandLineTestClasses != null || m_commandLineMethods != null) { + if (null != m_commandLineMethods) { + m_cmdlineSuites = createCommandLineSuitesForMethods(m_commandLineMethods); + } else { + m_cmdlineSuites = createCommandLineSuitesForClasses(m_commandLineTestClasses); + } + + for (XmlSuite s : m_cmdlineSuites) { + for (XmlTest t : s.getTests()) { + t.setPreserveOrder(m_preserveOrder ? "true " : "false"); + } + m_suites.add(s); + if (m_groupByInstances != null) { + s.setGroupByInstances(m_groupByInstances); + } + } + } + } + + private void initializeCommandLineSuitesParams() + { + if (null == m_cmdlineSuites) { + return; + } + + for (XmlSuite s : m_cmdlineSuites) { + if (m_useThreadCount) { + s.setThreadCount(m_threadCount); + } + if (m_useParallelMode) { + s.setParallel(m_parallelMode); + } + if (m_configFailurePolicy != null) { + s.setConfigFailurePolicy(m_configFailurePolicy.toString()); + } + } + + } + + private void initializeCommandLineSuitesGroups() + { + // If groups were specified on the command line, they should override groups + // specified in the XML file + boolean hasIncludedGroups = null != m_includedGroups && m_includedGroups.length > 0; + boolean hasExcludedGroups = null != m_excludedGroups && m_excludedGroups.length > 0; + List suites = m_cmdlineSuites != null ? m_cmdlineSuites : m_suites; + if (hasIncludedGroups || hasExcludedGroups) { + for (XmlSuite s : suites) { + //set on each test, instead of just the first one of the suite + for (XmlTest t : s.getTests()) { + if (hasIncludedGroups) { + t.setIncludedGroups(Arrays.asList(m_includedGroups)); + } + if (hasExcludedGroups) { + t.setExcludedGroups(Arrays.asList(m_excludedGroups)); + } + } + } + } + } + + private void addReporter(Class r) + { + if (!m_reporters.contains(r)) { + m_reporters.add(ClassHelper.newInstance(r)); + } + } + + private void initializeDefaultListeners() + { + m_testListeners.add(new ExitCodeListener(this)); + + if (m_useDefaultListeners) { + addReporter(SuiteHTMLReporter.class); + addReporter(Main.class); + addReporter(FailedReporter.class); + addReporter(XMLReporter.class); + if (System.getProperty("oldTestngEmailableReporter") != null) { + addReporter(EmailableReporter.class); + } else if (System.getProperty("noEmailableReporter") == null) { + addReporter(EmailableReporter2.class); + } + addReporter(JUnitReportReporter.class); + if (m_verbose != null && m_verbose > 4) { + addListener(new VerboseReporter("[TestNG] ")); + } + } + } + + private void initializeConfiguration() + { + ITestObjectFactory factory = m_objectFactory; + // + // Install the listeners found in ServiceLoader (or use the class + // loader for tests, if specified). + // + addServiceLoaderListeners(); + + // + // Install the listeners found in the suites + // + for (XmlSuite s : m_suites) { + for (String listenerName : s.getListeners()) { + Class listenerClass = ClassHelper.forName(listenerName); + + // If specified listener does not exist, a TestNGException will be thrown + if (listenerClass == null) { + throw new TestNGException( + "Listener " + listenerName + + " was not found in project's classpath" + ); + } + + Object listener = ClassHelper.newInstance(listenerClass); + addListener(listener); + } + + // + // Install the method selectors + // + for (XmlMethodSelector methodSelector : s.getMethodSelectors()) { + addMethodSelector(methodSelector.getClassName(), methodSelector.getPriority()); + } + + // + // Find if we have an object factory + // + if (s.getObjectFactory() != null) { + if (factory == null) { + factory = s.getObjectFactory(); + } else { + throw new TestNGException("Found more than one object-factory tag in your suites"); + } + } + } + + m_configuration.setAnnotationFinder(new JDK15AnnotationFinder(getAnnotationTransformer())); + m_configuration.setHookable(m_hookable); + m_configuration.setConfigurable(m_configurable); + m_configuration.setObjectFactory(factory); + } + + /** + * Using reflection to remain Java 5 compliant. + */ + private void addServiceLoaderListeners() + { + try { + Class c = Class.forName("java.util.ServiceLoader"); + List parameters = Lists.newArrayList(); + parameters.add(ITestNGListener.class); + Method loadMethod; + if (m_serviceLoaderClassLoader != null) { + parameters.add(m_serviceLoaderClassLoader); + loadMethod = c.getMethod("load", Class.class, ClassLoader.class); + } else { + loadMethod = c.getMethod("load", Class.class); + } + Iterable loader = + (Iterable) loadMethod.invoke(c, parameters.toArray()); +// Object loader = c. +// ServiceLoader loader = m_serviceLoaderClassLoader != null +// ? ServiceLoader.load(ITestNGListener.class, m_serviceLoaderClassLoader) +// : ServiceLoader.load(ITestNGListener.class); + for (ITestNGListener l : loader) { + Utils.log("[TestNG]", 2, "Adding ServiceLoader listener:" + l); + addListener(l); + addServiceLoaderListener(l); + } + } + catch (ClassNotFoundException ex) { + // Ignore + } + catch (NoSuchMethodException ex) { + // Ignore + } + catch (IllegalAccessException ex) { + // Ignore + } + catch (InvocationTargetException ex) { + // Ignore + } + } + + /** + * Before suites are executed, do a sanity check to ensure all required + * conditions are met. If not, throw an exception to stop test execution + * + * @throws TestNGException if the sanity check fails + */ + private void sanityCheck() + { + checkTestNames(m_suites); + checkSuiteNames(m_suites); + } + + /** + * Ensure that two XmlTest within the same XmlSuite don't have the same name + */ + private void checkTestNames(List suites) + { + for (XmlSuite suite : suites) { + Set testNames = Sets.newHashSet(); + for (XmlTest test : suite.getTests()) { + if (testNames.contains(test.getName())) { + throw new TestNGException( + "Two tests in the same suite " + + "cannot have the same name: " + test.getName() + ); + } else { + testNames.add(test.getName()); + } + } + checkTestNames(suite.getChildSuites()); + } + } + + /** + * Ensure that two XmlSuite don't have the same name + * Otherwise will be clash in SuiteRunnerMap + * See issue #302 + */ + private void checkSuiteNames(List suites) + { + checkSuiteNamesInternal(suites, Sets.newHashSet()); + } + + private void checkSuiteNamesInternal(List suites, Set names) + { + for (XmlSuite suite : suites) { + final String name = suite.getName(); + if (names.contains(name)) { + throw new TestNGException("Two suites cannot have the same name: " + name); + } + names.add(name); + checkSuiteNamesInternal(suite.getChildSuites(), names); + } + } + + /** + * Run TestNG. + */ + public void run() + { + initializeSuitesAndJarFile(); + initializeConfiguration(); + initializeDefaultListeners(); + initializeCommandLineSuites(); + initializeCommandLineSuitesParams(); + initializeCommandLineSuitesGroups(); + + sanityCheck(); + + List suiteRunners = null; + + runExecutionListeners(true /* start */); + + m_start = System.currentTimeMillis(); + + // + // Slave mode + // + if (m_slavefileName != null) { + SuiteSlave slave = new SuiteSlave(m_slavefileName, this); + slave.waitForSuites(); + } + + // + // Regular mode + // + else if (m_masterfileName == null) { + suiteRunners = runSuitesLocally(); + } + + // + // Master mode + // + else { + SuiteDispatcher dispatcher = new SuiteDispatcher(m_masterfileName); + suiteRunners = dispatcher.dispatch( + getConfiguration(), + m_suites, getOutputDirectory(), + getTestListeners() + ); + } + + m_end = System.currentTimeMillis(); + runExecutionListeners(false /* finish */); + + if (null != suiteRunners) { + generateReports(suiteRunners); + } + + if (!m_hasTests) { + setStatus(HAS_NO_TEST); + if (TestRunner.getVerbose() > 1) { + System.err.println("[TestNG] No tests found. Nothing was run"); + usage(); + } + } + } + + private void p(String string) + { + System.out.println("[TestNG] " + string); + } + + private void runExecutionListeners(boolean start) + { + for (List listeners + : Arrays.asList(m_executionListeners, m_configuration.getExecutionListeners())) { + for (IExecutionListener l : listeners) { + if (start) { + l.onExecutionStart(); + } else { + l.onExecutionFinish(); + } + } + } + } + + public void addExecutionListener(IExecutionListener l) + { + m_executionListeners.add(l); + } + + private static void usage() + { + if (m_jCommander == null) { + m_jCommander = new JCommander(new CommandLineArgs()); + } + m_jCommander.usage(); + } + + private void generateReports(List suiteRunners) + { + for (IReporter reporter : m_reporters) { + try { + long start = System.currentTimeMillis(); + reporter.generateReport(m_suites, suiteRunners, m_outputDir); + Utils.log( + "TestNG", 2, "Time taken by " + reporter + ": " + + (System.currentTimeMillis() - start) + " ms" + ); + } + catch (Exception ex) { + System.err.println("[TestNG] Reporter " + reporter + " failed"); + ex.printStackTrace(System.err); + } + } + } + + /** + * This needs to be public for maven2, for now..At least + * until an alternative mechanism is found. + */ + public List runSuitesLocally() + { + SuiteRunnerMap suiteRunnerMap = new SuiteRunnerMap(); + if (m_suites.size() > 0) { + if (m_suites.get(0).getVerbose() >= 2) { + Version.displayBanner(); + } + + // First initialize the suite runners to ensure there are no configuration issues. + // Create a map with XmlSuite as key and corresponding SuiteRunner as value + for (XmlSuite xmlSuite : m_suites) { + createSuiteRunners(suiteRunnerMap, xmlSuite); + } + + // + // Run suites + // + if (m_suiteThreadPoolSize == 1 && !m_randomizeSuites) { + // Single threaded and not randomized: run the suites in order + for (XmlSuite xmlSuite : m_suites) { + runSuitesSequentially( + xmlSuite, suiteRunnerMap, getVerbose(xmlSuite), + getDefaultSuiteName() + ); + } + } else { + // Multithreaded: generate a dynamic graph that stores the suite hierarchy. This is then + // used to run related suites in specific order. Parent suites are run only + // once all the child suites have completed execution + DynamicGraph suiteGraph = new DynamicGraph(); + for (XmlSuite xmlSuite : m_suites) { + populateSuiteGraph(suiteGraph, suiteRunnerMap, xmlSuite); + } + + IThreadWorkerFactory factory = new SuiteWorkerFactory( + suiteRunnerMap, + 0 /* verbose hasn't been set yet */, getDefaultSuiteName() + ); + GraphThreadPoolExecutor pooledExecutor = + new GraphThreadPoolExecutor( + suiteGraph, factory, m_suiteThreadPoolSize, + m_suiteThreadPoolSize, Integer.MAX_VALUE, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue() + ); + + Utils.log("TestNG", 2, "Starting executor for all suites"); + // Run all suites in parallel + pooledExecutor.run(); + try { + pooledExecutor.awaitTermination(Long.MAX_VALUE, TimeUnit.MILLISECONDS); + pooledExecutor.shutdownNow(); + } + catch (InterruptedException handled) { + Thread.currentThread().interrupt(); + error("Error waiting for concurrent executors to finish " + handled.getMessage()); + } + } + } else { + setStatus(HAS_NO_TEST); + error("No test suite found. Nothing to run"); + usage(); + } + + // + // Generate the suites report + // + return Lists.newArrayList(suiteRunnerMap.values()); + } + + private static void error(String s) + { + LOGGER.error(s); + } + + /** + * @return the verbose level, checking in order: the verbose level on + * the suite, the verbose level on the TestNG object, or 1. + */ + private int getVerbose(XmlSuite xmlSuite) + { + int result = xmlSuite.getVerbose() != null ? xmlSuite.getVerbose() + : (m_verbose != null ? m_verbose : DEFAULT_VERBOSE); + return result; + } + + /** + * Recursively runs suites. Runs the children suites before running the parent + * suite. This is done so that the results for parent suite can reflect the + * combined results of the children suites. + * + * @param xmlSuite XML Suite to be executed + * @param suiteRunnerMap Maps {@code XmlSuite}s to respective {@code ISuite} + * @param verbose verbose level + * @param defaultSuiteName default suite name + */ + private void runSuitesSequentially( + XmlSuite xmlSuite, + SuiteRunnerMap suiteRunnerMap, int verbose, String defaultSuiteName + ) + { + for (XmlSuite childSuite : xmlSuite.getChildSuites()) { + runSuitesSequentially(childSuite, suiteRunnerMap, verbose, defaultSuiteName); + } + SuiteRunnerWorker srw = new SuiteRunnerWorker( + suiteRunnerMap.get(xmlSuite), suiteRunnerMap, + verbose, defaultSuiteName + ); + srw.run(); + } + + /** + * Populates the dynamic graph with the reverse hierarchy of suites. Edges are + * added pointing from child suite runners to parent suite runners, hence making + * parent suite runners dependent on all the child suite runners + * + * @param suiteGraph dynamic graph representing the reverse hierarchy of SuiteRunners + * @param suiteRunnerMap Map with XMLSuite as key and its respective SuiteRunner as value + * @param xmlSuite XML Suite + */ + private void populateSuiteGraph( + DynamicGraph suiteGraph /* OUT */, + SuiteRunnerMap suiteRunnerMap, XmlSuite xmlSuite + ) + { + ISuite parentSuiteRunner = suiteRunnerMap.get(xmlSuite); + if (xmlSuite.getChildSuites().isEmpty()) { + suiteGraph.addNode(parentSuiteRunner); + } else { + for (XmlSuite childSuite : xmlSuite.getChildSuites()) { + suiteGraph.addEdge(parentSuiteRunner, suiteRunnerMap.get(childSuite)); + populateSuiteGraph(suiteGraph, suiteRunnerMap, childSuite); + } + } + } + + /** + * Creates the {@code SuiteRunner}s and populates the suite runner map with + * this information + * + * @param suiteRunnerMap Map with XMLSuite as key and it's respective + * SuiteRunner as value. This is updated as part of this method call + * @param xmlSuite Xml Suite (and its children) for which {@code SuiteRunner}s are created + */ + private void createSuiteRunners(SuiteRunnerMap suiteRunnerMap /* OUT */, XmlSuite xmlSuite) + { + if (null != m_isJUnit && !m_isJUnit.equals(XmlSuite.DEFAULT_JUNIT)) { + xmlSuite.setJUnit(m_isJUnit); + } + + // If the skip flag was invoked on the command line, it + // takes precedence + if (null != m_skipFailedInvocationCounts) { + xmlSuite.setSkipFailedInvocationCounts(m_skipFailedInvocationCounts); + } + + // Override the XmlSuite verbose value with the one from TestNG + if (m_verbose != null) { + xmlSuite.setVerbose(m_verbose); + } + + if (null != m_configFailurePolicy) { + xmlSuite.setConfigFailurePolicy(m_configFailurePolicy); + } + + for (XmlTest t : xmlSuite.getTests()) { + for (Map.Entry ms : m_methodDescriptors.entrySet()) { + XmlMethodSelector xms = new XmlMethodSelector(); + xms.setName(ms.getKey()); + xms.setPriority(ms.getValue()); + t.getMethodSelectors().add(xms); + } + } + + suiteRunnerMap.put(xmlSuite, createSuiteRunner(xmlSuite)); + + for (XmlSuite childSuite : xmlSuite.getChildSuites()) { + createSuiteRunners(suiteRunnerMap, childSuite); + } + } + + /** + * Creates a suite runner and configures its initial state + * + * @param xmlSuite + * + * @return returns the newly created suite runner + */ + private SuiteRunner createSuiteRunner(XmlSuite xmlSuite) + { + SuiteRunner result = new SuiteRunner( + getConfiguration(), xmlSuite, + m_outputDir, + m_testRunnerFactory, + m_useDefaultListeners, + m_methodInterceptor, + m_invokedMethodListeners, + m_testListeners + ); + + for (ISuiteListener isl : m_suiteListeners) { + result.addListener(isl); + } + + for (IReporter r : result.getReporters()) { + addListener(r); + } + + for (IConfigurationListener cl : m_configuration.getConfigurationListeners()) { + result.addListener(cl); + } + + return result; + } + + protected IConfiguration getConfiguration() + { + return m_configuration; + } + + /** + * The TestNG entry point for command line execution. + * + * @param argv the TestNG command line parameters. + * + * @throws FileNotFoundException + */ + public static void main(String[] argv) + { + TestNG testng = privateMain(argv, null); + System.exit(testng.getStatus()); + } + + /** + * Note: this method is not part of the public API and is meant for internal usage only. + */ + public static TestNG privateMain(String[] argv, ITestListener listener) + { + TestNG result = new TestNG(); + + if (null != listener) { + result.addListener(listener); + } + + // + // Parse the arguments + // + try { + CommandLineArgs cla = new CommandLineArgs(); + m_jCommander = new JCommander(cla, argv); + validateCommandLineParameters(cla); + result.configure(cla); + } + catch (ParameterException ex) { + exitWithError(ex.getMessage()); + } + + // + // Run + // + try { + result.run(); + } + catch (TestNGException ex) { + if (TestRunner.getVerbose() > 1) { + ex.printStackTrace(System.out); + } else { + error(ex.getMessage()); + } + result.setStatus(HAS_FAILURE); + } + + return result; + } + + /** + * Configure the TestNG instance based on the command line parameters. + */ + protected void configure(CommandLineArgs cla) + { + if (cla.verbose != null) { + setVerbose(cla.verbose); + } + setOutputDirectory(cla.outputDirectory); + + String testClasses = cla.testClass; + if (null != testClasses) { + String[] strClasses = testClasses.split(","); + List classes = Lists.newArrayList(); + for (String c : strClasses) { + classes.add(ClassHelper.fileToClass(c)); + } + + setTestClasses(classes.toArray(new Class[classes.size()])); + } + + setOutputDirectory(cla.outputDirectory); + + if (cla.testNames != null) { + setTestNames(Arrays.asList(cla.testNames.split(","))); + } + +// List testNgXml = (List) cmdLineArgs.get(CommandLineArgs.SUITE_DEF); +// if (null != testNgXml) { +// setTestSuites(testNgXml); +// } + + // Note: can't use a Boolean field here because we are allowing a boolean + // parameter with an arity of 1 ("-usedefaultlisteners false") + if (cla.useDefaultListeners != null) { + setUseDefaultListeners("true".equalsIgnoreCase(cla.useDefaultListeners)); + } + + setGroups(cla.groups); + setExcludedGroups(cla.excludedGroups); + setTestJar(cla.testJar); + setXmlPathInJar(cla.xmlPathInJar); + setJUnit(cla.junit); + setMixed(cla.mixed); + setMaster(cla.master); + setSlave(cla.slave); + setSkipFailedInvocationCounts(cla.skipFailedInvocationCounts); + if (cla.parallelMode != null) { + setParallel(cla.parallelMode); + } + if (cla.configFailurePolicy != null) { + setConfigFailurePolicy(cla.configFailurePolicy); + } + if (cla.threadCount != null) { + setThreadCount(cla.threadCount); + } + if (cla.dataProviderThreadCount != null) { + setDataProviderThreadCount(cla.dataProviderThreadCount); + } + if (cla.suiteName != null) { + setDefaultSuiteName(cla.suiteName); + } + if (cla.testName != null) { + setDefaultTestName(cla.testName); + } + if (cla.listener != null) { + String sep = ";"; + if (cla.listener.indexOf(",") >= 0) { + sep = ","; + } + String[] strs = Utils.split(cla.listener, sep); + List classes = Lists.newArrayList(); + + for (String cls : strs) { + classes.add(ClassHelper.fileToClass(cls)); + } + + setListenerClasses(classes); + } + + if (null != cla.methodSelectors) { + String[] strs = Utils.split(cla.methodSelectors, ","); + for (String cls : strs) { + String[] sel = Utils.split(cls, ":"); + try { + if (sel.length == 2) { + addMethodSelector(sel[0], Integer.valueOf(sel[1])); + } else { + error("Method selector value was not in the format org.example.Selector:4"); + } + } + catch (NumberFormatException nfe) { + error("Method selector value was not in the format org.example.Selector:4"); + } + } + } + + if (cla.objectFactory != null) { + setObjectFactory(ClassHelper.fileToClass(cla.objectFactory)); + } + if (cla.testRunnerFactory != null) { + setTestRunnerFactoryClass( + ClassHelper.fileToClass(cla.testRunnerFactory) + ); + } + //######### PATCH Begins + if (System.getProperty("testrunfactory") != null) { + setTestRunnerFactoryClass( + ClassHelper.fileToClass(System.getProperty("testrunfactory")) + ); + } + + //######### PATCH ends + + if (cla.reporter != null) { + ReporterConfig reporterConfig = ReporterConfig.deserialize(cla.reporter); + addReporter(reporterConfig); + } + + if (cla.commandLineMethods.size() > 0) { + m_commandLineMethods = cla.commandLineMethods; + } + + if (cla.suiteFiles != null) { + setTestSuites(cla.suiteFiles); + } + + setSuiteThreadPoolSize(cla.suiteThreadPoolSize); + setRandomizeSuites(cla.randomizeSuites); + } + + public void setSuiteThreadPoolSize(Integer suiteThreadPoolSize) + { + m_suiteThreadPoolSize = suiteThreadPoolSize; + } + + public Integer getSuiteThreadPoolSize() + { + return m_suiteThreadPoolSize; + } + + public void setRandomizeSuites(boolean randomizeSuites) + { + m_randomizeSuites = randomizeSuites; + } + + /** + * This method is invoked by Maven's Surefire, only remove it once + * Surefire has been modified to no longer call it. + */ + public void setSourcePath(String path) + { + // nop + } + + /** + * This method is invoked by Maven's Surefire to configure the runner, + * do not remove unless you know for sure that Surefire has been updated + * to use the new configure(CommandLineArgs) method. + * + * @deprecated use new configure(CommandLineArgs) method + */ + @SuppressWarnings({"unchecked"}) + @Deprecated + public void configure(Map cmdLineArgs) + { + CommandLineArgs result = new CommandLineArgs(); + + Integer verbose = (Integer) cmdLineArgs.get(CommandLineArgs.LOG); + if (null != verbose) { + result.verbose = verbose; + } + result.outputDirectory = (String) cmdLineArgs.get(CommandLineArgs.OUTPUT_DIRECTORY); + + String testClasses = (String) cmdLineArgs.get(CommandLineArgs.TEST_CLASS); + if (null != testClasses) { + result.testClass = testClasses; + } + + String testNames = (String) cmdLineArgs.get(CommandLineArgs.TEST_NAMES); + if (testNames != null) { + result.testNames = testNames; + } + + String useDefaultListeners = (String) cmdLineArgs.get(CommandLineArgs.USE_DEFAULT_LISTENERS); + if (null != useDefaultListeners) { + result.useDefaultListeners = useDefaultListeners; + } + + result.groups = (String) cmdLineArgs.get(CommandLineArgs.GROUPS); + result.excludedGroups = (String) cmdLineArgs.get(CommandLineArgs.EXCLUDED_GROUPS); + result.testJar = (String) cmdLineArgs.get(CommandLineArgs.TEST_JAR); + result.xmlPathInJar = (String) cmdLineArgs.get(CommandLineArgs.XML_PATH_IN_JAR); + result.junit = (Boolean) cmdLineArgs.get(CommandLineArgs.JUNIT); + result.mixed = (Boolean) cmdLineArgs.get(CommandLineArgs.MIXED); + result.master = (String) cmdLineArgs.get(CommandLineArgs.MASTER); + result.slave = (String) cmdLineArgs.get(CommandLineArgs.SLAVE); + result.skipFailedInvocationCounts = (Boolean) cmdLineArgs.get( + CommandLineArgs.SKIP_FAILED_INVOCATION_COUNTS + ); + String parallelMode = (String) cmdLineArgs.get(CommandLineArgs.PARALLEL); + if (parallelMode != null) { + result.parallelMode = parallelMode; + } + + String threadCount = (String) cmdLineArgs.get(CommandLineArgs.THREAD_COUNT); + if (threadCount != null) { + result.threadCount = Integer.parseInt(threadCount); + } + + // Not supported by Surefire yet + Integer dptc = (Integer) cmdLineArgs.get(CommandLineArgs.DATA_PROVIDER_THREAD_COUNT); + if (dptc != null) { + result.dataProviderThreadCount = dptc; + } + String defaultSuiteName = (String) cmdLineArgs.get(CommandLineArgs.SUITE_NAME); + if (defaultSuiteName != null) { + result.suiteName = defaultSuiteName; + } + + String defaultTestName = (String) cmdLineArgs.get(CommandLineArgs.TEST_NAME); + if (defaultTestName != null) { + result.testName = defaultTestName; + } + + Object listeners = cmdLineArgs.get(CommandLineArgs.LISTENER); + if (listeners instanceof List) { + result.listener = Utils.join((List) listeners, ","); + } else { + result.listener = (String) listeners; + } + + String ms = (String) cmdLineArgs.get(CommandLineArgs.METHOD_SELECTORS); + if (null != ms) { + result.methodSelectors = ms; + } + + String objectFactory = (String) cmdLineArgs.get(CommandLineArgs.OBJECT_FACTORY); + if (null != objectFactory) { + result.objectFactory = objectFactory; + } + + String runnerFactory = (String) cmdLineArgs.get(CommandLineArgs.TEST_RUNNER_FACTORY); + if (null != runnerFactory) { + result.testRunnerFactory = runnerFactory; + } + + String reporterConfigs = (String) cmdLineArgs.get(CommandLineArgs.REPORTER); + if (reporterConfigs != null) { + result.reporter = reporterConfigs; + } + + String failurePolicy = (String) cmdLineArgs.get(CommandLineArgs.CONFIG_FAILURE_POLICY); + if (failurePolicy != null) { + result.configFailurePolicy = failurePolicy; + } + + configure(result); + } + + /** + * Only run the specified tests from the suite. + */ + public void setTestNames(List testNames) + { + m_testNames = testNames; + } + + public void setSkipFailedInvocationCounts(Boolean skip) + { + m_skipFailedInvocationCounts = skip; + } + + private void addReporter(ReporterConfig reporterConfig) + { + Object instance = reporterConfig.newReporterInstance(); + if (instance != null) { + addListener(instance); + } else { + LOGGER.warn("Could not find reporte class : " + reporterConfig.getClassName()); + } + } + + /** + * Specify if this run should be in Master-Slave mode as Master + * + * @param fileName remote.properties path + */ + public void setMaster(String fileName) + { + m_masterfileName = fileName; + } + + /** + * Specify if this run should be in Master-Slave mode as slave + * + * @param fileName remote.properties path + */ + public void setSlave(String fileName) + { + m_slavefileName = fileName; + } + + /** + * Specify if this run should be made in JUnit mode + * + * @param isJUnit + */ + public void setJUnit(Boolean isJUnit) + { + m_isJUnit = isJUnit; + } + + /** + * Specify if this run should be made in mixed mode + */ + public void setMixed(Boolean isMixed) + { + if (isMixed == null) { + return; + } + m_isMixed = isMixed; + } + + /** + * @deprecated The TestNG version is now established at load time. This + * method is not required anymore and is now a no-op. + */ + @Deprecated + public static void setTestNGVersion() + { + LOGGER.info("setTestNGVersion has been deprecated."); + } + + /** + * Returns true if this is the JDK 1.4 JAR version of TestNG, false otherwise. + * + * @return true if this is the JDK 1.4 JAR version of TestNG, false otherwise. + */ + @Deprecated + public static boolean isJdk14() + { + return false; + } + + /** + * Double check that the command line parameters are valid. + */ + protected static void validateCommandLineParameters(CommandLineArgs args) + { + String testClasses = args.testClass; + List testNgXml = args.suiteFiles; + String testJar = args.testJar; + String slave = args.slave; + List methods = args.commandLineMethods; + + if (testClasses == null && slave == null && testJar == null + && (testNgXml == null || testNgXml.isEmpty()) + && (methods == null || methods.isEmpty())) { + throw new ParameterException( + "You need to specify at least one testng.xml, one class" + + " or one method" + ); + } + + String groups = args.groups; + String excludedGroups = args.excludedGroups; + + if (testJar == null && + (null != groups || null != excludedGroups) && testClasses == null + && (testNgXml == null || testNgXml.isEmpty())) { + throw new ParameterException("Groups option should be used with testclass option"); + } + + if (args.slave != null && args.master != null) { + throw new ParameterException( + CommandLineArgs.SLAVE + " can't be combined with " + + CommandLineArgs.MASTER + ); + } + + Boolean junit = args.junit; + Boolean mixed = args.mixed; + if (junit && mixed) { + throw new ParameterException( + CommandLineArgs.MIXED + " can't be combined with " + + CommandLineArgs.JUNIT + ); + } + } + + /** + * @return true if at least one test failed. + */ + public boolean hasFailure() + { + return (getStatus() & HAS_FAILURE) == HAS_FAILURE; + } + + /** + * @return true if at least one test failed within success percentage. + */ + public boolean hasFailureWithinSuccessPercentage() + { + return (getStatus() & HAS_FSP) == HAS_FSP; + } + + /** + * @return true if at least one test was skipped. + */ + public boolean hasSkip() + { + return (getStatus() & HAS_SKIPPED) == HAS_SKIPPED; + } + + static void exitWithError(String msg) + { + System.err.println(msg); + usage(); + System.exit(1); + } + + public String getOutputDirectory() + { + return m_outputDir; + } + + public IAnnotationTransformer getAnnotationTransformer() + { + return m_annotationTransformer; + } + + public void setAnnotationTransformer(IAnnotationTransformer t) + { + // compare by reference! + if (m_annotationTransformer != m_defaultAnnoProcessor && m_annotationTransformer != t) { + LOGGER.warn("AnnotationTransformer already set"); + } + m_annotationTransformer = t; + } + + /** + * @return the defaultSuiteName + */ + public String getDefaultSuiteName() + { + return m_defaultSuiteName; + } + + /** + * @param defaultSuiteName the defaultSuiteName to set + */ + public void setDefaultSuiteName(String defaultSuiteName) + { + m_defaultSuiteName = defaultSuiteName; + } + + /** + * @return the defaultTestName + */ + public String getDefaultTestName() + { + return m_defaultTestName; + } + + /** + * @param defaultTestName the defaultTestName to set + */ + public void setDefaultTestName(String defaultTestName) + { + m_defaultTestName = defaultTestName; + } + + /** + * Sets the policy for whether or not to ever invoke a configuration method again after + * it has failed once. Possible values are defined in {@link XmlSuite}. The default + * value is {@link XmlSuite#SKIP}. + * + * @param failurePolicy the configuration failure policy + */ + public void setConfigFailurePolicy(String failurePolicy) + { + m_configFailurePolicy = failurePolicy; + } + + /** + * Returns the configuration failure policy. + * + * @return config failure policy + */ + public String getConfigFailurePolicy() + { + return m_configFailurePolicy; + } + + // DEPRECATED: to be removed after a major version change + + /** + * @deprecated since 5.1 + */ + @Deprecated + public static TestNG getDefault() + { + return m_instance; + } + + /** + * @deprecated since 5.1 + */ + @Deprecated + public void setHasFailure(boolean hasFailure) + { + m_status |= HAS_FAILURE; + } + + /** + * @deprecated since 5.1 + */ + @Deprecated + public void setHasFailureWithinSuccessPercentage(boolean hasFailureWithinSuccessPercentage) + { + m_status |= HAS_FSP; + } + + /** + * @deprecated since 5.1 + */ + @Deprecated + public void setHasSkip(boolean hasSkip) + { + m_status |= HAS_SKIPPED; + } + + public static class ExitCodeListener implements IResultListener2 + { + private TestNG m_mainRunner; + + public ExitCodeListener() + { + m_mainRunner = TestNG.m_instance; + } + + public ExitCodeListener(TestNG runner) + { + m_mainRunner = runner; + } + + @Override + public void beforeConfiguration(ITestResult tr) + { + } + + @Override + public void onTestFailure(ITestResult result) + { + setHasRunTests(); + m_mainRunner.setStatus(HAS_FAILURE); + } + + @Override + public void onTestSkipped(ITestResult result) + { + setHasRunTests(); + m_mainRunner.setStatus(HAS_SKIPPED); + } + + @Override + public void onTestFailedButWithinSuccessPercentage(ITestResult result) + { + setHasRunTests(); + m_mainRunner.setStatus(HAS_FSP); + } + + @Override + public void onTestSuccess(ITestResult result) + { + setHasRunTests(); + } + + @Override + public void onStart(ITestContext context) + { + setHasRunTests(); + } + + @Override + public void onFinish(ITestContext context) + { + } + + @Override + public void onTestStart(ITestResult result) + { + setHasRunTests(); + } + + private void setHasRunTests() + { + m_mainRunner.m_hasTests = true; + } + + /** + * @see org.testng.IConfigurationListener#onConfigurationFailure(org.testng.ITestResult) + */ + @Override + public void onConfigurationFailure(ITestResult itr) + { + m_mainRunner.setStatus(HAS_FAILURE); + } + + /** + * @see org.testng.IConfigurationListener#onConfigurationSkip(org.testng.ITestResult) + */ + @Override + public void onConfigurationSkip(ITestResult itr) + { + m_mainRunner.setStatus(HAS_SKIPPED); + } + + /** + * @see org.testng.IConfigurationListener#onConfigurationSuccess(org.testng.ITestResult) + */ + @Override + public void onConfigurationSuccess(ITestResult itr) + { + } + } + + private void setConfigurable(IConfigurable c) + { + // compare by reference! + if (m_configurable != null && m_configurable != c) { + LOGGER.warn("Configurable already set"); + } + m_configurable = c; + } + + private void setHookable(IHookable h) + { + // compare by reference! + if (m_hookable != null && m_hookable != h) { + LOGGER.warn("Hookable already set"); + } + m_hookable = h; + } + + public void setMethodInterceptor(IMethodInterceptor i) + { + // compare by reference! + if (m_methodInterceptor != null && m_methodInterceptor != i) { + LOGGER.warn("MethodInterceptor already set"); + } + m_methodInterceptor = i; + } + + public void setDataProviderThreadCount(int count) + { + m_dataProviderThreadCount = count; + } + + /** + * Add a class loader to the searchable loaders. + */ + public void addClassLoader(final ClassLoader loader) + { + if (loader != null) { + ClassHelper.addClassLoader(loader); + } + } + + public void setPreserveOrder(boolean b) + { + m_preserveOrder = b; + } + + protected long getStart() + { + return m_start; + } + + protected long getEnd() + { + return m_end; + } + + public void setGroupByInstances(boolean b) + { + m_groupByInstances = b; + } + + ///// + // ServiceLoader testing + // + + private URLClassLoader m_serviceLoaderClassLoader; + private List m_serviceLoaderListeners = Lists.newArrayList(); + + /* + * Used to test ServiceClassLoader + */ + public void setServiceLoaderClassLoader(URLClassLoader ucl) + { + m_serviceLoaderClassLoader = ucl; + } + + /* + * Used to test ServiceClassLoader + */ + private void addServiceLoaderListener(ITestNGListener l) + { + m_serviceLoaderListeners.add(l); + } + + /* + * Used to test ServiceClassLoader + */ + public List getServiceLoaderListeners() + { + return m_serviceLoaderListeners; + } + + // + // ServiceLoader testing + ///// +} diff --git a/integration-tests/src/main/java/org/testng/remote/RemoteTestNG.java b/integration-tests/src/main/java/org/testng/remote/RemoteTestNG.java new file mode 100644 index 00000000000..8c7fc8d6db2 --- /dev/null +++ b/integration-tests/src/main/java/org/testng/remote/RemoteTestNG.java @@ -0,0 +1,336 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package org.testng.remote; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.ParameterException; +import org.testng.CommandLineArgs; +import org.testng.IInvokedMethodListener; +import org.testng.ISuite; +import org.testng.ISuiteListener; +import org.testng.ITestRunnerFactory; +import org.testng.TestNG; +import org.testng.TestNGException; +import org.testng.TestRunner; +import org.testng.collections.Lists; +import org.testng.internal.ClassHelper; +import org.testng.remote.strprotocol.GenericMessage; +import org.testng.remote.strprotocol.IMessageSender; +import org.testng.remote.strprotocol.MessageHelper; +import org.testng.remote.strprotocol.MessageHub; +import org.testng.remote.strprotocol.RemoteTestListener; +import org.testng.remote.strprotocol.SerializedMessageSender; +import org.testng.remote.strprotocol.StringMessageSender; +import org.testng.remote.strprotocol.SuiteMessage; +import org.testng.reporters.JUnitXMLReporter; +import org.testng.reporters.TestHTMLReporter; +import org.testng.xml.XmlSuite; +import org.testng.xml.XmlTest; + +import java.util.Arrays; +import java.util.List; + +import static org.testng.internal.Utils.defaultIfStringEmpty; + +/** + * Class copied from TestNG library ver 6.8.7 to apply a workaround for http://jira.codehaus.org/browse/SUREFIRE-622 + * To Locate the PATCHED AREA search for keyword "PATCH" in this class file + *

+ * Extension of TestNG registering a remote TestListener. + * + * @author Cedric Beust + */ +public class RemoteTestNG extends TestNG +{ + // The following constants are referenced by the Eclipse plug-in, make sure you + // modify the plug-in as well if you change any of them. + public static final String DEBUG_PORT = "12345"; + public static final String DEBUG_SUITE_FILE = "testng-customsuite.xml"; + public static final String DEBUG_SUITE_DIRECTORY = System.getProperty("java.io.tmpdir"); + public static final String PROPERTY_DEBUG = "testng.eclipse.debug"; + public static final String PROPERTY_VERBOSE = "testng.eclipse.verbose"; + private static final String LOCALHOST = "localhost"; + // End of Eclipse constants. + /** + * Port used for the serialized protocol + */ + private static Integer m_serPort = null; + private static boolean m_debug; + private static boolean m_dontExit; + private static boolean m_ack; + private ITestRunnerFactory m_customTestRunnerFactory; + private String m_host; + /** + * Port used for the string protocol + */ + private Integer m_port = null; + + public static void main(String[] args) throws ParameterException + { + CommandLineArgs cla = new CommandLineArgs(); + RemoteArgs ra = new RemoteArgs(); + new JCommander(Arrays.asList(cla, ra), args); + m_dontExit = ra.dontExit; + if (cla.port != null && ra.serPort != null) { + throw new TestNGException( + "Can only specify one of " + CommandLineArgs.PORT + + " and " + RemoteArgs.PORT + ); + } + m_debug = cla.debug; + m_ack = ra.ack; + if (m_debug) { +// while (true) { + initAndRun(args, cla, ra); +// } + } else { + initAndRun(args, cla, ra); + } + } + + private static void initAndRun(String[] args, CommandLineArgs cla, RemoteArgs ra) + { + RemoteTestNG remoteTestNg = new RemoteTestNG(); + if (m_debug) { + // In debug mode, override the port and the XML file to a fixed location + cla.port = Integer.parseInt(DEBUG_PORT); + ra.serPort = cla.port; + cla.suiteFiles = Arrays.asList( + new String[]{ + DEBUG_SUITE_DIRECTORY + DEBUG_SUITE_FILE + } + ); + } + remoteTestNg.configure(cla); + remoteTestNg.setHost(cla.host); + m_serPort = ra.serPort; + remoteTestNg.m_port = cla.port; + if (isVerbose()) { + StringBuilder sb = new StringBuilder("Invoked with "); + for (String s : args) { + sb.append(s).append(" "); + } + p(sb.toString()); +// remoteTestNg.setVerbose(1); +// } else { +// remoteTestNg.setVerbose(0); + } + validateCommandLineParameters(cla); + remoteTestNg.run(); +// if (m_debug) { +// // Run in a loop if in debug mode so it is possible to run several launches +// // without having to relauch RemoteTestNG. +// while (true) { +// remoteTestNg.run(); +// remoteTestNg.configure(cla); +// } +// } else { +// remoteTestNg.run(); +// } + } + + private static void p(String s) + { + if (isVerbose()) { + System.out.println("[RemoteTestNG] " + s); + } + } + + public static boolean isVerbose() + { + boolean result = System.getProperty(PROPERTY_VERBOSE) != null || isDebug(); + return result; + } + + public static boolean isDebug() + { + return m_debug || System.getProperty(PROPERTY_DEBUG) != null; + } + + private void calculateAllSuites(List suites, List outSuites) + { + for (XmlSuite s : suites) { + outSuites.add(s); +// calculateAllSuites(s.getChildSuites(), outSuites); + } + } + + @Override + public void run() + { + IMessageSender sender = m_serPort != null + ? new SerializedMessageSender(m_host, m_serPort, m_ack) + : new StringMessageSender(m_host, m_port); + final MessageHub msh = new MessageHub(sender); + msh.setDebug(isDebug()); + try { + msh.connect(); + // We couldn't do this until now in debug mode since the .xml file didn't exist yet. + // Now that we have connected with the Eclipse client, we know that it created the .xml + // file so we can proceed with the initialization + initializeSuitesAndJarFile(); + + List suites = Lists.newArrayList(); + calculateAllSuites(m_suites, suites); +// System.out.println("Suites: " + m_suites.get(0).getChildSuites().size() +// + " and:" + suites.get(0).getChildSuites().size()); + if (suites.size() > 0) { + + int testCount = 0; + + for (int i = 0; i < suites.size(); i++) { + testCount += (suites.get(i)).getTests().size(); + } + + GenericMessage gm = new GenericMessage(MessageHelper.GENERIC_SUITE_COUNT); + gm.setSuiteCount(suites.size()); + gm.setTestCount(testCount); + msh.sendMessage(gm); + + addListener(new RemoteSuiteListener(msh)); + setTestRunnerFactory(new DelegatingTestRunnerFactory(buildTestRunnerFactory(), msh)); + +// System.out.println("RemoteTestNG starting"); + super.run(); + } else { + System.err.println("No test suite found. Nothing to run"); + } + } + catch (Throwable cause) { + cause.printStackTrace(System.err); + } + finally { +// System.out.println("RemoteTestNG finishing: " + (getEnd() - getStart()) + " ms"); + msh.shutDown(); + if (!m_debug && !m_dontExit) { + System.exit(0); + } + } + } + + /** + * Override by the plugin if you need to configure differently the TestRunner + * (usually this is needed if different listeners/reporters are needed). + * Note: you don't need to worry about the wiring listener, because it is added + * automatically. + */ + protected ITestRunnerFactory buildTestRunnerFactory() + { + //################### PATCH STARTS + if (System.getProperty("testrunfactory") != null) { + m_customTestRunnerFactory = (ITestRunnerFactory) ClassHelper.newInstance( + ClassHelper.fileToClass( + System.getProperty( + "testrunfactory" + ) + ) + ); + //################## PATCH ENDS + } else if (null == m_customTestRunnerFactory) { + m_customTestRunnerFactory = new ITestRunnerFactory() + { + @Override + public TestRunner newTestRunner( + ISuite suite, XmlTest xmlTest, + List listeners + ) + { + TestRunner runner = + new TestRunner( + getConfiguration(), suite, xmlTest, + false /*skipFailedInvocationCounts */, + listeners + ); + if (m_useDefaultListeners) { + runner.addListener(new TestHTMLReporter()); + runner.addListener(new JUnitXMLReporter()); + } + + return runner; + } + }; + } + + return m_customTestRunnerFactory; + } + + private String getHost() + { + return m_host; + } + + public void setHost(String host) + { + m_host = defaultIfStringEmpty(host, LOCALHOST); + } + + private int getPort() + { + return m_port; + } + + /** + * A ISuiteListener wiring the results using the internal string-based protocol. + */ + private static class RemoteSuiteListener implements ISuiteListener + { + private final MessageHub m_messageSender; + + RemoteSuiteListener(MessageHub smsh) + { + m_messageSender = smsh; + } + + @Override + public void onFinish(ISuite suite) + { + m_messageSender.sendMessage(new SuiteMessage(suite, false /*start*/)); + } + + @Override + public void onStart(ISuite suite) + { + m_messageSender.sendMessage(new SuiteMessage(suite, true /*start*/)); + } + } + + private static class DelegatingTestRunnerFactory implements ITestRunnerFactory + { + private final ITestRunnerFactory m_delegateFactory; + private final MessageHub m_messageSender; + + DelegatingTestRunnerFactory(ITestRunnerFactory trf, MessageHub smsh) + { + m_delegateFactory = trf; + m_messageSender = smsh; + } + + @Override + public TestRunner newTestRunner( + ISuite suite, XmlTest test, + List listeners + ) + { + TestRunner tr = m_delegateFactory.newTestRunner(suite, test, listeners); + tr.addListener(new RemoteTestListener(suite, test, m_messageSender)); + return tr; + } + } +} diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/AbstractIndexerTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/AbstractIndexerTest.java new file mode 100644 index 00000000000..1e333b8b8d7 --- /dev/null +++ b/integration-tests/src/test/java/io/druid/tests/indexer/AbstractIndexerTest.java @@ -0,0 +1,85 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.tests.indexer; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import io.druid.testing.clients.CoordinatorResourceTestClient; +import io.druid.testing.clients.OverlordResourceTestClient; +import io.druid.testing.utils.FromFileTestQueryHelper; +import io.druid.testing.utils.RetryUtil; +import org.apache.commons.io.IOUtils; +import org.joda.time.Interval; + +import java.io.IOException; +import java.io.InputStream; +import java.io.StringWriter; +import java.util.concurrent.Callable; + +public abstract class AbstractIndexerTest +{ + + @Inject + protected CoordinatorResourceTestClient coordinator; + @Inject + protected OverlordResourceTestClient indexer; + @Inject + protected ObjectMapper jsonMapper; + + @Inject + protected FromFileTestQueryHelper queryHelper; + + protected void unloadAndKillData(final String dataSource) throws Exception + { + Interval interval = new Interval("2013-01-01T00:00:00.000Z/2013-12-01T00:00:00.000Z"); + coordinator.unloadSegmentsForDataSource(dataSource, interval); + RetryUtil.retryUntilFalse( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return coordinator.areSegmentsLoaded(dataSource); + } + }, "Segment Unloading" + ); + coordinator.deleteSegmentsDataSource(dataSource, interval); + RetryUtil.retryUntilTrue( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return (indexer.getPendingTasks().size() + indexer.getRunningTasks().size() + indexer.getWaitingTasks() + .size()) == 0; + } + }, "Waiting for Tasks Completion" + ); + } + + protected String getTaskAsString(String file) throws IOException + { + InputStream inputStream = ITRealtimeIndexTaskTest.class.getResourceAsStream(file); + StringWriter writer = new StringWriter(); + IOUtils.copy(inputStream, writer, "UTF-8"); + return writer.toString(); + } + +} diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITIndexerTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITIndexerTest.java new file mode 100644 index 00000000000..89c68d7ef61 --- /dev/null +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITIndexerTest.java @@ -0,0 +1,81 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.tests.indexer; + +import com.google.api.client.repackaged.com.google.common.base.Throwables; +import com.google.inject.Inject; +import com.metamx.common.logger.Logger; +import io.druid.testing.IntegrationTestingConfig; +import io.druid.testing.guice.DruidTestModuleFactory; +import io.druid.testing.utils.RetryUtil; +import org.testng.annotations.Guice; +import org.testng.annotations.Test; + +import java.util.concurrent.Callable; + +@Guice(moduleFactory = DruidTestModuleFactory.class) +public class ITIndexerTest extends AbstractIndexerTest +{ + private static final Logger LOG = new Logger(ITIndexerTest.class); + private static String INDEX_TASK = "/indexer/wikipedia_index_task.json"; + private static String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; + private static String INDEX_DATASOURCE = "wikipedia_index_test"; + + + @Inject + private IntegrationTestingConfig config; + + @Test + public void testIndexData() throws Exception + { + loadData(); + try { + queryHelper.testQueriesFromFile(INDEX_QUERIES_RESOURCE, 2); + } + catch (Exception e) { + e.printStackTrace(); + Throwables.propagate(e); + } + finally { + unloadAndKillData(INDEX_DATASOURCE); + } + + } + + private void loadData() throws Exception + { + final String taskID = indexer.submitTask(getTaskAsString(INDEX_TASK)); + LOG.info("TaskID for loading index task %s", taskID); + indexer.waitUntilTaskCompletes(taskID); + + RetryUtil.retryUntilTrue( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return coordinator.areSegmentsLoaded(INDEX_DATASOURCE); + } + }, "Segment Load" + ); + } + + +} diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java new file mode 100644 index 00000000000..7066a75beae --- /dev/null +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITRealtimeIndexTaskTest.java @@ -0,0 +1,142 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.tests.indexer; + +import com.google.api.client.repackaged.com.google.common.base.Throwables; +import com.google.inject.Inject; +import com.metamx.common.logger.Logger; +import com.metamx.http.client.HttpClient; +import io.druid.curator.discovery.ServerDiscoveryFactory; +import io.druid.curator.discovery.ServerDiscoverySelector; +import io.druid.guice.annotations.Global; +import io.druid.testing.IntegrationTestingConfig; +import io.druid.testing.clients.EventReceiverFirehoseTestClient; +import io.druid.testing.guice.DruidTestModuleFactory; +import io.druid.testing.utils.RetryUtil; +import io.druid.testing.utils.ServerDiscoveryUtil; +import org.joda.time.DateTime; +import org.testng.annotations.Guice; +import org.testng.annotations.Test; + +import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; + +/** + * Steps + * 1) Submit a RealtimeIndexTask + * 2) Load Data using EventReceiverFirehose + * 3) Runs queries and verifies that the ingested data is available for queries + * 4) Waits for handover of the segment to historical node + * 5) Queries data from historical node and verifies handover + * 6) Removes and Delete the created Data Segment + */ +@Guice(moduleFactory = DruidTestModuleFactory.class) +public class ITRealtimeIndexTaskTest extends AbstractIndexerTest +{ + private static final Logger LOG = new Logger(ITRealtimeIndexTaskTest.class); + private static final String REALTIME_TASK_RESOURCE = "/indexer/wikipedia_realtime_index_task.json"; + private static final String EVENT_RECEIVER_SERVICE_NAME = "eventReceiverServiceName"; + private static final String EVENT_DATA_FILE = "/indexer/wikipedia_index_data.json"; + private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; + private static final String INDEX_DATASOURCE = "wikipedia_index_test"; + @Inject + ServerDiscoveryFactory factory; + @Inject + @Global + HttpClient httpClient; + + @Inject + IntegrationTestingConfig config; + + @Test + public void testRealtimeIndexTask() throws Exception + { + try { + // the task will run for 3 minutes and then shutdown itself + String task = setShutOffTime( + getTaskAsString(REALTIME_TASK_RESOURCE), + new DateTime(System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(2)) + ); + String taskID = indexer.submitTask(task); + postEvents(); + + // sleep for a while to let the events ingested + TimeUnit.SECONDS.sleep(5); + + // should hit the queries on realtime task + this.queryHelper.testQueriesFromFile(INDEX_QUERIES_RESOURCE, 2); + // wait for the task to complete + indexer.waitUntilTaskCompletes(taskID); + + // task should complete only after the segments are loaded by historical node + RetryUtil.retryUntil( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return coordinator.areSegmentsLoaded(INDEX_DATASOURCE); + } + }, + true, + 60000, + 10, + "Real-time generated segments loaded" + ); + + // run queries on historical nodes + this.queryHelper.testQueriesFromFile(INDEX_QUERIES_RESOURCE, 2); + } + catch (Exception e) { + e.printStackTrace(); + Throwables.propagate(e); + } + finally { + unloadAndKillData(INDEX_DATASOURCE); + } + } + + private String setShutOffTime(String taskAsString, DateTime time) + { + return taskAsString.replace("#SHUTOFFTIME", time.toString()); + } + + public void postEvents() throws Exception + { + final ServerDiscoverySelector eventReceiverSelector = factory.createSelector(EVENT_RECEIVER_SERVICE_NAME); + eventReceiverSelector.start(); + try { + ServerDiscoveryUtil.waitUntilInstanceReady(eventReceiverSelector, "Event Receiver"); + // Access the docker VM mapped host and port instead of service announced in zookeeper + String host = config.getMiddleManagerHost() + ":" + eventReceiverSelector.pick().getPort(); + LOG.info("Event Receiver Found at host %s", host); + EventReceiverFirehoseTestClient client = new EventReceiverFirehoseTestClient( + host, + EVENT_RECEIVER_SERVICE_NAME, + jsonMapper, + httpClient + ); + client.postEventsFromFile(EVENT_DATA_FILE); + } + finally { + eventReceiverSelector.stop(); + } + } +} diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java new file mode 100644 index 00000000000..7e7c87f9d4e --- /dev/null +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java @@ -0,0 +1,172 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.tests.indexer; + +import com.beust.jcommander.internal.Lists; +import com.google.api.client.repackaged.com.google.common.base.Throwables; +import com.google.inject.Inject; +import com.metamx.common.logger.Logger; +import com.metamx.http.client.HttpClient; +import io.druid.curator.discovery.ServerDiscoveryFactory; +import io.druid.curator.discovery.ServerDiscoverySelector; +import io.druid.guice.annotations.Global; +import io.druid.testing.IntegrationTestingConfig; +import io.druid.testing.clients.EventReceiverFirehoseTestClient; +import io.druid.testing.guice.DruidTestModuleFactory; +import io.druid.testing.utils.RetryUtil; +import io.druid.testing.utils.ServerDiscoveryUtil; +import org.joda.time.DateTime; +import org.testng.annotations.Guice; +import org.testng.annotations.Test; + +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; + +@Guice(moduleFactory = DruidTestModuleFactory.class) +public class ITUnionQueryTest extends AbstractIndexerTest +{ + private static final Logger LOG = new Logger(ITUnionQueryTest.class); + private static final String REALTIME_TASK_RESOURCE = "/indexer/wikipedia_realtime_index_task.json"; + private static final String EVENT_RECEIVER_SERVICE_PREFIX = "eventReceiverServiceName"; + private static final String UNION_DATA_FILE = "/indexer/wikipedia_index_data.json"; + private static final String UNION_QUERIES_RESOURCE = "/indexer/union_queries.json"; + private static final String UNION_DATASOURCE = "wikipedia_index_test"; + + @Inject + ServerDiscoveryFactory factory; + + @Inject + @Global + HttpClient httpClient; + + @Inject + IntegrationTestingConfig config; + + @Test + public void testRealtimeIndexTask() throws Exception + { + final int numTasks = 4; + + try { + // Load 4 datasources with same dimensions + String task = setShutOffTime( + getTaskAsString(REALTIME_TASK_RESOURCE), + new DateTime(System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(3)) + ); + List taskIDs = Lists.newArrayList(); + for (int i = 0; i < numTasks; i++) { + taskIDs.add( + indexer.submitTask( + withServiceName( + withDataSource(task, UNION_DATASOURCE + i), + EVENT_RECEIVER_SERVICE_PREFIX + i + ) + ) + ); + } + for (int i = 0; i < numTasks; i++) { + postEvents(i); + } + + // sleep for a while to let the events ingested + TimeUnit.SECONDS.sleep(5); + + // should hit the queries on realtime task + LOG.info("Running Union Queries.."); + this.queryHelper.testQueriesFromFile(UNION_QUERIES_RESOURCE, 2); + + // wait for the task to complete + for (int i = 0; i < numTasks; i++) { + indexer.waitUntilTaskCompletes(taskIDs.get(i)); + } + // task should complete only after the segments are loaded by historical node + for (int i = 0; i < numTasks; i++) { + final int taskNum = i; + RetryUtil.retryUntil( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return coordinator.areSegmentsLoaded(UNION_DATASOURCE + taskNum); + } + }, + true, + 60000, + 10, + "Real-time generated segments loaded" + ); + } + // run queries on historical nodes + this.queryHelper.testQueriesFromFile(UNION_QUERIES_RESOURCE, 2); + + } + catch (Exception e) { + e.printStackTrace(); + throw Throwables.propagate(e); + } + finally { + for (int i = 0; i < numTasks; i++) { + unloadAndKillData(UNION_DATASOURCE + i); + } + } + + } + + private String setShutOffTime(String taskAsString, DateTime time) + { + return taskAsString.replace("#SHUTOFFTIME", time.toString()); + } + + private String withDataSource(String taskAsString, String dataSource) + { + return taskAsString.replace(UNION_DATASOURCE, dataSource); + } + + private String withServiceName(String taskAsString, String serviceName) + { + return taskAsString.replace(EVENT_RECEIVER_SERVICE_PREFIX, serviceName); + } + + public void postEvents(int id) throws Exception + { + final ServerDiscoverySelector eventReceiverSelector = factory.createSelector(EVENT_RECEIVER_SERVICE_PREFIX + id); + eventReceiverSelector.start(); + try { + ServerDiscoveryUtil.waitUntilInstanceReady(eventReceiverSelector, "Event Receiver"); + // Access the docker VM mapped host and port instead of service announced in zookeeper + String host = config.getMiddleManagerHost() + ":" + eventReceiverSelector.pick().getPort(); + + LOG.info("Event Receiver Found at host [%s]", host); + + EventReceiverFirehoseTestClient client = new EventReceiverFirehoseTestClient( + host, + EVENT_RECEIVER_SERVICE_PREFIX + id, + jsonMapper, + httpClient + ); + client.postEventsFromFile(UNION_DATA_FILE); + } + finally { + eventReceiverSelector.stop(); + } + } +} diff --git a/integration-tests/src/test/java/io/druid/tests/query/ITTwitterQueryTest.java b/integration-tests/src/test/java/io/druid/tests/query/ITTwitterQueryTest.java new file mode 100644 index 00000000000..584efdcf01b --- /dev/null +++ b/integration-tests/src/test/java/io/druid/tests/query/ITTwitterQueryTest.java @@ -0,0 +1,65 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.tests.query; + +import com.google.inject.Inject; +import io.druid.testing.clients.CoordinatorResourceTestClient; +import io.druid.testing.guice.DruidTestModuleFactory; +import io.druid.testing.utils.FromFileTestQueryHelper; +import io.druid.testing.utils.RetryUtil; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Guice; +import org.testng.annotations.Test; + +import java.util.concurrent.Callable; + +@Guice(moduleFactory = DruidTestModuleFactory.class) +public class ITTwitterQueryTest +{ + private static final String TWITTER_DATA_SOURCE = "twitterstream"; + private static final String TWITTER_QUERIES_RESOURCE = "/queries/twitterstream_queries.json"; + @Inject + CoordinatorResourceTestClient coordinatorClient; + @Inject + private FromFileTestQueryHelper queryHelper; + + @BeforeMethod + public void before() + { + // ensure that the segments twitter segments are loaded completely + RetryUtil.retryUntilTrue( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return coordinatorClient.areSegmentsLoaded(TWITTER_DATA_SOURCE); + } + }, "twitter segment load" + ); + } + + @Test + public void testQueriesFromFile() throws Exception + { + queryHelper.testQueriesFromFile(TWITTER_QUERIES_RESOURCE, 2); + } + +} diff --git a/integration-tests/src/test/java/io/druid/tests/query/ITWikipediaQueryTest.java b/integration-tests/src/test/java/io/druid/tests/query/ITWikipediaQueryTest.java new file mode 100644 index 00000000000..6279ca78252 --- /dev/null +++ b/integration-tests/src/test/java/io/druid/tests/query/ITWikipediaQueryTest.java @@ -0,0 +1,65 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.tests.query; + +import com.google.inject.Inject; +import io.druid.testing.clients.CoordinatorResourceTestClient; +import io.druid.testing.guice.DruidTestModuleFactory; +import io.druid.testing.utils.FromFileTestQueryHelper; +import io.druid.testing.utils.RetryUtil; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Guice; +import org.testng.annotations.Test; + +import java.util.concurrent.Callable; + +@Guice(moduleFactory = DruidTestModuleFactory.class) +public class ITWikipediaQueryTest +{ + private static final String WIKIPEDIA_DATA_SOURCE = "wikipedia_editstream"; + private static final String WIKIPEDIA_QUERIES_RESOURCE = "/queries/wikipedia_editstream_queries.json"; + @Inject + private CoordinatorResourceTestClient coordinatorClient; + @Inject + private FromFileTestQueryHelper queryHelper; + + @BeforeMethod + public void before() + { + // ensure that twitter segments are loaded completely + RetryUtil.retryUntilTrue( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return coordinatorClient.areSegmentsLoaded(WIKIPEDIA_DATA_SOURCE); + } + }, "wikipedia segment load" + ); + } + + @Test + public void testQueriesFromFile() throws Exception + { + queryHelper.testQueriesFromFile(WIKIPEDIA_QUERIES_RESOURCE, 2); + } + +} diff --git a/integration-tests/src/test/resources/indexer/select.query b/integration-tests/src/test/resources/indexer/select.query new file mode 100644 index 00000000000..463ffef08e1 --- /dev/null +++ b/integration-tests/src/test/resources/indexer/select.query @@ -0,0 +1,19 @@ +{ + "queryType": "select", + "intervals": ["2013-08-31/2013-09-01"], + "dataSource": "wikipedia_index_test", + "granularity": "all", + "filter": { + "type": "selector", + "dimension": "language", + "value": "en" + }, + "pagingSpec": { + "threshold": 10 + }, + "context": { + "useCache": "false", + "populateCache": "false", + "timeout": 60000 + } + } diff --git a/integration-tests/src/test/resources/indexer/select_reindex.query b/integration-tests/src/test/resources/indexer/select_reindex.query new file mode 100644 index 00000000000..e6a162377dc --- /dev/null +++ b/integration-tests/src/test/resources/indexer/select_reindex.query @@ -0,0 +1,19 @@ +{ + "queryType": "select", + "intervals": ["2013-08-31/2013-09-01"], + "dataSource": "wikipedia_reindex_test", + "granularity": "all", + "filter": { + "type": "selector", + "dimension": "language", + "value": "en" + }, + "pagingSpec": { + "threshold": 10 + }, + "context": { + "useCache": "false", + "populateCache": "false", + "timeout": 60000 + } + } diff --git a/integration-tests/src/test/resources/indexer/union_queries.json b/integration-tests/src/test/resources/indexer/union_queries.json new file mode 100644 index 00000000000..9c0ce5a0de5 --- /dev/null +++ b/integration-tests/src/test/resources/indexer/union_queries.json @@ -0,0 +1,564 @@ +[ + { + "description": "timeseries, filtered, all aggs, all", + "query": { + "queryType": "timeseries", + "dataSource": { + "type": "union", + "dataSources": [ + "wikipedia_index_test1", "wikipedia_index_test2", "wikipedia_index_test3", + "wikipedia_index_test0" + ] + }, + "intervals": ["2013-08-31/2013-09-01"], + "granularity": "all", + "filter": { + "type": "selector", + "dimension": "language", + "value": "en" + }, + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + }, + { + "type": "doubleSum", + "fieldName": "added", + "name": "added" + }, + { + "type": "doubleSum", + "fieldName": "deleted", + "name": "deleted" + }, + { + "type": "doubleSum", + "fieldName": "delta", + "name": "delta" + } + ], + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": { + "added": 2064.0, + "count": 8, + "delta": 748.0, + "deleted": 1316.0, + "rows": 8 + } + } + ] + }, + { + "description": "topN, all aggs, page dim, uniques metric", + "query": { + "queryType": "topN", + "dataSource": { + "type": "union", + "dataSources": [ + "wikipedia_index_test1", "wikipedia_index_test2", "wikipedia_index_test3", + "wikipedia_index_test0" + ] + }, + "intervals": ["2013-08-31/2013-09-01"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + }, + { + "type": "doubleSum", + "fieldName": "added", + "name": "added" + }, + { + "type": "doubleSum", + "fieldName": "deleted", + "name": "deleted" + }, + { + "type": "doubleSum", + "fieldName": "delta", + "name": "delta" + } + ], + "dimension": "page", + "metric": "added", + "threshold": 3, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": [ + { + "added": 3620.0, + "count": 4, + "page": "Crimson Typhoon", + "delta": 3600.0, + "deleted": 20.0, + "rows": 4 + }, + { + "added": 1836.0, + "count": 4, + "page": "Striker Eureka", + "delta": 1320.0, + "deleted": 516.0, + "rows": 4 + }, + { + "added": 492.0, + "count": 4, + "page": "Cherno Alpha", + "delta": 444.0, + "deleted": 48.0, + "rows": 4 + } + ] + } + ] + }, + { + "description": "topN, all aggs, page dim, count metric, postAggs", + "query": { + "queryType": "topN", + "dataSource": { + "type": "union", + "dataSources": [ + "wikipedia_index_test1", "wikipedia_index_test2", "wikipedia_index_test3", + "wikipedia_index_test0" + ] + }, + "intervals": ["2013-08-31/2013-09-01"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + }, + { + "type": "doubleSum", + "fieldName": "added", + "name": "added" + }, + { + "type": "doubleSum", + "fieldName": "deleted", + "name": "deleted" + }, + { + "type": "doubleSum", + "fieldName": "delta", + "name": "delta" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "sumOfAddedDeletedConst", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "added", + "fieldName": "added" + }, + { + "type": "arithmetic", + "name": "", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "constant", + "name": "constant", + "value": 1000 + } + ] + } + ] + } + ], + "dimension": "page", + "metric": "added", + "threshold": 3, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": [ + { + "added": 3620.0, + "count": 4, + "page": "Crimson Typhoon", + "delta": 3600.0, + "deleted": 20.0, + "sumOfAddedDeletedConst": 4640.0, + "rows": 4 + }, + { + "added": 1836.0, + "count": 4, + "page": "Striker Eureka", + "delta": 1320.0, + "deleted": 516.0, + "sumOfAddedDeletedConst": 3352.0, + "rows": 4 + }, + { + "added": 492.0, + "count": 4, + "page": "Cherno Alpha", + "delta": 444.0, + "deleted": 48.0, + "sumOfAddedDeletedConst": 1540.0, + "rows": 4 + } + ] + } + ] + }, + { + "description": "topN, lexicographic, two aggs, language dim, postAggs", + "query": { + "queryType": "topN", + "dataSource": { + "type": "union", + "dataSources": [ + "wikipedia_index_test1", "wikipedia_index_test2", "wikipedia_index_test3", + "wikipedia_index_test0" + ] + }, + "intervals": ["2013-08-31/2013-09-01"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "sumOfRowsAndCount", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "rows", + "fieldName": "rows" + }, + { + "type": "fieldAccess", + "name": "count", + "fieldName": "count" + } + ] + } + ], + "dimension": "language", + "metric": { + "type": "lexicographic", + "previousStop": "a" + }, + "threshold": 3, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": [ + { + "sumOfRowsAndCount": 16.0, + "count": 8, + "language": "en", + "rows": 8 + }, + { + "sumOfRowsAndCount": 8.0, + "count": 4, + "language": "ja", + "rows": 4 + }, + { + "sumOfRowsAndCount": 8.0, + "count": 4, + "language": "ru", + "rows": 4 + } + ] + } + ] + }, + { + "description": "groupBy, two aggs, namespace dim, postAggs", + "query": { + "queryType": "groupBy", + "dataSource": { + "type": "union", + "dataSources": [ + "wikipedia_index_test1", "wikipedia_index_test2", "wikipedia_index_test3", + "wikipedia_index_test0" + ] + }, + "intervals": ["2013-08-31/2013-09-01"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "sumOfRowsAndCount", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "rows", + "fieldName": "rows" + }, + { + "type": "fieldAccess", + "name": "count", + "fieldName": "count" + } + ] + } + ], + "dimensions": ["namespace"], + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "version": "v1", + "timestamp": "2013-08-31T00:00:00.000Z", + "event": { + "sumOfRowsAndCount": 16.0, + "count": 8, + "rows": 8, + "namespace": "article" + } + }, + { + "version": "v1", + "timestamp": "2013-08-31T00:00:00.000Z", + "event": { + "sumOfRowsAndCount": 24.0, + "count": 12, + "rows": 12, + "namespace": "wikipedia" + } + } + ] + }, + { + "description": "groupBy, two aggs, namespace + robot dim, postAggs", + "query": { + "queryType": "groupBy", + "dataSource": { + "type": "union", + "dataSources": [ + "wikipedia_index_test1", "wikipedia_index_test2", "wikipedia_index_test3", + "wikipedia_index_test0" + ] + }, + "intervals": ["2013-08-31/2013-09-01"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "sumOfRowsAndCount", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "rows", + "fieldName": "rows" + }, + { + "type": "fieldAccess", + "name": "count", + "fieldName": "count" + } + ] + } + ], + "dimensions": ["namespace", "robot"], + "limitSpec": { + "type": "default", + "limit": 3, + "orderBy": ["robot", "namespace"] + }, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "version": "v1", + "timestamp": "2013-08-31T00:00:00.000Z", + "event": { + "sumOfRowsAndCount": 8.0, + "count": 4, + "robot": "false", + "rows": 4, + "namespace": "article" + } + }, + { + "version": "v1", + "timestamp": "2013-08-31T00:00:00.000Z", + "event": { + "sumOfRowsAndCount": 8.0, + "count": 4, + "robot": "true", + "rows": 4, + "namespace": "article" + } + }, + { + "version": "v1", + "timestamp": "2013-08-31T00:00:00.000Z", + "event": { + "sumOfRowsAndCount": 24.0, + "count": 12, + "robot": "true", + "rows": 12, + "namespace": "wikipedia" + } + } + ] + }, + { + "query": { + "queryType": "search", + "intervals": ["2013-08-31/2013-09-01"], + "dataSource": { + "type": "union", + "dataSources": [ + "wikipedia_index_test1", "wikipedia_index_test2", "wikipedia_index_test3", + "wikipedia_index_test0" + ] + }, + "granularity": "all", + "query": { + "type": "insensitive_contains", + "value": "ip" + }, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T00:00:00.000Z", + "result": [ + { + "dimension": "user", + "value": "triplets" + }, + { + "dimension": "namespace", + "value": "wikipedia" + } + ] + } + ] + }, + { + "description": "timeboundary, 1 agg, union", + "query": { + "queryType": "timeBoundary", + "dataSource": { + "type": "union", + "dataSources": [ + "wikipedia_index_test1", "wikipedia_index_test2", "wikipedia_index_test3", + "wikipedia_index_test0" + ] + } + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": { + "minTime": "2013-08-31T01:02:33.000Z", + "maxTime": "2013-08-31T12:41:27.000Z" + } + } + ] + } +] diff --git a/integration-tests/src/test/resources/indexer/union_select_query.json b/integration-tests/src/test/resources/indexer/union_select_query.json new file mode 100644 index 00000000000..76e4d0be63e --- /dev/null +++ b/integration-tests/src/test/resources/indexer/union_select_query.json @@ -0,0 +1,121 @@ +{ + "query": { + "queryType": "select", + "intervals": ["2013-08-31/2013-09-01"], + "dataSource": { + "type": "union", + "dataSources": [ + "wikipedia_index_test" + ] + }, + "granularity": "all", + "filter": { + "type": "selector", + "dimension": "language", + "value": "en" + }, + "pagingSpec": { + "threshold": 10 + }, + "context": { + "useCache": "false", + "populateCache": "false", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": { + "pagingIdentifiers": { + "wikipedia_index_test0_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:43.993Z": 0, + "wikipedia_index_test1_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.108Z": 0, + "wikipedia_index_test2_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.236Z": 0, + "wikipedia_index_test3_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.374Z": 0 + }, + "events": [ + { + "segmentId": "wikipedia_index_test0_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:43.993Z", + "offset": 0, + "event": { + "timestamp": "2013-08-31T01:02:33.000Z", + "page": "Gypsy Danger", + "added": 57.0, + "deleted": 200.0 + } + }, + { + "segmentId": "wikipedia_index_test1_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.108Z", + "offset": 0, + "event": { + "timestamp": "2013-08-31T01:02:33.000Z", + "page": "Gypsy Danger", + "added": 57.0, + "deleted": 200.0 + } + }, + { + "segmentId": "wikipedia_index_test2_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.236Z", + "offset": 0, + "event": { + "timestamp": "2013-08-31T01:02:33.000Z", + "page": "Gypsy Danger", + "added": 57.0, + "deleted": 200.0 + } + }, + { + "segmentId": "wikipedia_index_test3_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.374Z", + "offset": 0, + "event": { + "timestamp": "2013-08-31T01:02:33.000Z", + "page": "Gypsy Danger", + "added": 57.0, + "deleted": 200.0 + } + }, + { + "segmentId": "wikipedia_index_test0_2013-08-31T0com.metamx.common.ISE: one or more twitter queries failed0:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:43.993Z", + "offset": 0, + "event": { + "timestamp": "2013-08-31T03:32:45.000Z", + "page": "Striker Eureka", + "added": 459.0, + "deleted": 129.0 + } + }, + { + "segmentId": "wikipedia_index_test1_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.108Z", + "offset": 0, + "event": { + "timestamp": "2013-08-31T03:32:45.000Z", + "page": "Striker Eureka", + "added": 459.0, + "deleted": 129.0 + } + }, + { + "segmentId": "wikipedia_index_test2_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.236Z", + "offset": 0, + "event": { + "timestamp": "2013-08-31T03:32:45.000Z", + "page": "Striker Eureka", + "added": 459.0, + "deleted": 129.0 + } + }, + { + "segmentId": "wikipedia_index_test3_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2014-05-01T15:27:44.374Z", + "offset": 0, + "event": { + "timestamp": "2013-08-31T03:32:45.000Z", + "page": "Striker Eureka", + "added": 459.0, + "deleted": 129.0 + } + } + ] + } + } + ] +} diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_data.json b/integration-tests/src/test/resources/indexer/wikipedia_index_data.json new file mode 100644 index 00000000000..592996e1805 --- /dev/null +++ b/integration-tests/src/test/resources/indexer/wikipedia_index_data.json @@ -0,0 +1,5 @@ +{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} +{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} +{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} +{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} +{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} \ No newline at end of file diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_queries.json b/integration-tests/src/test/resources/indexer/wikipedia_index_queries.json new file mode 100644 index 00000000000..caea782fcb5 --- /dev/null +++ b/integration-tests/src/test/resources/indexer/wikipedia_index_queries.json @@ -0,0 +1,16 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "wikipedia_index_test" + }, + "expectedResults":[ { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-08-31T12:41:27.000Z" + } + } ] + } +] \ No newline at end of file diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_index_task.json new file mode 100644 index 00000000000..369f9aac6b4 --- /dev/null +++ b/integration-tests/src/test/resources/indexer/wikipedia_index_task.json @@ -0,0 +1,59 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "wikipedia_index_test", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-01" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", "language", "user", "unpatrolled", "newPage", "robot", "anonymous", + "namespace", "continent", "country", "region", "city" + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "local", + "baseDir": "/resources/indexer", + "filter": "wikipedia_index_data.json" + } + }, + "tuningConfig": { + "type": "index" + } + } +} \ No newline at end of file diff --git a/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_task.json new file mode 100644 index 00000000000..a204abd9512 --- /dev/null +++ b/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_task.json @@ -0,0 +1,71 @@ +{ + "type": "index_realtime", + "spec": { + "dataSchema": { + "dataSource": "wikipedia_index_test", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second" + }, + "parser": { + "type" : "map", + "parseSpec": { + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec" : { + "dimensions": [ + "page", "language", "user", "unpatrolled", "newPage", "robot", "anonymous", + "namespace", "continent", "country", "region", "city" + ] + } + } + } + }, + "ioConfig": { + "type": "realtime", + "firehose": { + "type": "timed", + "shutoffTime": "#SHUTOFFTIME", + "delegate": { + "type": "receiver", + "serviceName": "eventReceiverServiceName", + "bufferSize": 100000 + } + } + + }, + "tuningConfig": { + "type": "realtime", + "maxRowsInMemory": 1, + "intermediatePersistPeriod": "PT1M", + "windowPeriod": "PT1M", + "rejectionPolicy": { + "type": "none" + } + } + } + +} diff --git a/integration-tests/src/test/resources/indexer/wikipedia_reindex_task.json b/integration-tests/src/test/resources/indexer/wikipedia_reindex_task.json new file mode 100644 index 00000000000..7d4b937dfd9 --- /dev/null +++ b/integration-tests/src/test/resources/indexer/wikipedia_reindex_task.json @@ -0,0 +1,62 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "wikipedia_index_test", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-01" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "page", "language", "user", "unpatrolled", "newPage", "robot", "anonymous", + "namespace", "continent", "country", "region", "city" + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "ingestSegment", + "dataSource": "wikipedia_index_test", + "dimensions": ["user", "nonexist"], + "metrics": ["added", "added2"], + "interval": "2013-08-31/2013-09-01" + } + }, + "tuningConfig": { + "type": "index" + } + } +} diff --git a/integration-tests/src/test/resources/queries/twitterstream_queries.json b/integration-tests/src/test/resources/queries/twitterstream_queries.json new file mode 100644 index 00000000000..78bb134ef5c --- /dev/null +++ b/integration-tests/src/test/resources/queries/twitterstream_queries.json @@ -0,0 +1,780 @@ +[ + { + "description": "timeseries, 2 aggs", + "query": { + "queryType": "timeseries", + "dataSource": "twitterstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-04T00:00:00.000"], + "granularity": "day", + "aggregations": [ + { + "type": "doubleSum", + "name": "num_tweets", + "fieldName": "count" + }, + { + "type": "doubleSum", + "name": "tweet_length", + "fieldName": "tweet_length" + } + ], + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-01-01T00:00:00.000Z", + "result": { + "tweet_length": 2.40241323E8, + "num_tweets": 3754028.0 + } + }, + { + "timestamp": "2013-01-02T00:00:00.000Z", + "result": { + "tweet_length": 2.46397801E8, + "num_tweets": 3799466.0 + } + }, + { + "timestamp": "2013-01-03T00:00:00.000Z", + "result": { + "tweet_length": 2.31365019E8, + "num_tweets": 3552419.0 + } + } + ] + }, + { + "description": "topN, 2 aggs, lexicographic", + "query": { + "queryType": "topN", + "dataSource": "twitterstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-04T00:00:00.000"], + "granularity": "day", + "aggregations": [ + { + "type": "doubleSum", + "name": "num_tweets", + "fieldName": "count" + }, + { + "type": "doubleSum", + "name": "tweet_length", + "fieldName": "tweet_length" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "avg_tweet_len", + "fn": "/", + "fields": [ + { + "type": "fieldAccess", + "name": "tweet_length", + "fieldName": "tweet_length" + }, + { + "type": "fieldAccess", + "name": "num_tweets", + "fieldName": "num_tweets" + } + ] + } + ], + "dimension": "user_name", + "metric": { + "type": "lexicographic" + }, + "threshold": 2, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-01-01T00:00:00.000Z", + "result": [ + { + "user_name": "000000000000087", + "tweet_length": 14.0, + "num_tweets": 1.0, + "avg_tweet_len": 14.0 + }, + { + "user_name": "0000000000mghi", + "tweet_length": 291.0, + "num_tweets": 4.0, + "avg_tweet_len": 72.75 + } + ] + }, + { + "timestamp": "2013-01-02T00:00:00.000Z", + "result": [ + { + "user_name": "000000000037", + "tweet_length": 13.0, + "num_tweets": 1.0, + "avg_tweet_len": 13.0 + }, + { + "user_name": "0000000000mghi", + "tweet_length": 21.0, + "num_tweets": 1.0, + "avg_tweet_len": 21.0 + } + ] + }, + { + "timestamp": "2013-01-03T00:00:00.000Z", + "result": [ + { + "user_name": "000000007", + "tweet_length": 37.0, + "num_tweets": 1.0, + "avg_tweet_len": 37.0 + }, + { + "user_name": "00000000b", + "tweet_length": 119.0, + "num_tweets": 1.0, + "avg_tweet_len": 119.0 + } + ] + } + ] + }, + { + "description": "topN, 2 aggs", + "query": { + "queryType": "topN", + "dataSource": "twitterstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-04T00:00:00.000"], + "granularity": "day", + "aggregations": [ + { + "type": "doubleSum", + "name": "num_tweets", + "fieldName": "count" + }, + { + "type": "doubleSum", + "name": "tweet_length", + "fieldName": "tweet_length" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "avg_tweet_len", + "fn": "/", + "fields": [ + { + "type": "fieldAccess", + "name": "tweet_length", + "fieldName": "tweet_length" + }, + { + "type": "fieldAccess", + "name": "num_tweets", + "fieldName": "num_tweets" + } + ] + } + ], + "dimension": "user_name", + "metric": { + "type": "numeric", + "metric": "num_tweets" + }, + "threshold": 2, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-01-01T00:00:00.000Z", + "result": [ + { + "user_name": "Favstar_Bot", + "tweet_length": 2002.0, + "num_tweets": 33.0, + "avg_tweet_len": 60.666666666666664 + }, + { + "user_name": "SportsAB", + "tweet_length": 1114.0, + "num_tweets": 26.0, + "avg_tweet_len": 42.84615384615385 + } + ] + }, + { + "timestamp": "2013-01-02T00:00:00.000Z", + "result": [ + { + "user_name": "Favstar_Bot", + "tweet_length": 2185.0, + "num_tweets": 36.0, + "avg_tweet_len": 60.69444444444444 + }, + { + "user_name": "SportsAB", + "tweet_length": 1148.0, + "num_tweets": 23.0, + "avg_tweet_len": 49.91304347826087 + } + ] + }, + { + "timestamp": "2013-01-03T00:00:00.000Z", + "result": [ + { + "user_name": "SportsAB", + "tweet_length": 882.0, + "num_tweets": 22.0, + "avg_tweet_len": 40.09090909090909 + }, + { + "user_name": "furin0620", + "tweet_length": 867.0, + "num_tweets": 21.0, + "avg_tweet_len": 41.285714285714285 + } + ] + } + ] + }, + { + "description": "topN, 2 aggs, filtered", + "query": { + "queryType": "topN", + "dataSource": "twitterstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-04T00:00:00.000"], + "granularity": "day", + "filter": { + "type": "or", + "fields": [ + { + "type": "selector", + "dimension": "user_name", + "value": "Favstar_Bot" + }, + { + "type": "selector", + "dimension": "user_name", + "value": "SportsAB" + }, + { + "type": "selector", + "dimension": "user_name", + "value": "furin0620" + } + ] + }, + "aggregations": [ + { + "type": "doubleSum", + "name": "num_tweets", + "fieldName": "count" + }, + { + "type": "doubleSum", + "name": "tweet_length", + "fieldName": "tweet_length" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "avg_tweet_len", + "fn": "/", + "fields": [ + { + "type": "fieldAccess", + "name": "tweet_length", + "fieldName": "tweet_length" + }, + { + "type": "fieldAccess", + "name": "num_tweets", + "fieldName": "num_tweets" + } + ] + } + ], + "dimension": "user_name", + "metric": { + "type": "numeric", + "metric": "num_tweets" + }, + "threshold": 2, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-01-01T00:00:00.000Z", + "result": [ + { + "user_name": "Favstar_Bot", + "tweet_length": 2002.0, + "num_tweets": 33.0, + "avg_tweet_len": 60.666666666666664 + }, + { + "user_name": "SportsAB", + "tweet_length": 1114.0, + "num_tweets": 26.0, + "avg_tweet_len": 42.84615384615385 + } + ] + }, + { + "timestamp": "2013-01-02T00:00:00.000Z", + "result": [ + { + "user_name": "Favstar_Bot", + "tweet_length": 2185.0, + "num_tweets": 36.0, + "avg_tweet_len": 60.69444444444444 + }, + { + "user_name": "SportsAB", + "tweet_length": 1148.0, + "num_tweets": 23.0, + "avg_tweet_len": 49.91304347826087 + } + ] + }, + { + "timestamp": "2013-01-03T00:00:00.000Z", + "result": [ + { + "user_name": "SportsAB", + "tweet_length": 882.0, + "num_tweets": 22.0, + "avg_tweet_len": 40.09090909090909 + }, + { + "user_name": "furin0620", + "tweet_length": 867.0, + "num_tweets": 21.0, + "avg_tweet_len": 41.285714285714285 + } + ] + } + ] + }, + { + "description": "groupBy", + "query": { + "queryType": "groupBy", + "dataSource": "twitterstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-04T00:00:00.000"], + "granularity": "day", + "aggregations": [ + { + "type": "doubleSum", + "name": "num_tweets", + "fieldName": "count" + }, + { + "type": "doubleSum", + "name": "tweet_length", + "fieldName": "tweet_length" + } + ], + "dimensions": ["has_links"] + }, + "expectedResults": [ + { + "version": "v1", + "timestamp": "2013-01-01T00:00:00.000Z", + "event": { + "has_links": "No", + "tweet_length": 2.08803904E8, + "num_tweets": 3377791.0 + } + }, + { + "version": "v1", + "timestamp": "2013-01-01T00:00:00.000Z", + "event": { + "has_links": "Yes", + "tweet_length": 3.143742E7, + "num_tweets": 376237.0 + } + }, + { + "version": "v1", + "timestamp": "2013-01-02T00:00:00.000Z", + "event": { + "has_links": "No", + "tweet_length": 2.10402688E8, + "num_tweets": 3375243.0 + } + }, + { + "version": "v1", + "timestamp": "2013-01-02T00:00:00.000Z", + "event": { + "has_links": "Yes", + "tweet_length": 3.599512E7, + "num_tweets": 424223.0 + } + }, + { + "version": "v1", + "timestamp": "2013-01-03T00:00:00.000Z", + "event": { + "has_links": "No", + "tweet_length": 1.96451456E8, + "num_tweets": 3144985.0 + } + }, + { + "version": "v1", + "timestamp": "2013-01-03T00:00:00.000Z", + "event": { + "has_links": "Yes", + "tweet_length": 3.4913568E7, + "num_tweets": 407434.0 + } + } + ] + }, + { + "query": { + "queryType": "search", + "intervals": ["2013-01-01T00:00:00.000/2013-01-04T00:00:00.000"], + "dataSource": "twitterstream", + "granularity": "all", + "searchDimensions": ["user_name"], + "sort": { + "type": "lexicographic" + }, + "query": { + "type": "insensitive_contains", + "value": "Sports" + }, + "limit": 3, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-01-01T00:00:00.000Z", + "result": [ + { + "dimension": "user_name", + "value": "1011Sports" + }, + { + "dimension": "user_name", + "value": "11AliveSports" + }, + { + "dimension": "user_name", + "value": "1World_Sports" + } + ] + } + ] + }, + { + "description": "groupByArbitraryInterval", + "query": { + "queryType": "groupBy", + "dataSource": "twitterstream", + "intervals": ["2013-01-01T15:10:10.090/2013-01-03T19:30:01.090"], + "granularity": "day", + "aggregations": [ + { + "type": "doubleSum", + "name": "num_tweets", + "fieldName": "count" + }, + { + "type": "doubleSum", + "name": "tweet_length", + "fieldName": "tweet_length" + } + ], + "dimensions": ["has_links"] + }, + "expectedResults": [ + { + "version": "v1", + "timestamp": "2013-01-01T00:00:00.000Z", + "event": { + "has_links": "No", + "tweet_length": 7.4820448E7, + "num_tweets": 1170229.0 + } + }, + { + "version": "v1", + "timestamp": "2013-01-01T00:00:00.000Z", + "event": { + "has_links": "Yes", + "tweet_length": 1.149719E7, + "num_tweets": 136582.0 + } + }, + { + "version": "v1", + "timestamp": "2013-01-02T00:00:00.000Z", + "event": { + "has_links": "No", + "tweet_length": 2.10402688E8, + "num_tweets": 3375243.0 + } + }, + { + "version": "v1", + "timestamp": "2013-01-02T00:00:00.000Z", + "event": { + "has_links": "Yes", + "tweet_length": 3.599512E7, + "num_tweets": 424223.0 + } + }, + { + "version": "v1", + "timestamp": "2013-01-03T00:00:00.000Z", + "event": { + "has_links": "No", + "tweet_length": 1.59141088E8, + "num_tweets": 2567986.0 + } + }, + { + "version": "v1", + "timestamp": "2013-01-03T00:00:00.000Z", + "event": { + "has_links": "Yes", + "tweet_length": 2.8345444E7, + "num_tweets": 328917.0 + } + } + ] + }, + { + "description": "segmentMetadata", + "query": { + "queryType": "segmentMetadata", + "dataSource": "twitterstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-04T00:00:00.000"], + "toInclude": { + "type": "list", + "columns": ["has_links", "has_links"] + } + }, + "expectedResults": [ + { + "id": "twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9", + "intervals": ["2013-01-01T00:00:00.000Z/2013-01-02T00:00:00.000Z"], + "columns": { + "has_links": { + "type": "STRING", + "size": 7773438, + "cardinality": 2, + "errorMessage": null + } + }, + "size": 747056474 + }, + { + "id": "twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9", + "intervals": ["2013-01-02T00:00:00.000Z/2013-01-03T00:00:00.000Z"], + "columns": { + "has_links": { + "type": "STRING", + "size": 7901000, + "cardinality": 2, + "errorMessage": null + } + }, + "size": 755796690 + }, + { + "id": "twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9", + "intervals": ["2013-01-03T00:00:00.000Z/2013-01-04T00:00:00.000Z"], + "columns": { + "has_links": { + "type": "STRING", + "size": 7405654, + "cardinality": 2, + "errorMessage": null + } + }, + "size": 706893542 + } + ] + }, + { + "description": "topN, 2 aggs, topN over dependent postAgg", + "query": { + "queryType": "topN", + "dataSource": "twitterstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-04T00:00:00.000"], + "granularity": "day", + "aggregations": [ + { + "type": "doubleSum", + "name": "num_tweets", + "fieldName": "count" + }, + { + "type": "doubleSum", + "name": "tweet_length", + "fieldName": "tweet_length" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "avg_tweet_len", + "fn": "/", + "fields": [ + { + "type": "fieldAccess", + "name": "tweet_length", + "fieldName": "tweet_length" + }, + { + "type": "fieldAccess", + "name": "num_tweets", + "fieldName": "num_tweets" + } + ] + }, + { + "type": "arithmetic", + "name": "avg_tweet_len_half", + "fn": "/", + "fields": [ + { + "type": "fieldAccess", + "name": "avg_tweet_len", + "fieldName": "avg_tweet_len" + }, + { + "type": "constant", + "value": "2" + } + ] + }, + { + "type": "arithmetic", + "name": "avg_tweet_len_doubled", + "fn": "*", + "fields": [ + { + "type": "fieldAccess", + "name": "avg_tweet_len", + "fieldName": "avg_tweet_len" + }, + { + "type": "constant", + "value": "2" + } + ] + } + ], + "dimension": "user_name", + "metric": { + "type": "numeric", + "metric": "avg_tweet_len_doubled" + }, + "threshold": 2, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-01-01T00:00:00.000Z", + "result": [ + { + "user_name": "___soMALIa___", + "tweet_length": 539.0, + "avg_tweet_len_half": 269.5, + "avg_tweet_len_doubled": 1078.0, + "num_tweets": 1.0, + "avg_tweet_len": 539.0 + }, + { + "user_name": "SophiiiaSlr", + "tweet_length": 530.0, + "avg_tweet_len_half": 265.0, + "avg_tweet_len_doubled": 1060.0, + "num_tweets": 1.0, + "avg_tweet_len": 530.0 + } + ] + }, + { + "timestamp": "2013-01-02T00:00:00.000Z", + "result": [ + { + "user_name": "FallenReckless", + "tweet_length": 518.0, + "avg_tweet_len_half": 259.0, + "avg_tweet_len_doubled": 1036.0, + "num_tweets": 1.0, + "avg_tweet_len": 518.0 + }, + { + "user_name": "SigaMike", + "tweet_length": 514.0, + "avg_tweet_len_half": 257.0, + "avg_tweet_len_doubled": 1028.0, + "num_tweets": 1.0, + "avg_tweet_len": 514.0 + } + ] + }, + { + "timestamp": "2013-01-03T00:00:00.000Z", + "result": [ + { + "user_name": "Alejo_InReverse", + "tweet_length": 560.0, + "avg_tweet_len_half": 280.0, + "avg_tweet_len_doubled": 1120.0, + "num_tweets": 1.0, + "avg_tweet_len": 560.0 + }, + { + "user_name": "GavLeftHome", + "tweet_length": 506.0, + "avg_tweet_len_half": 253.0, + "avg_tweet_len_doubled": 1012.0, + "num_tweets": 1.0, + "avg_tweet_len": 506.0 + } + ] + } + ] + } +] diff --git a/integration-tests/src/test/resources/queries/wikipedia_editstream_queries.json b/integration-tests/src/test/resources/queries/wikipedia_editstream_queries.json new file mode 100644 index 00000000000..157e8b04750 --- /dev/null +++ b/integration-tests/src/test/resources/queries/wikipedia_editstream_queries.json @@ -0,0 +1,1063 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query": { + "queryType": "timeseries", + "dataSource": "wikipedia_editstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-08T00:00:00.000"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + } + ], + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-01-01T00:00:00.000Z", + "result": { + "rows": 2390950 + } + } + ] + }, + { + "description": "timeseries, all aggs, all", + "query": { + "queryType": "timeseries", + "dataSource": "wikipedia_editstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-08T00:00:00.000"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + }, + { + "type": "doubleSum", + "fieldName": "added", + "name": "added" + }, + { + "type": "doubleSum", + "fieldName": "deleted", + "name": "deleted" + }, + { + "type": "doubleSum", + "fieldName": "variation", + "name": "variation" + }, + { + "type": "doubleSum", + "fieldName": "delta", + "name": "delta" + }, + { + "type": "approxHistogramFold", + "name": "delta_hist", + "fieldName": "delta_hist" + }, + { + "type": "hyperUnique", + "fieldName": "unique_users", + "name": "unique_users" + } + ], + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-01-01T00:00:00.000Z", + "result": { + "added": 9.11526338E8, + "count": 2815650, + "delta": 5.48967603E8, + "variation": 1.274085073E9, + "delta_hist": { + "breaks": [ + -2634692.25, -2048505.0, -1462317.75, -876130.4375, -289943.125, 296244.1875, + 882431.5, 1468619.0 + ], + "counts": [1.0, 2.0, 1.0, 56.0, 2815544.0, 41.0, 5.0] + }, + "unique_users": 229361.39005604674, + "deleted": -3.62558735E8, + "rows": 2390950 + } + } + ] + }, + { + "description": "timeseries, filtered, all aggs, all", + "query": { + "queryType": "timeseries", + "dataSource": "wikipedia_editstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-08T00:00:00.000"], + "granularity": "all", + "filter": { + "type": "and", + "fields": [ + { + "type": "selector", + "dimension": "namespace", + "value": "article" + }, + { + "type": "or", + "fields": [ + { + "type": "selector", + "dimension": "language", + "value": "en" + }, + { + "type": "selector", + "dimension": "unpatrolled", + "value": "0" + } + ] + } + ] + }, + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + }, + { + "type": "doubleSum", + "fieldName": "added", + "name": "added" + }, + { + "type": "doubleSum", + "fieldName": "deleted", + "name": "deleted" + }, + { + "type": "doubleSum", + "fieldName": "variation", + "name": "variation" + }, + { + "type": "doubleSum", + "fieldName": "delta", + "name": "delta" + }, + { + "type": "approxHistogramFold", + "name": "delta_hist", + "fieldName": "delta_hist" + }, + { + "type": "hyperUnique", + "fieldName": "unique_users", + "name": "unique_users" + } + ], + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-01-01T00:00:00.000Z", + "result": { + "added": 3.49393993E8, + "count": 1829240, + "delta": 2.24089868E8, + "variation": 4.74698118E8, + "delta_hist": { + "breaks": [ + -754245.3125, -565684.0, -377122.6875, -188561.359375, -0.03125, 188561.296875, + 377122.625, 565684.0 + ], + "counts": [1.0, 2.0, 11.0, 572.0, 1828642.0, 10.0, 2.0] + }, + "unique_users": 166138.2309016003, + "deleted": -1.25304125E8, + "rows": 1556534 + } + } + ] + }, + { + "description": "timeseries, 3 aggs, 1 post agg, all", + "query": { + "queryType": "timeseries", + "dataSource": "wikipedia_editstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-08T00:00:00.000"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + }, + { + "type": "doubleSum", + "fieldName": "added", + "name": "added" + }, + { + "type": "doubleSum", + "fieldName": "deleted", + "name": "deleted" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "sumOfAddedDeletedConst", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "added", + "fieldName": "added" + }, + { + "type": "arithmetic", + "name": "", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "constant", + "name": "constant", + "value": 1000 + } + ] + } + ] + } + ], + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-01-01T00:00:00.000Z", + "result": { + "added": 9.11526338E8, + "count": 2815650, + "deleted": -3.62558735E8, + "sumOfAddedDeletedConst": 5.48968603E8, + "rows": 2390950 + } + } + ] + }, + { + "description": "topN, 1 agg", + "query": { + "queryType": "topN", + "dataSource": "wikipedia_editstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-08T00:00:00.000"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + } + ], + "dimension": "page", + "metric": "rows", + "threshold": 3, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-01-01T00:00:00.000Z", + "result": [ + { + "page": "Wikipedia:Vandalismusmeldung", + "rows": 991 + }, + { + "page": "Wikipedia:Administrators'_noticeboard/Incidents", + "rows": 990 + }, + { + "page": "Wikipedia:Administrator_intervention_against_vandalism", + "rows": 800 + } + ] + } + ] + }, + { + "description": "topN, all aggs, page dim, uniques metric", + "query": { + "queryType": "topN", + "dataSource": "wikipedia_editstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-08T00:00:00.000"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + }, + { + "type": "doubleSum", + "fieldName": "added", + "name": "added" + }, + { + "type": "doubleSum", + "fieldName": "deleted", + "name": "deleted" + }, + { + "type": "doubleSum", + "fieldName": "variation", + "name": "variation" + }, + { + "type": "doubleSum", + "fieldName": "delta", + "name": "delta" + }, + { + "type": "hyperUnique", + "fieldName": "unique_users", + "name": "unique_users" + } + ], + "dimension": "page", + "metric": "unique_users", + "threshold": 3, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-01-01T00:00:00.000Z", + "result": [ + { + "added": 1812960.0, + "count": 1697, + "page": "Wikipedia:Administrators'_noticeboard/Incidents", + "delta": 770071.0, + "variation": 2855849.0, + "unique_users": 323.21435881635085, + "deleted": -1042889.0, + "rows": 990 + }, + { + "added": 70162.0, + "count": 967, + "page": "2013", + "delta": 40872.0, + "variation": 99452.0, + "unique_users": 301.08619358578636, + "deleted": -29290.0, + "rows": 773 + }, + { + "added": 519152.0, + "count": 1700, + "page": "Wikipedia:Vandalismusmeldung", + "delta": -5446.0, + "variation": 1043750.0, + "unique_users": 298.7707608914404, + "deleted": -524598.0, + "rows": 991 + } + ] + } + ] + }, + { + "description": "topN, all aggs, page dim, count metric, filtered", + "query": { + "queryType": "topN", + "dataSource": "wikipedia_editstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-08T00:00:00.000"], + "granularity": "all", + "filter": { + "type": "and", + "fields": [ + { + "type": "selector", + "dimension": "namespace", + "value": "article" + }, + { + "type": "or", + "fields": [ + { + "type": "selector", + "dimension": "language", + "value": "en" + }, + { + "type": "selector", + "dimension": "unpatrolled", + "value": "0" + } + ] + } + ] + }, + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + }, + { + "type": "doubleSum", + "fieldName": "added", + "name": "added" + }, + { + "type": "doubleSum", + "fieldName": "deleted", + "name": "deleted" + }, + { + "type": "doubleSum", + "fieldName": "variation", + "name": "variation" + }, + { + "type": "doubleSum", + "fieldName": "delta", + "name": "delta" + }, + { + "type": "hyperUnique", + "fieldName": "unique_users", + "name": "unique_users" + } + ], + "dimension": "page", + "metric": "count", + "threshold": 3, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-01-01T00:00:00.000Z", + "result": [ + { + "added": 61739.0, + "count": 852, + "page": "2013", + "delta": 35313.0, + "variation": 88165.0, + "unique_users": 245.09832757908927, + "deleted": -26426.0, + "rows": 692 + }, + { + "added": 28288.0, + "count": 513, + "page": "Gérard_Depardieu", + "delta": 7027.0, + "variation": 49549.0, + "unique_users": 203.8133555888084, + "deleted": -21261.0, + "rows": 398 + }, + { + "added": 10951.0, + "count": 459, + "page": "Zichyújfalu", + "delta": 9030.0, + "variation": 12872.0, + "unique_users": 13.041435202975777, + "deleted": -1921.0, + "rows": 447 + } + ] + } + ] + }, + { + "description": "topN, all aggs, page dim, count metric, postAggs", + "query": { + "queryType": "topN", + "dataSource": "wikipedia_editstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-08T00:00:00.000"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + }, + { + "type": "doubleSum", + "fieldName": "added", + "name": "added" + }, + { + "type": "doubleSum", + "fieldName": "deleted", + "name": "deleted" + }, + { + "type": "doubleSum", + "fieldName": "variation", + "name": "variation" + }, + { + "type": "doubleSum", + "fieldName": "delta", + "name": "delta" + }, + { + "type": "hyperUnique", + "fieldName": "unique_users", + "name": "unique_users" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "sumOfAddedDeletedConst", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "added", + "fieldName": "added" + }, + { + "type": "arithmetic", + "name": "", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "constant", + "name": "constant", + "value": 1000 + } + ] + } + ] + } + ], + "dimension": "page", + "metric": "count", + "threshold": 3, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-01-01T00:00:00.000Z", + "result": [ + { + "added": 151409.0, + "count": 1770, + "page": "User:Cyde/List_of_candidates_for_speedy_deletion/Subpage", + "delta": 670.0, + "variation": 302148.0, + "unique_users": 1.0002442201269182, + "deleted": -150739.0, + "sumOfAddedDeletedConst": 1670.0, + "rows": 168 + }, + { + "added": 519152.0, + "count": 1700, + "page": "Wikipedia:Vandalismusmeldung", + "delta": -5446.0, + "variation": 1043750.0, + "unique_users": 298.7707608914404, + "deleted": -524598.0, + "sumOfAddedDeletedConst": -4446.0, + "rows": 991 + }, + { + "added": 1812960.0, + "count": 1697, + "page": "Wikipedia:Administrators'_noticeboard/Incidents", + "delta": 770071.0, + "variation": 2855849.0, + "unique_users": 323.21435881635085, + "deleted": -1042889.0, + "sumOfAddedDeletedConst": 771071.0, + "rows": 990 + } + ] + } + ] + }, + { + "description": "topN, lexicographic, two aggs, language dim, postAggs", + "query": { + "queryType": "topN", + "dataSource": "wikipedia_editstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-08T00:00:00.000"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "sumOfRowsAndCount", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "rows", + "fieldName": "rows" + }, + { + "type": "fieldAccess", + "name": "count", + "fieldName": "count" + } + ] + } + ], + "dimension": "language", + "metric": { + "type": "lexicographic", + "previousStop": "a" + }, + "threshold": 3, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-01-01T00:00:00.000Z", + "result": [ + { + "sumOfRowsAndCount": 63046.0, + "count": 33674, + "language": "ar", + "rows": 29372 + }, + { + "sumOfRowsAndCount": 26269.0, + "count": 13652, + "language": "bg", + "rows": 12617 + }, + { + "sumOfRowsAndCount": 60831.0, + "count": 32066, + "language": "ca", + "rows": 28765 + } + ] + } + ] + }, + { + "description": "topN, inverted, two aggs, namespace dim, postAggs", + "query": { + "queryType": "topN", + "dataSource": "wikipedia_editstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-08T00:00:00.000"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "sumOfRowsAndCount", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "rows", + "fieldName": "rows" + }, + { + "type": "fieldAccess", + "name": "count", + "fieldName": "count" + } + ] + } + ], + "dimension": "namespace", + "metric": { + "type": "inverted", + "metric": "count" + }, + "threshold": 3, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-01-01T00:00:00.000Z", + "result": [ + { + "sumOfRowsAndCount": 1553213.0, + "count": 835585, + "namespace": "wikipedia", + "rows": 717628 + }, + { + "sumOfRowsAndCount": 3653387.0, + "count": 1980065, + "namespace": "article", + "rows": 1673322 + } + ] + } + ] + }, + { + "description": "groupBy, two aggs, namespace dim, postAggs", + "query": { + "queryType": "groupBy", + "dataSource": "wikipedia_editstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-08T00:00:00.000"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "sumOfRowsAndCount", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "rows", + "fieldName": "rows" + }, + { + "type": "fieldAccess", + "name": "count", + "fieldName": "count" + } + ] + } + ], + "dimensions": ["namespace"], + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "version": "v1", + "timestamp": "2013-01-01T00:00:00.000Z", + "event": { + "sumOfRowsAndCount": 3653387.0, + "count": 1980065, + "rows": 1673322, + "namespace": "article" + } + }, + { + "version": "v1", + "timestamp": "2013-01-01T00:00:00.000Z", + "event": { + "sumOfRowsAndCount": 1553213.0, + "count": 835585, + "rows": 717628, + "namespace": "wikipedia" + } + } + ] + }, + { + "description": "groupBy, two aggs, namespace + robot dim, postAggs", + "query": { + "queryType": "groupBy", + "dataSource": "wikipedia_editstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-08T00:00:00.000"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "sumOfRowsAndCount", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "rows", + "fieldName": "rows" + }, + { + "type": "fieldAccess", + "name": "count", + "fieldName": "count" + } + ] + } + ], + "dimensions": ["namespace", "robot"], + "limitSpec": { + "type": "default", + "limit": 3, + "orderBy": ["robot", "namespace"] + }, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "version": "v1", + "timestamp": "2013-01-01T00:00:00.000Z", + "event": { + "sumOfRowsAndCount": 2268154.0, + "count": 1286354, + "robot": "0", + "rows": 981800, + "namespace": "article" + } + }, + { + "version": "v1", + "timestamp": "2013-01-01T00:00:00.000Z", + "event": { + "sumOfRowsAndCount": 1385233.0, + "count": 693711, + "robot": "1", + "rows": 691522, + "namespace": "article" + } + }, + { + "version": "v1", + "timestamp": "2013-01-01T00:00:00.000Z", + "event": { + "sumOfRowsAndCount": 878393.0, + "count": 492643, + "robot": "0", + "rows": 385750, + "namespace": "wikipedia" + } + } + ] + }, + { + "query": { + "queryType": "search", + "intervals": ["2013-01-01T00:00:00.000/2013-01-08T00:00:00.000"], + "dataSource": "wikipedia_editstream", + "filter": { + "type": "and", + "fields": [ + { + "type": "selector", + "dimension": "namespace", + "value": "article" + }, + { + "type": "or", + "fields": [ + { + "type": "selector", + "dimension": "language", + "value": "en" + }, + { + "type": "selector", + "dimension": "unpatrolled", + "value": "0" + } + ] + } + ] + }, + "granularity": "all", + "searchDimensions": ["page", "namespace"], + "query": { + "type": "insensitive_contains", + "value": "league_of_legends" + }, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2012-12-29T00:00:00.000Z", + "result": [ + { + "dimension": "page", + "value": "League_of_Legends" + }, + { + "dimension": "page", + "value": "The_best_ADs_in_The_League_of_legends" + } + ] + } + ] + }, + { + "query": { + "queryType": "timeBoundary", + "dataSource": "wikipedia_editstream", + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2012-12-29T00:00:00.000Z", + "result": { + "minTime": "2012-12-29T00:00:00.000Z", + "maxTime": "2013-01-10T07:59:00.000Z" + } + } + ] + }, + { + "description": "segmentMetadata", + "query": { + "queryType": "segmentMetadata", + "dataSource": "wikipedia_editstream", + "intervals": ["2013-01-01T00:00:00.000/2013-01-08T00:00:00.000"], + "toInclude": { + "type": "list", + "columns": ["country_name", "language"] + } + }, + "expectedResults": [ + { + "id": "wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9", + "intervals": ["2012-12-29T00:00:00.000Z/2013-01-10T08:00:00.000Z"], + "columns": { + "country_name": { + "type": "STRING", + "size": 41922148, + "cardinality": 208, + "errorMessage": null + }, + "language": { + "type": "STRING", + "size": 8924222, + "cardinality": 36, + "errorMessage": null + } + }, + "size": 902457341 + } + ] + } +] diff --git a/integration-tests/src/test/resources/testng.xml b/integration-tests/src/test/resources/testng.xml new file mode 100644 index 00000000000..4e2b8f147f4 --- /dev/null +++ b/integration-tests/src/test/resources/testng.xml @@ -0,0 +1,10 @@ + + + + + + + + + + diff --git a/integration-tests/stop_cluster.sh b/integration-tests/stop_cluster.sh new file mode 100755 index 00000000000..b7bd21bac52 --- /dev/null +++ b/integration-tests/stop_cluster.sh @@ -0,0 +1,5 @@ +for node in druid-historical druid-coordinator druid-overlord druid-router druid-broker druid-middlemanager druid-zookeeper druid-metadata-storage; +do +docker stop $node +docker rm $node +done diff --git a/pom.xml b/pom.xml index 2126ba85b65..1c06f81a66d 100644 --- a/pom.xml +++ b/pom.xml @@ -55,6 +55,7 @@ processing server services + integration-tests extensions/cassandra-storage extensions/hdfs-storage @@ -509,6 +510,11 @@ + + org.testng + testng + 6.8.7 + From 9acc8c72ee1e506a37a688378565581da031e298 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 7 Jan 2015 15:07:14 -0800 Subject: [PATCH 66/71] new faster container builds in Travis-CI --- .travis.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.travis.yml b/.travis.yml index 2fd5140efca..f888220552e 100644 --- a/.travis.yml +++ b/.travis.yml @@ -3,3 +3,5 @@ language: java jdk: - oraclejdk7 - oraclejdk8 + +sudo: false From d5f4182de4535a3999bfbfc703530bf4731cea97 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 7 Jan 2015 22:55:53 -0800 Subject: [PATCH 67/71] global test timeouts + fix test race condition --- .../ChainedExecutionQueryRunnerTest.java | 47 ++++++++++--------- 1 file changed, 26 insertions(+), 21 deletions(-) diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index 9ba455eaed8..58d9331f3e2 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -42,11 +42,10 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; public class ChainedExecutionQueryRunnerTest { - @Test + @Test(timeout = 60000) public void testQueryCancellation() throws Exception { ExecutorService exec = PrioritizedExecutorService.create( @@ -124,8 +123,8 @@ public class ChainedExecutionQueryRunnerTest ); // wait for query to register and start - Assert.assertTrue(queryIsRegistered.await(1, TimeUnit.SECONDS)); - Assert.assertTrue(queriesStarted.await(1, TimeUnit.SECONDS)); + queryIsRegistered.await(); + queriesStarted.await(); // cancel the query Assert.assertTrue(capturedFuture.hasCaptured()); @@ -139,14 +138,16 @@ public class ChainedExecutionQueryRunnerTest Assert.assertTrue(e.getCause() instanceof QueryInterruptedException); cause = (QueryInterruptedException)e.getCause(); } - Assert.assertTrue(queriesInterrupted.await(500, TimeUnit.MILLISECONDS)); + queriesInterrupted.await(); Assert.assertNotNull(cause); Assert.assertTrue(future.isCancelled()); Assert.assertTrue(runner1.hasStarted); Assert.assertTrue(runner2.hasStarted); Assert.assertTrue(runner1.interrupted); Assert.assertTrue(runner2.interrupted); - Assert.assertTrue(!runner3.hasStarted || runner3.interrupted); + synchronized (runner3) { + Assert.assertTrue(!runner3.hasStarted || runner3.interrupted); + } Assert.assertFalse(runner1.hasCompleted); Assert.assertFalse(runner2.hasCompleted); Assert.assertFalse(runner3.hasCompleted); @@ -154,7 +155,7 @@ public class ChainedExecutionQueryRunnerTest EasyMock.verify(watcher); } - @Test + @Test(timeout = 60000) public void testQueryTimeout() throws Exception { ExecutorService exec = PrioritizedExecutorService.create( @@ -233,8 +234,8 @@ public class ChainedExecutionQueryRunnerTest ); // wait for query to register and start - Assert.assertTrue(queryIsRegistered.await(1, TimeUnit.SECONDS)); - Assert.assertTrue(queriesStarted.await(1, TimeUnit.SECONDS)); + queryIsRegistered.await(); + queriesStarted.await(); Assert.assertTrue(capturedFuture.hasCaptured()); ListenableFuture future = capturedFuture.getValue(); @@ -248,14 +249,16 @@ public class ChainedExecutionQueryRunnerTest Assert.assertEquals("Query timeout", e.getCause().getMessage()); cause = (QueryInterruptedException)e.getCause(); } - Assert.assertTrue(queriesInterrupted.await(500, TimeUnit.MILLISECONDS)); + queriesInterrupted.await(); Assert.assertNotNull(cause); Assert.assertTrue(future.isCancelled()); Assert.assertTrue(runner1.hasStarted); Assert.assertTrue(runner2.hasStarted); Assert.assertTrue(runner1.interrupted); Assert.assertTrue(runner2.interrupted); - Assert.assertTrue(!runner3.hasStarted || runner3.interrupted); + synchronized (runner3) { + Assert.assertTrue(!runner3.hasStarted || runner3.interrupted); + } Assert.assertFalse(runner1.hasCompleted); Assert.assertFalse(runner2.hasCompleted); Assert.assertFalse(runner3.hasCompleted); @@ -268,9 +271,9 @@ public class ChainedExecutionQueryRunnerTest private final CountDownLatch start; private final CountDownLatch stop; - private boolean hasStarted = false; - private boolean hasCompleted = false; - private boolean interrupted = false; + private volatile boolean hasStarted = false; + private volatile boolean hasCompleted = false; + private volatile boolean interrupted = false; public DyingQueryRunner(CountDownLatch start, CountDownLatch stop) { @@ -281,17 +284,19 @@ public class ChainedExecutionQueryRunnerTest @Override public Sequence run(Query query, Map responseContext) { - hasStarted = true; - start.countDown(); - if (Thread.interrupted()) { - interrupted = true; - stop.countDown(); - throw new QueryInterruptedException("I got killed"); + synchronized (this) { // ensure hasStarted and interrupted are updated simultaneously + hasStarted = true; + start.countDown(); + if (Thread.interrupted()) { + interrupted = true; + stop.countDown(); + throw new QueryInterruptedException("I got killed"); + } } // do a lot of work try { - Thread.sleep(500); + Thread.sleep(5000); } catch (InterruptedException e) { interrupted = true; From 67757b6aeac264b55a506d31f46edf2b2ce06f66 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 8 Jan 2015 09:34:31 -0800 Subject: [PATCH 68/71] Change IndexerZkConfig to use @JacksonInject instead of just straight @Inject * Updated IndexerZkConfig to use no setters, and take all arguments from constructor instead * Also added more unit tests --- .../initialization/IndexerZkConfig.java | 49 +++++++---- .../overlord/RemoteTaskRunnerTest.java | 4 +- .../worker/WorkerTaskMonitorTest.java | 4 +- .../worker/http/WorkerResourceTest.java | 4 +- .../initialization/IndexerZkConfigTest.java | 85 ++++++++++++++----- 5 files changed, 100 insertions(+), 46 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/server/initialization/IndexerZkConfig.java b/indexing-service/src/main/java/io/druid/server/initialization/IndexerZkConfig.java index 6333ea3f0e9..aefa80313e2 100644 --- a/indexing-service/src/main/java/io/druid/server/initialization/IndexerZkConfig.java +++ b/indexing-service/src/main/java/io/druid/server/initialization/IndexerZkConfig.java @@ -19,9 +19,9 @@ package io.druid.server.initialization; -import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.inject.Inject; import org.apache.curator.utils.ZKPaths; /** @@ -29,19 +29,41 @@ import org.apache.curator.utils.ZKPaths; */ public class IndexerZkConfig { - @Inject - @JsonIgnore - private ZkPathsConfig zkPathsConfig = new ZkPathsConfig(); + @JsonCreator + public IndexerZkConfig( + @JacksonInject ZkPathsConfig zkPathsConfig, + @JsonProperty("base") String base, + @JsonProperty("announcementsPath") String announcementsPath, + @JsonProperty("tasksPath") String tasksPath, + @JsonProperty("status") String status, + @JsonProperty("leaderLatchPath") String leaderLatchPath + ) + { + this.zkPathsConfig = zkPathsConfig; + this.base = base; + this.announcementsPath = announcementsPath; + this.tasksPath = tasksPath; + this.status = status; + this.leaderLatchPath = leaderLatchPath; + } + + @JacksonInject + private final ZkPathsConfig zkPathsConfig; + @JsonProperty - private String base; + private final String base; + @JsonProperty - private String announcementsPath; + private final String announcementsPath; + @JsonProperty - private String tasksPath; + private final String tasksPath; + @JsonProperty - private String status; + private final String status; + @JsonProperty - private String leaderLatchPath; + private final String leaderLatchPath; private String defaultIndexerPath(final String subPath) { @@ -77,11 +99,4 @@ public class IndexerZkConfig { return zkPathsConfig; } - - // Setter required for easy debugging - public IndexerZkConfig setZkPathsConfig(ZkPathsConfig zkPathsConfig) - { - this.zkPathsConfig = zkPathsConfig; - return this; - } } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java index 01bfbca8480..63174d9640f 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java @@ -396,14 +396,14 @@ public class RemoteTaskRunnerTest remoteTaskRunner = new RemoteTaskRunner( jsonMapper, config, - new IndexerZkConfig().setZkPathsConfig(new ZkPathsConfig() + new IndexerZkConfig(new ZkPathsConfig() { @Override public String getBase() { return basePath; } - }), + },null,null,null,null,null), cf, new SimplePathChildrenCacheFactory.Builder().build(), null, diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index 76b269b855e..a8b55c8ffd3 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -140,7 +140,7 @@ public class WorkerTaskMonitorTest workerCuratorCoordinator = new WorkerCuratorCoordinator( jsonMapper, - new IndexerZkConfig().setZkPathsConfig( + new IndexerZkConfig( new ZkPathsConfig() { @Override @@ -148,7 +148,7 @@ public class WorkerTaskMonitorTest { return basePath; } - }), + },null,null,null,null,null), new TestRemoteTaskRunnerConfig(), cf, worker diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/http/WorkerResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/http/WorkerResourceTest.java index 2b786532e79..91577521be9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/http/WorkerResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/http/WorkerResourceTest.java @@ -77,14 +77,14 @@ public class WorkerResourceTest curatorCoordinator = new WorkerCuratorCoordinator( jsonMapper, - new IndexerZkConfig().setZkPathsConfig(new ZkPathsConfig() + new IndexerZkConfig(new ZkPathsConfig() { @Override public String getBase() { return basePath; } - }), + },null,null,null,null,null), new RemoteTaskRunnerConfig(), cf, worker diff --git a/indexing-service/src/test/java/io/druid/server/initialization/IndexerZkConfigTest.java b/indexing-service/src/test/java/io/druid/server/initialization/IndexerZkConfigTest.java index d19d7b6490d..26331fabcf3 100644 --- a/indexing-service/src/test/java/io/druid/server/initialization/IndexerZkConfigTest.java +++ b/indexing-service/src/test/java/io/druid/server/initialization/IndexerZkConfigTest.java @@ -34,6 +34,7 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Ignore; import org.junit.Test; import java.lang.reflect.Field; @@ -42,7 +43,9 @@ import java.lang.reflect.Method; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; +import java.util.Properties; import java.util.UUID; /** @@ -52,7 +55,7 @@ public class IndexerZkConfigTest { private static final String indexerPropertyString = "test.druid.zk.paths.indexer"; private static final String zkServiceConfigString = "test.druid.zk.paths"; - private static final Collection clobberableProperties = new ArrayList<>(); + private static final Collection clobberableProperties = new HashSet<>(); private static final Module simpleZkConfigModule = new Module() { @@ -65,13 +68,11 @@ public class IndexerZkConfigTest JsonConfigProvider.bind(binder, indexerPropertyString, IndexerZkConfig.class); JsonConfigProvider.bind( binder, zkServiceConfigString, - CuratorConfig.class + ZkPathsConfig.class ); } }; - private static final Map priorValues = new HashMap<>(); - @BeforeClass public static void setup() { @@ -85,22 +86,9 @@ public class IndexerZkConfigTest clobberableProperties.add(String.format("%s.%s", zkServiceConfigString, field.getName())); } } - for (String clobberableProperty : clobberableProperties) { - priorValues.put(clobberableProperty, System.getProperty(clobberableProperty)); - } } - @AfterClass - public static void cleanup() - { - for (Map.Entry entry : priorValues.entrySet()) { - if (null != entry.getKey() && null != entry.getValue()) { - System.setProperty(entry.getKey(), entry.getValue()); - } - } - } - - private Map propertyValues = new HashMap<>(); + private Properties propertyValues = new Properties(); private int assertions = 0; @Before @@ -109,7 +97,6 @@ public class IndexerZkConfigTest for (String property : clobberableProperties) { propertyValues.put(property, UUID.randomUUID().toString()); } - System.getProperties().putAll(propertyValues); assertions = 0; } @@ -150,6 +137,28 @@ public class IndexerZkConfigTest } } + @Test + public void testNullConfig(){ + propertyValues.clear(); + + final Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + ImmutableList.of(simpleZkConfigModule) + ); + JsonConfigurator configurator = injector.getBinding(JsonConfigurator.class).getProvider().get(); + + JsonConfigProvider zkPathsConfig = JsonConfigProvider.of(zkServiceConfigString, ZkPathsConfig.class); + zkPathsConfig.inject(propertyValues, configurator); + + JsonConfigProvider indexerZkConfig = JsonConfigProvider.of( + indexerPropertyString, + IndexerZkConfig.class + ); + indexerZkConfig.inject(propertyValues, configurator); + + Assert.assertEquals("/druid/indexer/leaderLatchPath", indexerZkConfig.get().get().getLeaderLatchPath()); + } + @Test public void testSimpleConfig() throws IllegalAccessException, NoSuchMethodException, InvocationTargetException { @@ -160,16 +169,46 @@ public class IndexerZkConfigTest JsonConfigurator configurator = injector.getBinding(JsonConfigurator.class).getProvider().get(); JsonConfigProvider zkPathsConfig = JsonConfigProvider.of(zkServiceConfigString, ZkPathsConfig.class); - zkPathsConfig.inject(System.getProperties(), configurator); + zkPathsConfig.inject(propertyValues, configurator); JsonConfigProvider indexerZkConfig = JsonConfigProvider.of( indexerPropertyString, IndexerZkConfig.class ); - indexerZkConfig.inject(System.getProperties(), configurator); + indexerZkConfig.inject(propertyValues, configurator); - validateEntries(indexerZkConfig.get().get()); - validateEntries(zkPathsConfig.get().get()); + + IndexerZkConfig zkConfig = indexerZkConfig.get().get(); + ZkPathsConfig zkPathsConfig1 = zkPathsConfig.get().get(); + + validateEntries(zkConfig); + validateEntries(zkPathsConfig1); Assert.assertEquals(clobberableProperties.size(), assertions); } + + @Test + public void testExactConfig(){ + final Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + ImmutableList.of(simpleZkConfigModule) + ); + propertyValues.setProperty(zkServiceConfigString + ".base", "/druid/metrics"); + + + JsonConfigurator configurator = injector.getBinding(JsonConfigurator.class).getProvider().get(); + + JsonConfigProvider zkPathsConfig = JsonConfigProvider.of( + zkServiceConfigString, + ZkPathsConfig.class + ); + + zkPathsConfig.inject(propertyValues, configurator); + + ZkPathsConfig zkPathsConfig1 = zkPathsConfig.get().get(); + + IndexerZkConfig indexerZkConfig = new IndexerZkConfig(zkPathsConfig1,null,null,null,null,null); + + Assert.assertEquals("indexer", indexerZkConfig.getBase()); + Assert.assertEquals("/druid/metrics/indexer/announcements", indexerZkConfig.getAnnouncementsPath()); + } } From a02ef1dbf12bd926ffe069c868047338e6f0d8e9 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 8 Jan 2015 14:38:10 -0800 Subject: [PATCH 69/71] update batch ingest docs --- docs/content/Batch-ingestion.md | 25 +++++++++++++++++++++---- 1 file changed, 21 insertions(+), 4 deletions(-) diff --git a/docs/content/Batch-ingestion.md b/docs/content/Batch-ingestion.md index 77fe78b9ba8..9288052de12 100644 --- a/docs/content/Batch-ingestion.md +++ b/docs/content/Batch-ingestion.md @@ -88,10 +88,27 @@ The spec\_file is a path to a file that contains JSON and an example looks like: }, "tuningConfig" : { "type" : "hadoop", - "targetPartitionSize" : 5000000, - "jobProperties": { - "mapreduce.job.queuename": "default" - } + "workingPath": "/tmp", + "partitionsSpec" : { + "type" : "dimension", + "partitionDimension" : null, + "targetPartitionSize" : 5000000, + "maxPartitionSize" : 7500000, + "assumeGrouped" : false, + "numShards" : -1 + }, + "shardSpecs" : { }, + "leaveIntermediate" : false, + "cleanupOnFailure" : true, + "overwriteFiles" : false, + "ignoreInvalidRows" : false, + "jobProperties" : { }, + "combineText" : false, + "persistInHeap" : false, + "ingestOffheap" : false, + "bufferSize" : 134217728, + "aggregationBufferRatio" : 0.5, + "rowFlushBoundary" : 300000 } } ``` From 84cc32ba3285740c78d100f1c415393e100877f3 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 8 Jan 2015 13:53:45 -0800 Subject: [PATCH 70/71] Add more logging for when ingestion may be throttled --- .../main/java/io/druid/concurrent/Execs.java | 3 +- .../realtime/FireDepartmentMetrics.java | 43 ++++++++++++++++- .../realtime/RealtimeMetricsMonitor.java | 10 +++- .../realtime/plumber/RealtimePlumber.java | 47 ++++++++++++++----- 4 files changed, 86 insertions(+), 17 deletions(-) diff --git a/common/src/main/java/io/druid/concurrent/Execs.java b/common/src/main/java/io/druid/concurrent/Execs.java index 66af2a196ba..c7f13883719 100644 --- a/common/src/main/java/io/druid/concurrent/Execs.java +++ b/common/src/main/java/io/druid/concurrent/Execs.java @@ -59,7 +59,8 @@ public class Execs /** * @param nameFormat nameformat for threadFactory - * @param capacity maximum capacity after which the executorService will block on accepting new tasks + * @param capacity maximum capacity after which the executorService will block on accepting new tasks + * * @return ExecutorService which blocks accepting new tasks when the capacity reached */ public static ExecutorService newBlockingSingleThreaded(final String nameFormat, final int capacity) diff --git a/server/src/main/java/io/druid/segment/realtime/FireDepartmentMetrics.java b/server/src/main/java/io/druid/segment/realtime/FireDepartmentMetrics.java index 81b1bc9e316..e48cdc389b3 100644 --- a/server/src/main/java/io/druid/segment/realtime/FireDepartmentMetrics.java +++ b/server/src/main/java/io/druid/segment/realtime/FireDepartmentMetrics.java @@ -19,10 +19,10 @@ package io.druid.segment.realtime; -import java.util.concurrent.atomic.AtomicLong; - import com.google.common.base.Preconditions; +import java.util.concurrent.atomic.AtomicLong; + /** */ public class FireDepartmentMetrics @@ -31,6 +31,9 @@ public class FireDepartmentMetrics private final AtomicLong thrownAwayCount = new AtomicLong(0); private final AtomicLong unparseableCount = new AtomicLong(0); private final AtomicLong rowOutputCount = new AtomicLong(0); + private final AtomicLong numPersists = new AtomicLong(0); + private final AtomicLong persistTimeMillis = new AtomicLong(0); + private final AtomicLong persistBackPressureMillis = new AtomicLong(0); public void incrementProcessed() { @@ -52,6 +55,21 @@ public class FireDepartmentMetrics rowOutputCount.addAndGet(numRows); } + public void incrementNumPersists() + { + numPersists.incrementAndGet(); + } + + public void incrementPersistTimeMillis(long millis) + { + persistTimeMillis.addAndGet(millis); + } + + public void incrementPersistBackPressureMillis(long millis) + { + persistBackPressureMillis.addAndGet(millis); + } + public long processed() { return processedCount.get(); @@ -72,6 +90,21 @@ public class FireDepartmentMetrics return rowOutputCount.get(); } + public long numPersists() + { + return numPersists.get(); + } + + public long persistTimeMillis() + { + return persistTimeMillis.get(); + } + + public long persistBackPressureMillis() + { + return persistBackPressureMillis.get(); + } + public FireDepartmentMetrics snapshot() { final FireDepartmentMetrics retVal = new FireDepartmentMetrics(); @@ -79,6 +112,9 @@ public class FireDepartmentMetrics retVal.thrownAwayCount.set(thrownAwayCount.get()); retVal.unparseableCount.set(unparseableCount.get()); retVal.rowOutputCount.set(rowOutputCount.get()); + retVal.numPersists.set(numPersists.get()); + retVal.persistTimeMillis.set(persistTimeMillis.get()); + retVal.persistBackPressureMillis.set(persistBackPressureMillis.get()); return retVal; } @@ -95,6 +131,9 @@ public class FireDepartmentMetrics thrownAwayCount.addAndGet(otherSnapshot.thrownAway()); rowOutputCount.addAndGet(otherSnapshot.rowOutput()); unparseableCount.addAndGet(otherSnapshot.unparseable()); + numPersists.addAndGet(otherSnapshot.numPersists()); + persistTimeMillis.addAndGet(otherSnapshot.persistTimeMillis()); + persistBackPressureMillis.addAndGet(otherSnapshot.persistBackPressureMillis()); return this; } } diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeMetricsMonitor.java b/server/src/main/java/io/druid/segment/realtime/RealtimeMetricsMonitor.java index 649f2b5b157..bf8ce9bd669 100644 --- a/server/src/main/java/io/druid/segment/realtime/RealtimeMetricsMonitor.java +++ b/server/src/main/java/io/druid/segment/realtime/RealtimeMetricsMonitor.java @@ -29,7 +29,7 @@ import java.util.List; import java.util.Map; /** -*/ + */ public class RealtimeMetricsMonitor extends AbstractMonitor { private final Map previousValues; @@ -60,6 +60,14 @@ public class RealtimeMetricsMonitor extends AbstractMonitor emitter.emit(builder.build("events/unparseable", metrics.unparseable() - previous.unparseable())); emitter.emit(builder.build("events/processed", metrics.processed() - previous.processed())); emitter.emit(builder.build("rows/output", metrics.rowOutput() - previous.rowOutput())); + emitter.emit(builder.build("persists/num", metrics.numPersists() - previous.numPersists())); + emitter.emit(builder.build("persists/time", metrics.persistTimeMillis() - previous.persistTimeMillis())); + emitter.emit( + builder.build( + "persists/backPressure", + metrics.persistBackPressureMillis() - previous.persistBackPressureMillis() + ) + ); previousValues.put(fireDepartment, metrics); } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index e64111e98b6..1d163dce09d 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -3,6 +3,7 @@ package io.druid.segment.realtime.plumber; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Predicate; +import com.google.common.base.Stopwatch; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -69,12 +70,15 @@ import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; /** */ public class RealtimePlumber implements Plumber { private static final EmittingLogger log = new EmittingLogger(RealtimePlumber.class); + private static final int WARN_DELAY = 1000; + private final DataSchema schema; private final RealtimeTuningConfig config; private final RejectionPolicy rejectionPolicy; @@ -91,6 +95,7 @@ public class RealtimePlumber implements Plumber private final VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline( String.CASE_INSENSITIVE_ORDER ); + private volatile boolean shuttingDown = false; private volatile boolean stopped = false; private volatile ExecutorService persistExecutor = null; @@ -297,19 +302,35 @@ public class RealtimePlumber implements Plumber log.info("Submitting persist runnable for dataSource[%s]", schema.getDataSource()); + final Stopwatch runExecStopwatch = Stopwatch.createStarted(); + final Stopwatch persistStopwatch = Stopwatch.createStarted(); persistExecutor.execute( new ThreadRenamingRunnable(String.format("%s-incremental-persist", schema.getDataSource())) { @Override public void doRun() { - for (Pair pair : indexesToPersist) { - metrics.incrementRowOutputCount(persistHydrant(pair.lhs, schema, pair.rhs)); + try { + for (Pair pair : indexesToPersist) { + metrics.incrementRowOutputCount(persistHydrant(pair.lhs, schema, pair.rhs)); + } + commitRunnable.run(); + } + finally { + metrics.incrementNumPersists(); + metrics.incrementPersistTimeMillis(persistStopwatch.elapsed(TimeUnit.MILLISECONDS)); + persistStopwatch.stop(); } - commitRunnable.run(); } } ); + + final long startDelay = runExecStopwatch.elapsed(TimeUnit.MILLISECONDS); + metrics.incrementPersistBackPressureMillis(startDelay); + if (startDelay > WARN_DELAY) { + log.warn("Ingestion was throttled for [%,d] millis because persists were pending.", startDelay); + } + runExecStopwatch.stop(); } // Submits persist-n-merge task for a Sink to the mergeExecutor @@ -696,7 +717,7 @@ public class RealtimePlumber implements Plumber * being created. * * @param truncatedTime sink key - * @param sink sink to unannounce + * @param sink sink to unannounce */ protected void abandonSegment(final long truncatedTime, final Sink sink) { @@ -735,8 +756,8 @@ public class RealtimePlumber implements Plumber * Persists the given hydrant and returns the number of rows persisted * * @param indexToPersist hydrant to persist - * @param schema datasource schema - * @param interval interval to persist + * @param schema datasource schema + * @param interval interval to persist * * @return the number of rows persisted */ @@ -845,13 +866,13 @@ public class RealtimePlumber implements Plumber && config.getShardSpec().getPartitionNum() == segment.getShardSpec().getPartitionNum() && Iterables.any( sinks.keySet(), new Predicate() - { - @Override - public boolean apply(Long sinkKey) - { - return segment.getInterval().contains(sinkKey); - } - } + { + @Override + public boolean apply(Long sinkKey) + { + return segment.getInterval().contains(sinkKey); + } + } ); } } From 021a7de759ef9a58a285bd7c7d98b7a737055c51 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 13 Jan 2015 10:42:03 -0800 Subject: [PATCH 71/71] Add retry capabilities for realtime logic --- .../realtime/plumber/RealtimePlumber.java | 30 +++++++++++++++---- 1 file changed, 25 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 1d163dce09d..4f998f7185f 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -12,6 +12,7 @@ import com.google.common.primitives.Ints; import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.metamx.common.Granularity; +import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.guava.FunctionalIterable; @@ -30,6 +31,7 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QueryToolChest; +import io.druid.query.ReportTimelineMissingSegmentQueryRunner; import io.druid.query.SegmentDescriptor; import io.druid.query.spec.SpecificSegmentQueryRunner; import io.druid.query.spec.SpecificSegmentSpec; @@ -243,7 +245,22 @@ public class RealtimePlumber implements Plumber @Override public QueryRunner apply(TimelineObjectHolder holder) { + if (holder == null) { + throw new ISE("No timeline entry at all!"); + } + final Sink theSink = holder.getObject().getChunk(0).getObject(); + + if (theSink == null) { + throw new ISE("Missing sink for timeline entry[%s]!", holder); + } + + final SegmentDescriptor descriptor = new SegmentDescriptor( + holder.getInterval(), + theSink.getSegment().getVersion(), + theSink.getSegment().getShardSpec().getPartitionNum() + ); + return new SpecificSegmentQueryRunner( new MetricsEmittingQueryRunner( emitter, @@ -257,6 +274,13 @@ public class RealtimePlumber implements Plumber @Override public QueryRunner apply(FireHydrant input) { + // It is possible that we got a query for a segment, and while that query + // is in the jetty queue, the segment is abandoned. Here, we need to retry + // the query for the segment. + if (input == null || input.getSegment() == null) { + return new ReportTimelineMissingSegmentQueryRunner(descriptor); + } + // Prevent the underlying segment from closing when its being iterated final Closeable closeable = input.getSegment().increment(); try { @@ -276,11 +300,7 @@ public class RealtimePlumber implements Plumber ) ).withWaitMeasuredFromNow(), new SpecificSegmentSpec( - new SegmentDescriptor( - holder.getInterval(), - theSink.getSegment().getVersion(), - theSink.getSegment().getShardSpec().getPartitionNum() - ) + descriptor ) ); }