From 77c9afe92f601a5e622b88c44dc2f0b258af88c1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 23 Sep 2014 18:53:29 -0700 Subject: [PATCH 01/53] try to make javadoc happy --- .../query/aggregation/histogram/ApproximateHistogram.java | 2 -- .../aggregation/hyperloglog/HyperLogLogCollector.java | 5 ----- .../java/io/druid/query/groupby/having/HavingSpec.java | 2 -- .../segment/data/CompressedFloatsIndexedSupplier.java | 2 -- .../druid/segment/data/CompressedLongsIndexedSupplier.java | 1 - .../io/druid/segment/incremental/IncrementalIndex.java | 7 +++---- .../incremental/IncrementalIndexStorageAdapter.java | 6 ++---- .../java/io/druid/server/coordination/ZkCoordinator.java | 2 +- 8 files changed, 6 insertions(+), 21 deletions(-) diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java index b44fbd524e1..6f481d89161 100644 --- a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java @@ -1019,8 +1019,6 @@ public class ApproximateHistogram * @param count current size of the heap * @param heapIndex index of the item to be deleted * @param values values stored in the heap - * - * @return */ private static int heapDelete(int[] heap, int[] reverseIndex, int count, int heapIndex, float[] values) { diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java index b93c7ace592..7ccbdac2805 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java @@ -200,9 +200,6 @@ public abstract class HyperLogLogCollector implements Comparable /** * For testing. Do not depend on unless you like things breaking. - * - * @return */ GenericIndexed> getBaseFloatBuffers() { diff --git a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java index 8ad267168a5..aa8b9590f54 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java @@ -184,7 +184,6 @@ public class CompressedLongsIndexedSupplier implements Supplier /** * For testing. Do not use unless you like things breaking - * @return */ GenericIndexed> getBaseLongBuffers() { 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 87789e4ceba..da1317f6e4b 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -673,10 +673,9 @@ public class IncrementalIndex implements Iterable falseIdsReverse = biMap.inverse(); } - /** - * Returns the interned String value to allow fast comparisons using `==` instead of `.equals()` - * @see io.druid.segment.incremental.IncrementalIndexStorageAdapter.EntryHolderValueMatcherFactory#makeValueMatcher(String, String) - */ + + // Returns the interned String value to allow fast comparisons using `==` instead of `.equals()` + // see io.druid.segment.incremental.IncrementalIndexStorageAdapter.EntryHolderValueMatcherFactory#makeValueMatcher(String, String) public String get(String value) { return value == null ? null : poorMansInterning.get(value); diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java index 76cf0774861..22d25fa579b 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -532,10 +532,8 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter } for (String dimVal : dims[dimIndex]) { - /** - * using == here instead of .equals() to speed up lookups made possible by - * {@link io.druid.segment.incremental.IncrementalIndex.DimDim#poorMansInterning} - */ + // using == here instead of .equals() to speed up lookups made possible by + // io.druid.segment.incremental.IncrementalIndex.DimDim#poorMansInterning if (id == dimVal) { return true; } diff --git a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java index 34a0ade4574..ba139e8a370 100644 --- a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java @@ -169,7 +169,7 @@ public class ZkCoordinator extends BaseZkCoordinator catch (IOException e) { throw new SegmentLoadingException(e, "Failed to announce segment[%s]", segment.getIdentifier()); } - }; + } } catch (SegmentLoadingException e) { log.makeAlert(e, "Failed to load segment for dataSource") From 1a67d6cb2e678572925c34044dcc2e65e965ccd2 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 1 Oct 2014 15:10:23 -0700 Subject: [PATCH 02/53] some tests for broken approx histo --- ...ApproximateHistogramAggregatorFactory.java | 30 ++ ...mateHistogramFoldingAggregatorFactory.java | 3 +- .../ApproximateHistogramGroupByQueryTest.java | 176 +++++++ ...=> ApproximateHistogramTopNQueryTest.java} | 21 +- .../io/druid/query/groupby/GroupByQuery.java | 18 +- .../query/groupby/GroupByQueryRunnerTest.java | 479 ++++++++---------- .../groupby/GroupByQueryRunnerTestHelper.java | 72 +++ 7 files changed, 528 insertions(+), 271 deletions(-) create mode 100644 histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java rename histogram/src/test/java/io/druid/query/aggregation/histogram/{ApproximateHistogramQueryTest.java => ApproximateHistogramTopNQueryTest.java} (99%) create mode 100644 processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java index 458486423f9..28a707a9598 100644 --- a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java @@ -29,8 +29,14 @@ import com.google.common.primitives.Floats; import com.google.common.primitives.Ints; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.Aggregators; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.hyperloglog.HyperLogLogCollector; +import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregator; +import io.druid.query.aggregation.hyperloglog.HyperUniquesBufferAggregator; import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.FloatColumnSelector; +import io.druid.segment.ObjectColumnSelector; import org.apache.commons.codec.binary.Base64; import java.nio.ByteBuffer; @@ -78,6 +84,18 @@ public class ApproximateHistogramAggregatorFactory implements AggregatorFactory @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { + ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); + + if (selector == null) { + return Aggregators.noopAggregator(); + } + + final Class classOfObject = selector.classOfObject(); + if (classOfObject != null && (classOfObject.equals(Object.class) + || ApproximateHistogramAggregator.class.isAssignableFrom(classOfObject))) { + System.out.println("here"); + //return new HyperUniquesAggregator(name, selector); + } return new ApproximateHistogramAggregator( name, metricFactory.makeFloatColumnSelector(fieldName), @@ -90,6 +108,18 @@ public class ApproximateHistogramAggregatorFactory implements AggregatorFactory @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { + //ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); + // + //if (selector == null) { + // return Aggregators.noopBufferAggregator(); + //} + // + //final Class classOfObject = selector.classOfObject(); + //if (classOfObject != null && (classOfObject.equals(Object.class) || ApproximateHistogramAggregator.class.isAssignableFrom(classOfObject))) { + // System.out.println("here"); + // //return new HyperUniquesAggregator(name, selector); + //} + return new ApproximateHistogramBufferAggregator( metricFactory.makeFloatColumnSelector(fieldName), resolution, diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java index 04dc43a804b..f84b03ae5b8 100644 --- a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java @@ -76,7 +76,8 @@ public class ApproximateHistogramFoldingAggregatorFactory extends ApproximateHis }; } - if (ApproximateHistogram.class.isAssignableFrom(selector.classOfObject())) { + final Class classOfObject = selector.classOfObject(); + if (classOfObject.equals(Object.class) || ApproximateHistogram.class.isAssignableFrom(classOfObject)) { return new ApproximateHistogramFoldingAggregator( name, selector, diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java new file mode 100644 index 00000000000..33ee0b224d9 --- /dev/null +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -0,0 +1,176 @@ +package io.druid.query.aggregation.histogram; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import io.druid.collections.StupidPool; +import io.druid.data.input.Row; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.MaxAggregatorFactory; +import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.dimension.LegacyDimensionSpec; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.groupby.GroupByQueryConfig; +import io.druid.query.groupby.GroupByQueryEngine; +import io.druid.query.groupby.GroupByQueryQueryToolChest; +import io.druid.query.groupby.GroupByQueryRunnerFactory; +import io.druid.query.groupby.GroupByQueryRunnerTestHelper; +import io.druid.segment.TestHelper; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +/** + */ +@RunWith(Parameterized.class) +public class ApproximateHistogramGroupByQueryTest +{ + private final QueryRunner runner; + private GroupByQueryRunnerFactory factory; + private Supplier configSupplier; + + @Before + public void setUp() throws Exception + { + configSupplier = Suppliers.ofInstance(new GroupByQueryConfig()); + } + + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + final ObjectMapper mapper = new DefaultObjectMapper(); + final StupidPool pool = new StupidPool( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(1024 * 1024); + } + } + ); + + final GroupByQueryConfig config = new GroupByQueryConfig(); + config.setMaxIntermediateRows(10000); + + final Supplier configSupplier = Suppliers.ofInstance(config); + final GroupByQueryEngine engine = new GroupByQueryEngine(configSupplier, pool); + + final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( + engine, + QueryRunnerTestHelper.NOOP_QUERYWATCHER, + configSupplier, + new GroupByQueryQueryToolChest(configSupplier, mapper, engine) + ); + + GroupByQueryConfig singleThreadedConfig = new GroupByQueryConfig() + { + @Override + public boolean isSingleThreaded() + { + return true; + } + }; + singleThreadedConfig.setMaxIntermediateRows(10000); + + final Supplier singleThreadedConfigSupplier = Suppliers.ofInstance(singleThreadedConfig); + final GroupByQueryEngine singleThreadEngine = new GroupByQueryEngine(singleThreadedConfigSupplier, pool); + + final GroupByQueryRunnerFactory singleThreadFactory = new GroupByQueryRunnerFactory( + singleThreadEngine, + QueryRunnerTestHelper.NOOP_QUERYWATCHER, + singleThreadedConfigSupplier, + new GroupByQueryQueryToolChest(singleThreadedConfigSupplier, mapper, singleThreadEngine) + ); + + + Function function = new Function() + { + @Override + public Object apply(@Nullable Object input) + { + return new Object[]{factory, ((Object[]) input)[0]}; + } + }; + + return Lists.newArrayList( + Iterables.concat( + Iterables.transform( + QueryRunnerTestHelper.makeQueryRunners(factory), + function + ), + Iterables.transform( + QueryRunnerTestHelper.makeQueryRunners(singleThreadFactory), + function + ) + ) + ); + } + + public ApproximateHistogramGroupByQueryTest(GroupByQueryRunnerFactory factory, QueryRunner runner) + { + this.factory = factory; + this.runner = runner; + } + + @Test + public void testGroupByNWithApproximateHistogramAgg() + { + ApproximateHistogramAggregatorFactory aggFactory = new ApproximateHistogramAggregatorFactory( + "apphisto", + "index", + 10, + 5, + Float.NEGATIVE_INFINITY, + Float.POSITIVE_INFINITY + ); + + GroupByQuery query = new GroupByQuery.Builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setGranularity(QueryRunnerTestHelper.allGran) + .setDimensions(Arrays.asList(new LegacyDimensionSpec(QueryRunnerTestHelper.providerDimension))) + .setInterval(QueryRunnerTestHelper.fullOnInterval) + .setAggregatorSpecs( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index"), + aggFactory + ) + ) + ) + ) + .setPostAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.addRowsIndexConstant, + QueryRunnerTestHelper.dependentPostAgg, + new QuantilePostAggregator("quantile", "apphisto", 0.5f) + ) + ) + .build(); + + List expectedResults = Arrays.asList( + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, "approx-histo"); + } +} diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java similarity index 99% rename from histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramQueryTest.java rename to histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java index dd4406c5076..e362b4ad9b2 100644 --- a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramQueryTest.java +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java @@ -52,18 +52,8 @@ import java.util.List; import java.util.Map; @RunWith(Parameterized.class) -public class ApproximateHistogramQueryTest +public class ApproximateHistogramTopNQueryTest { - - private final QueryRunner runner; - - public ApproximateHistogramQueryTest( - QueryRunner runner - ) - { - this.runner = runner; - } - @Parameterized.Parameters public static Collection constructorFeeder() throws IOException { @@ -99,6 +89,15 @@ public class ApproximateHistogramQueryTest return retVal; } + private final QueryRunner runner; + + public ApproximateHistogramTopNQueryTest( + QueryRunner runner + ) + { + this.runner = runner; + } + @Test public void testTopNWithApproximateHistogramAgg() { diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index a0fab78da07..eedc90946f3 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -50,6 +50,7 @@ import io.druid.query.groupby.orderby.NoopLimitSpec; import io.druid.query.groupby.orderby.OrderByColumnSpec; import io.druid.query.spec.LegacySegmentSpec; import io.druid.query.spec.QuerySegmentSpec; +import org.joda.time.Interval; import java.util.List; import java.util.Map; @@ -344,7 +345,22 @@ public class GroupByQuery extends BaseQuery return this; } - public Builder setInterval(Object interval) + public Builder setInterval(QuerySegmentSpec interval) + { + return setQuerySegmentSpec(interval); + } + + public Builder setInterval(List intervals) + { + return setQuerySegmentSpec(new LegacySegmentSpec(intervals)); + } + + public Builder setInterval(Interval interval) + { + return setQuerySegmentSpec(new LegacySegmentSpec(interval)); + } + + public Builder setInterval(String interval) { return setQuerySegmentSpec(new LegacySegmentSpec(interval)); } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 9eb25885432..6beab76793d 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -21,27 +21,22 @@ package io.druid.query.groupby; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; -import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.collections.StupidPool; -import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.granularity.PeriodGranularity; import io.druid.granularity.QueryGranularity; import io.druid.jackson.DefaultObjectMapper; -import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; -import io.druid.query.QueryToolChest; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.JavaScriptAggregatorFactory; @@ -84,7 +79,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.Comparator; import java.util.List; -import java.util.Map; @RunWith(Parameterized.class) public class GroupByQueryRunnerTest @@ -195,28 +189,28 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), - createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), - createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -237,7 +231,7 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow( + GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", "rows", 26L, @@ -246,7 +240,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -267,7 +261,7 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow( + GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", "rows", 26L, @@ -276,7 +270,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -306,26 +300,26 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "a", "rows", 1L, "idx", 135L), - createExpectedRow("2011-04-01", "alias", "b", "rows", 1L, "idx", 118L), - createExpectedRow("2011-04-01", "alias", "e", "rows", 1L, "idx", 158L), - createExpectedRow("2011-04-01", "alias", "h", "rows", 1L, "idx", 120L), - createExpectedRow("2011-04-01", "alias", "m", "rows", 3L, "idx", 2870L), - createExpectedRow("2011-04-01", "alias", "n", "rows", 1L, "idx", 121L), - createExpectedRow("2011-04-01", "alias", "p", "rows", 3L, "idx", 2900L), - createExpectedRow("2011-04-01", "alias", "t", "rows", 2L, "idx", 197L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "a", "rows", 1L, "idx", 135L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "b", "rows", 1L, "idx", 118L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "e", "rows", 1L, "idx", 158L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "h", "rows", 1L, "idx", 120L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "m", "rows", 3L, "idx", 2870L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "n", "rows", 1L, "idx", 121L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "p", "rows", 3L, "idx", 2900L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "t", "rows", 2L, "idx", 197L), - createExpectedRow("2011-04-02", "alias", "a", "rows", 1L, "idx", 147L), - createExpectedRow("2011-04-02", "alias", "b", "rows", 1L, "idx", 112L), - createExpectedRow("2011-04-02", "alias", "e", "rows", 1L, "idx", 166L), - createExpectedRow("2011-04-02", "alias", "h", "rows", 1L, "idx", 113L), - createExpectedRow("2011-04-02", "alias", "m", "rows", 3L, "idx", 2447L), - createExpectedRow("2011-04-02", "alias", "n", "rows", 1L, "idx", 114L), - createExpectedRow("2011-04-02", "alias", "p", "rows", 3L, "idx", 2505L), - createExpectedRow("2011-04-02", "alias", "t", "rows", 2L, "idx", 223L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "a", "rows", 1L, "idx", 147L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "b", "rows", 1L, "idx", 112L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "e", "rows", 1L, "idx", 166L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "h", "rows", 1L, "idx", 113L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "m", "rows", 3L, "idx", 2447L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "n", "rows", 1L, "idx", 114L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "p", "rows", 3L, "idx", 2505L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "t", "rows", 2L, "idx", 223L) ); - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -364,28 +358,28 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "automotive", "rows", 1L, "idx", 135L), - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "business", "rows", 1L, "idx", 118L), - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "entertainment", "rows", 1L, "idx", 158L), - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "health", "rows", 1L, "idx", 120L), - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "mezzanine", "rows", 3L, "idx", 2870L), - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "news", "rows", 1L, "idx", 121L), - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "premium", "rows", 3L, "idx", 2900L), - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "technology", "rows", 1L, "idx", 78L), - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "travel", "rows", 1L, "idx", 119L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "automotive", "rows", 1L, "idx", 135L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "business", "rows", 1L, "idx", 118L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "entertainment", "rows", 1L, "idx", 158L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "health", "rows", 1L, "idx", 120L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "mezzanine", "rows", 3L, "idx", 2870L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "news", "rows", 1L, "idx", 121L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "premium", "rows", 3L, "idx", 2900L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "technology", "rows", 1L, "idx", 78L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "travel", "rows", 1L, "idx", 119L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "automotive", "rows", 1L, "idx", 147L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "business", "rows", 1L, "idx", 112L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "entertainment", "rows", 1L, "idx", 166L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "health", "rows", 1L, "idx", 113L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "mezzanine", "rows", 3L, "idx", 2447L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "news", "rows", 1L, "idx", 114L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "premium", "rows", 3L, "idx", 2505L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "technology", "rows", 1L, "idx", 97L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "travel", "rows", 1L, "idx", 126L) + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "automotive", "rows", 1L, "idx", 147L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "business", "rows", 1L, "idx", 112L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "entertainment", "rows", 1L, "idx", 166L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "health", "rows", 1L, "idx", 113L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "mezzanine", "rows", 3L, "idx", 2447L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "news", "rows", 1L, "idx", 114L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "premium", "rows", 3L, "idx", 2505L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "technology", "rows", 1L, "idx", 97L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "travel", "rows", 1L, "idx", 126L) ); - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -427,30 +421,30 @@ public class GroupByQueryRunnerTest ); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), - createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), - createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) ); TestHelper.assertExpectedObjects(expectedResults, runner.run(fullQuery), "direct"); TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery), "merged"); List allGranExpectedResults = Arrays.asList( - createExpectedRow("2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L), - createExpectedRow("2011-04-02", "alias", "business", "rows", 2L, "idx", 217L), - createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 2L, "idx", 319L), - createExpectedRow("2011-04-02", "alias", "health", "rows", 2L, "idx", 216L), - createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - createExpectedRow("2011-04-02", "alias", "news", "rows", 2L, "idx", 221L), - createExpectedRow("2011-04-02", "alias", "premium", "rows", 6L, "idx", 4416L), - createExpectedRow("2011-04-02", "alias", "technology", "rows", 2L, "idx", 177L), - createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 2L, "idx", 217L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 2L, "idx", 319L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 2L, "idx", 216L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 2L, "idx", 221L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 6L, "idx", 4416L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 2L, "idx", 177L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) ); TestHelper.assertExpectedObjects(allGranExpectedResults, runner.run(allGranQuery), "direct"); @@ -484,15 +478,15 @@ public class GroupByQueryRunnerTest final GroupByQuery fullQuery = builder.build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), - createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), - createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); @@ -559,15 +553,15 @@ public class GroupByQueryRunnerTest }; List allResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), - createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), - createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) ); List> expectedResults = Lists.newArrayList( @@ -642,15 +636,15 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), - createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), - createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), - createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L) ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); @@ -682,15 +676,15 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), - createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), - createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), - createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L) ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); @@ -721,15 +715,15 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4423.6533203125D), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4418.61865234375D), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319.94403076171875D), - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 270.3977966308594D), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243.65843200683594D), - createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 222.20980834960938D), - createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 218.7224884033203D), - createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216.97836303710938D), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 178.24917602539062D) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4423.6533203125D), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4418.61865234375D), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319.94403076171875D), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 270.3977966308594D), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243.65843200683594D), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 222.20980834960938D), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 218.7224884033203D), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216.97836303710938D), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 178.24917602539062D) ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); @@ -743,9 +737,9 @@ public class GroupByQueryRunnerTest public void testHavingSpec() { List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L) ); GroupByQuery.Builder builder = GroupByQuery @@ -811,7 +805,7 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L) ); final GroupByQueryEngine engine = new GroupByQueryEngine( @@ -855,15 +849,15 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "business", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "entertainment", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "health", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "mezzanine", "rows", 6L), - createExpectedRow("2011-04-01", "quality", "news", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "premium", "rows", 6L), - createExpectedRow("2011-04-01", "quality", "technology", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "business", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "entertainment", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "health", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "mezzanine", "rows", 6L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "news", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "premium", "rows", 6L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "technology", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L) ); TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); @@ -908,15 +902,15 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "business", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "entertainment", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "health", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "mezzanine", "rows", 6L), - createExpectedRow("2011-04-01", "quality", "news", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "premium", "rows", 6L), - createExpectedRow("2011-04-01", "quality", "technology", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "business", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "entertainment", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "health", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "mezzanine", "rows", 6L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "news", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "premium", "rows", 6L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "technology", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L) ); TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); @@ -976,29 +970,29 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), - createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), - createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); // Subqueries are handled by the ToolChest - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -1032,11 +1026,11 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "idx", 2900.0), - createExpectedRow("2011-04-02", "idx", 2505.0) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "idx", 2900.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "idx", 2505.0) ); - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -1070,10 +1064,10 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-02", "idx", 2505.0) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "idx", 2505.0) ); - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -1106,7 +1100,7 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); Assert.assertFalse(results.iterator().hasNext()); } @@ -1165,29 +1159,29 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), - createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), - createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 13870.0), - createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 13900.0), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 13870.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 13900.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0), - createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), - createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), - createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), - createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), - createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), - createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), - createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0), - createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0), - createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0) ); // Subqueries are handled by the ToolChest - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -1265,27 +1259,27 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), - createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), - createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), - createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0), - createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), - createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), - createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), - createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), - createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), - createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), - createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0), - createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0), - createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0) ); // Subqueries are handled by the ToolChest - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -1382,7 +1376,7 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow( + GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", "alias", "travel", @@ -1393,7 +1387,7 @@ public class GroupByQueryRunnerTest "js_outer_agg", 123.92274475097656 ), - createExpectedRow( + GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", "alias", "technology", @@ -1404,7 +1398,7 @@ public class GroupByQueryRunnerTest "js_outer_agg", 82.62254333496094 ), - createExpectedRow( + GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", "alias", "news", @@ -1415,7 +1409,7 @@ public class GroupByQueryRunnerTest "js_outer_agg", 125.58358001708984 ), - createExpectedRow( + GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", "alias", "health", @@ -1426,7 +1420,7 @@ public class GroupByQueryRunnerTest "js_outer_agg", 124.13470458984375 ), - createExpectedRow( + GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", "alias", "entertainment", @@ -1440,7 +1434,7 @@ public class GroupByQueryRunnerTest ); // Subqueries are handled by the ToolChest - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -1478,50 +1472,19 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 282L, "uniq", 1.0002442201269182), - createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 230L, "uniq", 1.0002442201269182), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 324L, "uniq", 1.0002442201269182), - createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 233L, "uniq", 1.0002442201269182), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 5317L, "uniq", 1.0002442201269182), - createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 235L, "uniq", 1.0002442201269182), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 5405L, "uniq", 1.0002442201269182), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 175L, "uniq", 1.0002442201269182), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 245L, "uniq", 1.0002442201269182) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 282L, "uniq", 1.0002442201269182), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 230L, "uniq", 1.0002442201269182), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 324L, "uniq", 1.0002442201269182), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 233L, "uniq", 1.0002442201269182), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 5317L, "uniq", 1.0002442201269182), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 235L, "uniq", 1.0002442201269182), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 5405L, "uniq", 1.0002442201269182), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 175L, "uniq", 1.0002442201269182), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 245L, "uniq", 1.0002442201269182) ); // Subqueries are handled by the ToolChest - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } - - private Iterable runQuery(GroupByQuery query) - { - - QueryToolChest toolChest = factory.getToolchest(); - QueryRunner theRunner = new FinalizeResultsQueryRunner<>( - toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)), - toolChest - ); - - Sequence queryResult = theRunner.run(query); - return Sequences.toList(queryResult, Lists.newArrayList()); - } - - private Row createExpectedRow(final String timestamp, Object... vals) - { - return createExpectedRow(new DateTime(timestamp), vals); - } - - private Row createExpectedRow(final DateTime timestamp, Object... vals) - { - Preconditions.checkArgument(vals.length % 2 == 0); - - Map theVals = Maps.newHashMap(); - for (int i = 0; i < vals.length; i += 2) { - theVals.put(vals[i].toString(), vals[i + 1]); - } - - DateTime ts = new DateTime(timestamp); - return new MapBasedRow(ts, theVals); - } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java new file mode 100644 index 00000000000..74dc5091a9d --- /dev/null +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java @@ -0,0 +1,72 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.groupby; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import io.druid.data.input.MapBasedRow; +import io.druid.data.input.Row; +import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; +import org.joda.time.DateTime; + +import java.util.Map; + +/** + */ +public class GroupByQueryRunnerTestHelper +{ + public static Iterable runQuery(QueryRunnerFactory factory, QueryRunner runner, GroupByQuery query) + { + + QueryToolChest toolChest = factory.getToolchest(); + QueryRunner theRunner = new FinalizeResultsQueryRunner<>( + toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)), + toolChest + ); + + Sequence queryResult = theRunner.run(query); + return Sequences.toList(queryResult, Lists.newArrayList()); + } + + public static Row createExpectedRow(final String timestamp, Object... vals) + { + return createExpectedRow(new DateTime(timestamp), vals); + } + + public static Row createExpectedRow(final DateTime timestamp, Object... vals) + { + Preconditions.checkArgument(vals.length % 2 == 0); + + Map theVals = Maps.newHashMap(); + for (int i = 0; i < vals.length; i += 2) { + theVals.put(vals[i].toString(), vals[i + 1]); + } + + DateTime ts = new DateTime(timestamp); + return new MapBasedRow(ts, theVals); + } + +} From 81641d78876c9795edf2ca30ca2e706fd371e8c5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 6 Oct 2014 12:47:16 -0700 Subject: [PATCH 03/53] http accessible while loading + status endpoint --- .../coordination/BaseZkCoordinator.java | 6 +++- .../druid/server/http/HistoricalResource.java | 32 +++++++++++++++++++ .../main/java/io/druid/cli/CliHistorical.java | 7 ++-- 3 files changed, 42 insertions(+), 3 deletions(-) create mode 100644 server/src/main/java/io/druid/server/http/HistoricalResource.java 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 f31cfb6311c..43776480829 100644 --- a/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java @@ -37,7 +37,6 @@ import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; import org.apache.curator.utils.ZKPaths; import java.io.IOException; -import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; /** @@ -212,6 +211,11 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler } } + public boolean isStarted() + { + return started; + } + public abstract void loadLocalCache(); public abstract DataSegmentChangeHandler getDataSegmentChangeHandler(); diff --git a/server/src/main/java/io/druid/server/http/HistoricalResource.java b/server/src/main/java/io/druid/server/http/HistoricalResource.java new file mode 100644 index 00000000000..a4f5956a0cc --- /dev/null +++ b/server/src/main/java/io/druid/server/http/HistoricalResource.java @@ -0,0 +1,32 @@ +package io.druid.server.http; + +import com.google.common.collect.ImmutableMap; +import io.druid.server.coordination.ZkCoordinator; + +import javax.inject.Inject; +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Response; + +@Path("/druid/historical/v1") +public class HistoricalResource +{ + private final ZkCoordinator coordinator; + + @Inject + public HistoricalResource( + ZkCoordinator coordinator + ) + { + this.coordinator = coordinator; + } + + @GET + @Path("/loadstatus") + @Produces("application/json") + public Response getLoadStatus() + { + return Response.ok(ImmutableMap.of("cacheInitialized", coordinator.isStarted())).build(); + } +} diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 6d4152b9226..2fda1a6dc75 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -38,6 +38,7 @@ import io.druid.query.QuerySegmentWalker; import io.druid.server.QueryResource; import io.druid.server.coordination.ServerManager; import io.druid.server.coordination.ZkCoordinator; +import io.druid.server.http.HistoricalResource; import io.druid.server.initialization.JettyServerInitializer; import io.druid.server.metrics.MetricsModule; import org.eclipse.jetty.server.Server; @@ -68,6 +69,8 @@ public class CliHistorical extends ServerRunnable @Override public void configure(Binder binder) { + // register Server before binding ZkCoordinator to ensure HTTP endpoints are available immediately + LifecycleModule.register(binder, Server.class); binder.bind(ServerManager.class).in(LazySingleton.class); binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); binder.bind(QuerySegmentWalker.class).to(ServerManager.class).in(LazySingleton.class); @@ -75,10 +78,10 @@ public class CliHistorical extends ServerRunnable binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig("historical")); binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class).in(LazySingleton.class); Jerseys.addResource(binder, QueryResource.class); + Jerseys.addResource(binder, HistoricalResource.class); LifecycleModule.register(binder, QueryResource.class); - + LifecycleModule.register(binder, HistoricalResource.class); LifecycleModule.register(binder, ZkCoordinator.class); - LifecycleModule.register(binder, Server.class); binder.bind(Cache.class).toProvider(CacheProvider.class).in(ManageLifecycle.class); JsonConfigProvider.bind(binder, "druid.historical.cache", CacheProvider.class); From 546fca2d67e56909b8eb4e0365a5cc5249733f6d Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 6 Oct 2014 14:18:25 -0700 Subject: [PATCH 04/53] docs for router node --- docs/content/Router.md | 121 +++++++++++++++++++++++++++++++++++++++++ 1 file changed, 121 insertions(+) create mode 100644 docs/content/Router.md diff --git a/docs/content/Router.md b/docs/content/Router.md new file mode 100644 index 00000000000..1bb923e3e91 --- /dev/null +++ b/docs/content/Router.md @@ -0,0 +1,121 @@ +--- +layout: doc_page +--- + +Router Node +=========== + +You should only ever need the router node if you have a Druid cluster well into the terabyte range. The router node can be used to route queries to different broker instances. By default, the broker routes queries based on how [Rules](Rules.html) are set up. For example, if 1 month of recent data is loaded into a `hot` cluster, queries that fall within the recent month can be routed to a dedicated set of brokers. Queries outside this range are routed to another set of brokers. This set up provides query isolation such that queries for more important data are not impacted by queries for less important data. + +Running +------- + +``` +io.druid.cli.Main server historical +``` + +Example Production Configuration +-------------------------------- + +In this example, we have two tiers in our production cluster: `hot` and `_default_tier`. Queries for the `hot` tier are routed through the `broker-hot` set of brokers, and queries for the `_default_tier` are routed through the `broker-cold` set of brokers. If any exceptions or network problems occur, queries are routed to the `broker-cold` set of brokers. In our example, we are running with a c3.2xlarge EC2 node. + +JVM settings: + +``` +-server +-server +-Xmx13g +-Xms13g +-XX:NewSize=256m +-XX:MaxNewSize=256m +-XX:+UseConcMarkSweepGC +-XX:+PrintGCDetails +-XX:+PrintGCTimeStamps +-XX:+UseLargePages +-XX:+HeapDumpOnOutOfMemoryError +-XX:HeapDumpPath=/mnt/galaxy/deploy/current/ +-Duser.timezone=UTC +-Dfile.encoding=UTF-8 +-Djava.io.tmpdir=/mnt/tmp + +-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/router + +druid.extensions.remoteRepositories=[] +druid.extensions.localRepository=lib +druid.extensions.coordinates=["io.druid.extensions:druid-histogram:0.6.156"] + +druid.zk.service.host=#{ZK_IPs} +druid.zk.paths.base=/druid/prod + +druid.discovery.curator.path=/prod/discovery + +druid.processing.numThreads=1 +druid.router.defaultBrokerServiceName=druid:prod:broker-cold +druid.router.coordinatorServiceName=druid:prod:coordinator +druid.router.tierToBrokerMap={"hot":"druid:prod:broker-hot","_default_tier":"druid:prod:broker-cold"} +druid.router.http.numConnections=50 +druid.router.http.readTimeout=PT5M + +druid.server.http.numThreads=100 + +druid.request.logging.type=emitter +druid.request.logging.feed=druid_requests + +druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor"] + +druid.emitter=http +druid.emitter.http.recipientBaseUrl=#{URL} + +druid.curator.compress=true +``` + +Runtime Configuration +--------------------- + +The router module uses several of the default modules in [Configuration](Configuration.html) and has the following set of configurations as well: + +|Property|Possible Values|Description|Default| +|--------|---------------|-----------|-------| +|`druid.router.defaultBrokerServiceName`|Any string.|The default broker to connect to in case service discovery fails.|""| +|`druid.router.tierToBrokerMap`|An ordered JSON map of tiers to broker names. The priority of brokers is based on the ordering.|Queries for a certain tier of data are routed to their appropriate broker.|`highestPriority`| +|`druid.router.defaultRule`|Any string.|The default rule for all datasources.|"_default"| +|`druid.router.rulesEndpoint`|Any string.|The coordinator endpoint to extract rules from.|"/druid/coordinator/v1/rules"| +|`druid.router.coordinatorServiceName`|Any string.|The service discovery name of the coordinator.|null. Must be set.| +|`druid.router.pollPeriod`|Any ISO8601 duration.|How often to poll for new rules.|PT1M| +|`druid.router.strategies`|An ordered JSON array of objects.|All custom strategies to use for routing.|[{"type":"timeBoundary"},{"type":"priority"}]| + +Router Strategies +----------------- +The router has a configurable list of strategies for how it select which broker to route queries to. The order of the strategies matter because as soon as something matches a condition, a broker is selected. + +### timeBoundary + +```json +{ + "type":"timeBoundary" +} +``` + +Including this strategy means all timeBoundary queries are always routed to the highest priority broker. + +### priority + +```json +{ + "type":"priority", + "minPriority":0, + "maxPriority":1 +} +``` + +Queries with a priority set < minPriority are routed to the lowest priority broker. Queries with priority > maxPriority are routed to the highest priority broker. minPriority default is 0, maxPriority default is 1. Using default strategy values, if a query with priority 0 (the default) is sent, it skips the priority selection logic. From ee0e8560297e94997ee7cba9ba512d62d7f11328 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 6 Oct 2014 14:23:23 -0700 Subject: [PATCH 05/53] correct mistakes --- docs/content/Router.md | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/docs/content/Router.md b/docs/content/Router.md index 1bb923e3e91..e3775047335 100644 --- a/docs/content/Router.md +++ b/docs/content/Router.md @@ -5,13 +5,13 @@ layout: doc_page Router Node =========== -You should only ever need the router node if you have a Druid cluster well into the terabyte range. The router node can be used to route queries to different broker instances. By default, the broker routes queries based on how [Rules](Rules.html) are set up. For example, if 1 month of recent data is loaded into a `hot` cluster, queries that fall within the recent month can be routed to a dedicated set of brokers. Queries outside this range are routed to another set of brokers. This set up provides query isolation such that queries for more important data are not impacted by queries for less important data. +You should only ever need the router node if you have a Druid cluster well into the terabyte range. The router node can be used to route queries to different broker nodes. By default, the broker routes queries based on how [Rules](Rules.html) are set up. For example, if 1 month of recent data is loaded into a `hot` cluster, queries that fall within the recent month can be routed to a dedicated set of brokers. Queries outside this range are routed to another set of brokers. This set up provides query isolation such that queries for more important data are not impacted by queries for less important data. Running ------- ``` -io.druid.cli.Main server historical +io.druid.cli.Main server router ``` Example Production Configuration @@ -23,7 +23,6 @@ JVM settings: ``` -server --server -Xmx13g -Xms13g -XX:NewSize=256m @@ -96,7 +95,7 @@ The router module uses several of the default modules in [Configuration](Configu Router Strategies ----------------- -The router has a configurable list of strategies for how it select which broker to route queries to. The order of the strategies matter because as soon as something matches a condition, a broker is selected. +The router has a configurable list of strategies for how it selects which brokers to route queries to. The order of the strategies matter because as soon as a strategy condition is matched, a broker is selected. ### timeBoundary @@ -118,4 +117,4 @@ Including this strategy means all timeBoundary queries are always routed to the } ``` -Queries with a priority set < minPriority are routed to the lowest priority broker. Queries with priority > maxPriority are routed to the highest priority broker. minPriority default is 0, maxPriority default is 1. Using default strategy values, if a query with priority 0 (the default) is sent, it skips the priority selection logic. +Queries with a priority set to less than minPriority are routed to the lowest priority broker. Queries with priority set to greater than maxPriority are routed to the highest priority broker. By default, minPriority is 0 and maxPriority is 1. Using these default values, if a query with priority 0 (the default query priority is 0) is sent, the query skips the priority selection logic. From f78b8f2c3109549cfcd74158741498f03f09f99d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 6 Oct 2014 17:38:13 -0700 Subject: [PATCH 06/53] fix ApproximateHistogram combining factory --- ...ApproximateHistogramAggregatorFactory.java | 26 +------ ...mateHistogramFoldingAggregatorFactory.java | 11 +-- .../aggregation/histogram/Histogram.java | 11 ++- .../ApproximateHistogramGroupByQueryTest.java | 69 +++++++++++++++++-- 4 files changed, 81 insertions(+), 36 deletions(-) diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java index 28a707a9598..c704c72ba2f 100644 --- a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java @@ -84,18 +84,6 @@ public class ApproximateHistogramAggregatorFactory implements AggregatorFactory @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { - ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); - - if (selector == null) { - return Aggregators.noopAggregator(); - } - - final Class classOfObject = selector.classOfObject(); - if (classOfObject != null && (classOfObject.equals(Object.class) - || ApproximateHistogramAggregator.class.isAssignableFrom(classOfObject))) { - System.out.println("here"); - //return new HyperUniquesAggregator(name, selector); - } return new ApproximateHistogramAggregator( name, metricFactory.makeFloatColumnSelector(fieldName), @@ -108,18 +96,6 @@ public class ApproximateHistogramAggregatorFactory implements AggregatorFactory @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { - //ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); - // - //if (selector == null) { - // return Aggregators.noopBufferAggregator(); - //} - // - //final Class classOfObject = selector.classOfObject(); - //if (classOfObject != null && (classOfObject.equals(Object.class) || ApproximateHistogramAggregator.class.isAssignableFrom(classOfObject))) { - // System.out.println("here"); - // //return new HyperUniquesAggregator(name, selector); - //} - return new ApproximateHistogramBufferAggregator( metricFactory.makeFloatColumnSelector(fieldName), resolution, @@ -143,7 +119,7 @@ public class ApproximateHistogramAggregatorFactory implements AggregatorFactory @Override public AggregatorFactory getCombiningFactory() { - return new ApproximateHistogramAggregatorFactory(name, name, resolution, numBuckets, lowerLimit, upperLimit); + return new ApproximateHistogramFoldingAggregatorFactory(name, name, resolution, numBuckets, lowerLimit, upperLimit); } @Override diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java index f84b03ae5b8..50a9e6ba973 100644 --- a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java @@ -76,8 +76,8 @@ public class ApproximateHistogramFoldingAggregatorFactory extends ApproximateHis }; } - final Class classOfObject = selector.classOfObject(); - if (classOfObject.equals(Object.class) || ApproximateHistogram.class.isAssignableFrom(classOfObject)) { + final Class cls = selector.classOfObject(); + if (cls.equals(Object.class) || ApproximateHistogram.class.isAssignableFrom(cls)) { return new ApproximateHistogramFoldingAggregator( name, selector, @@ -90,7 +90,7 @@ public class ApproximateHistogramFoldingAggregatorFactory extends ApproximateHis throw new IAE( "Incompatible type for metric[%s], expected a ApproximateHistogram, got a %s", fieldName, - selector.classOfObject() + cls ); } @@ -118,14 +118,15 @@ public class ApproximateHistogramFoldingAggregatorFactory extends ApproximateHis }; } - if (ApproximateHistogram.class.isAssignableFrom(selector.classOfObject())) { + final Class cls = selector.classOfObject(); + if (cls.equals(Object.class) || ApproximateHistogram.class.isAssignableFrom(cls)) { return new ApproximateHistogramFoldingBufferAggregator(selector, resolution, lowerLimit, upperLimit); } throw new IAE( "Incompatible type for metric[%s], expected a ApproximateHistogram, got a %s", fieldName, - selector.classOfObject() + cls ); } diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/Histogram.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/Histogram.java index 384e6eb1fbf..6785ceb9c10 100644 --- a/histogram/src/main/java/io/druid/query/aggregation/histogram/Histogram.java +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/Histogram.java @@ -84,5 +84,12 @@ public class Histogram return result; } - -} \ No newline at end of file + @Override + public String toString() + { + return "Histogram{" + + "breaks=" + Arrays.toString(breaks) + + ", counts=" + Arrays.toString(counts) + + '}'; + } +} diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index 33ee0b224d9..cd7aa5a70c1 100644 --- a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -1,3 +1,22 @@ +/* + * 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.query.aggregation.histogram; import com.fasterxml.jackson.databind.ObjectMapper; @@ -23,6 +42,8 @@ import io.druid.query.groupby.GroupByQueryEngine; import io.druid.query.groupby.GroupByQueryQueryToolChest; import io.druid.query.groupby.GroupByQueryRunnerFactory; import io.druid.query.groupby.GroupByQueryRunnerTestHelper; +import io.druid.query.groupby.orderby.DefaultLimitSpec; +import io.druid.query.groupby.orderby.OrderByColumnSpec; import io.druid.segment.TestHelper; import org.junit.Before; import org.junit.Test; @@ -130,7 +151,7 @@ public class ApproximateHistogramGroupByQueryTest } @Test - public void testGroupByNWithApproximateHistogramAgg() + public void testGroupByWithApproximateHistogramAgg() { ApproximateHistogramAggregatorFactory aggFactory = new ApproximateHistogramAggregatorFactory( "apphisto", @@ -146,8 +167,18 @@ public class ApproximateHistogramGroupByQueryTest .setGranularity(QueryRunnerTestHelper.allGran) .setDimensions(Arrays.asList(new LegacyDimensionSpec(QueryRunnerTestHelper.providerDimension))) .setInterval(QueryRunnerTestHelper.fullOnInterval) + .setLimitSpec( + new DefaultLimitSpec( + Lists.newArrayList( + new OrderByColumnSpec( + QueryRunnerTestHelper.providerDimension, + OrderByColumnSpec.Direction.DESCENDING + ) + ), 1 + ) + ) .setAggregatorSpecs( - Lists.newArrayList( + Lists.newArrayList( Iterables.concat( QueryRunnerTestHelper.commonAggregators, Lists.newArrayList( @@ -159,7 +190,7 @@ public class ApproximateHistogramGroupByQueryTest ) ) .setPostAggregatorSpecs( - Arrays.asList( + Arrays.asList( QueryRunnerTestHelper.addRowsIndexConstant, QueryRunnerTestHelper.dependentPostAgg, new QuantilePostAggregator("quantile", "apphisto", 0.5f) @@ -167,7 +198,37 @@ public class ApproximateHistogramGroupByQueryTest ) .build(); - List expectedResults = Arrays.asList( + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "1970-01-01T00:00:00.000Z", + "provider", "spot", + "rows", 837L, + "addRowsIndexConstant", 96444.5703125, + "dependentPostAgg", 97282.5703125, + "index", 95606.5703125, + "maxIndex", 277.2735290527344, + "minIndex", 59.02102279663086, + "quantile", 101.78856f, + "uniques", QueryRunnerTestHelper.UNIQUES_9, + "apphisto", + new Histogram( + new float[]{ + 4.457897186279297f, + 59.02102279663086f, + 113.58415222167969f, + 168.14727783203125f, + 222.7104034423828f, + 277.2735290527344f + }, + new double[]{ + 0.0, + 462.4309997558594, + 357.5404968261719, + 15.022850036621094, + 2.0056631565093994 + } + ) + ) ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); From f1d37d569c4736a9f0664b30869f0ca05e2b7004 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 7 Oct 2014 13:01:41 -0700 Subject: [PATCH 07/53] doc fixes --- docs/content/Router.md | 4 ++-- docs/content/toc.textile | 1 + pom.xml | 4 ++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/docs/content/Router.md b/docs/content/Router.md index e3775047335..ce8033836cf 100644 --- a/docs/content/Router.md +++ b/docs/content/Router.md @@ -85,8 +85,8 @@ The router module uses several of the default modules in [Configuration](Configu |Property|Possible Values|Description|Default| |--------|---------------|-----------|-------| -|`druid.router.defaultBrokerServiceName`|Any string.|The default broker to connect to in case service discovery fails.|""| -|`druid.router.tierToBrokerMap`|An ordered JSON map of tiers to broker names. The priority of brokers is based on the ordering.|Queries for a certain tier of data are routed to their appropriate broker.|`highestPriority`| +|`druid.router.defaultBrokerServiceName`|Any string.|The default broker to connect to in case service discovery fails.|"". Must be set.| +|`druid.router.tierToBrokerMap`|An ordered JSON map of tiers to broker names. The priority of brokers is based on the ordering.|Queries for a certain tier of data are routed to their appropriate broker.|{"_default": ""}| |`druid.router.defaultRule`|Any string.|The default rule for all datasources.|"_default"| |`druid.router.rulesEndpoint`|Any string.|The coordinator endpoint to extract rules from.|"/druid/coordinator/v1/rules"| |`druid.router.coordinatorServiceName`|Any string.|The service discovery name of the coordinator.|null. Must be set.| diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 437e383b760..d811e68b55a 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -84,6 +84,7 @@ h2. Experimental * "Geographic Queries":./GeographicQueries.html * "Select Query":./SelectQuery.html * "Approximate Histograms and Quantiles":./ApproxHisto.html +* "Router node":./Router.html h2. Development * "Versioning":./Versioning.html diff --git a/pom.xml b/pom.xml index 1ad6ccb9338..60304fd173c 100644 --- a/pom.xml +++ b/pom.xml @@ -39,9 +39,9 @@ UTF-8 - 0.26.7 + 0.26.8 2.6.0 - 0.2.10 + 0.2.11 From 30183d3b4dafbdd65adf6a0741f77a53023db7b9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 7 Oct 2014 12:40:12 -0700 Subject: [PATCH 08/53] update druid-api for json parsing improvements --- pom.xml | 2 +- .../java/io/druid/data/input/ProtoBufInputRowParser.java | 3 ++- .../io/druid/data/input/ProtoBufInputRowParserTest.java | 3 ++- .../java/io/druid/segment/realtime/FireDepartmentTest.java | 3 ++- .../segment/realtime/plumber/RealtimePlumberSchoolTest.java | 6 +++++- 5 files changed, 12 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index 60304fd173c..54158c11c82 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.8 2.6.0 - 0.2.11 + 0.2.12 diff --git a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java index df8bdb5cef8..4a449efbe82 100644 --- a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java +++ b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java @@ -69,7 +69,8 @@ public class ProtoBufInputRowParser implements ByteBufferInputRowParser if (parseSpec == null) { this.parseSpec = new JSONParseSpec( timestampSpec, - new DimensionsSpec(dimensions, dimensionExclusions, spatialDimensions) + new DimensionsSpec(dimensions, dimensionExclusions, spatialDimensions), + JSONParseSpec.JSON ); } else { this.parseSpec = parseSpec; diff --git a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java index 67d5fdc3574..570045daab0 100644 --- a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java +++ b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java @@ -59,7 +59,8 @@ public class ProtoBufInputRowParserTest ProtoBufInputRowParser parser = new ProtoBufInputRowParser( new JSONParseSpec( new TimestampSpec("timestamp", "iso"), - new DimensionsSpec(Arrays.asList(DIMENSIONS), Arrays.asList(), null) + new DimensionsSpec(Arrays.asList(DIMENSIONS), Arrays.asList(), null), + JSONParseSpec.JSON ), "prototest.desc", null, null, null, null diff --git a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java index 5fca643fc08..d212342abdf 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -61,7 +61,8 @@ public class FireDepartmentTest Arrays.asList("dim1", "dim2"), null, null - ) + ), + JSONParseSpec.JSON ), null, null, null, null ), 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 e610d3c53e7..ff4808f6815 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 @@ -123,7 +123,11 @@ public class RealtimePlumberSchoolTest @Override public ParseSpec getParseSpec() { - return new JSONParseSpec(new TimestampSpec("timestamp", "auto"), new DimensionsSpec(null, null, null)); + return new JSONParseSpec( + new TimestampSpec("timestamp", "auto"), + new DimensionsSpec(null, null, null), + JSONParseSpec.JSON + ); } @Override From 19dc32d314fb714aad4b53c24f6b66c93f8efeae Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 7 Oct 2014 13:50:32 -0700 Subject: [PATCH 09/53] add headers for druid vs pages --- docs/content/Druid-vs-Cassandra.md | 5 +++++ docs/content/Druid-vs-Hadoop.md | 4 ++++ docs/content/Druid-vs-Impala-or-Shark.md | 4 ++++ docs/content/Druid-vs-Redshift.md | 4 ++++ docs/content/Druid-vs-Vertica.md | 5 +++++ 5 files changed, 22 insertions(+) diff --git a/docs/content/Druid-vs-Cassandra.md b/docs/content/Druid-vs-Cassandra.md index 4ef29a1ab36..f8d1d59a473 100644 --- a/docs/content/Druid-vs-Cassandra.md +++ b/docs/content/Druid-vs-Cassandra.md @@ -1,6 +1,11 @@ --- layout: doc_page --- + +Druid vs. Cassandra +=================== + + We are not experts on Cassandra, if anything is incorrect about our portrayal, please let us know on the mailing list or via some other means. We will fix this page. Druid is highly optimized for scans and aggregations, it supports arbitrarily deep drill downs into data sets without the need to pre-compute, and it can ingest event streams in real-time and allow users to query events as they come in. Cassandra is a great key-value store and it has some features that allow you to use it to do more interesting things than what you can do with a pure key-value store. But, it is not built for the same use cases that Druid handles, namely regularly scanning over billions of entries per query. diff --git a/docs/content/Druid-vs-Hadoop.md b/docs/content/Druid-vs-Hadoop.md index 3b4ef1df76c..47d03dd7704 100644 --- a/docs/content/Druid-vs-Hadoop.md +++ b/docs/content/Druid-vs-Hadoop.md @@ -2,6 +2,10 @@ layout: doc_page --- +Druid vs Hadoop +=============== + + Hadoop has shown the world that it’s possible to house your data warehouse on commodity hardware for a fraction of the price of typical solutions. As people adopt Hadoop for their data warehousing needs, they find two things. 1. They can now query all of their data in a fairly flexible manner and answer any question they have diff --git a/docs/content/Druid-vs-Impala-or-Shark.md b/docs/content/Druid-vs-Impala-or-Shark.md index cb658c8e087..ab2cb122524 100644 --- a/docs/content/Druid-vs-Impala-or-Shark.md +++ b/docs/content/Druid-vs-Impala-or-Shark.md @@ -1,6 +1,10 @@ --- layout: doc_page --- + +Druid vs Impala or Shark +======================== + The question of Druid versus Impala or Shark basically comes down to your product requirements and what the systems were designed to do. Druid was designed to diff --git a/docs/content/Druid-vs-Redshift.md b/docs/content/Druid-vs-Redshift.md index 4fe06586467..faaaa3f0f33 100644 --- a/docs/content/Druid-vs-Redshift.md +++ b/docs/content/Druid-vs-Redshift.md @@ -1,6 +1,10 @@ --- layout: doc_page --- +Druid vs Redshift +================= + + ###How does Druid compare to Redshift? In terms of drawing a differentiation, Redshift is essentially ParAccel (Actian) which Amazon is licensing. diff --git a/docs/content/Druid-vs-Vertica.md b/docs/content/Druid-vs-Vertica.md index e6971ae03d9..4fe0eb78892 100644 --- a/docs/content/Druid-vs-Vertica.md +++ b/docs/content/Druid-vs-Vertica.md @@ -1,6 +1,11 @@ --- layout: doc_page --- + +Druid vs Vertica +================ + + How does Druid compare to Vertica? Vertica is similar to ParAccel/Redshift ([Druid-vs-Redshift](Druid-vs-Redshift.html)) described above in that it wasn’t built for real-time streaming data ingestion and it supports full SQL. From 2f21b2754cb005e2ce25cbc77c6bb65ef06be0ce Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 7 Oct 2014 13:57:03 -0700 Subject: [PATCH 10/53] fix compile --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 54158c11c82..f79f015c209 100644 --- a/pom.xml +++ b/pom.xml @@ -39,7 +39,7 @@ UTF-8 - 0.26.8 + 0.26.9 2.6.0 0.2.12 From 8f7f497b55b3223229ab478eab41ede9d63a2691 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 7 Oct 2014 14:19:59 -0700 Subject: [PATCH 11/53] jax-rs smile support --- pom.xml | 5 +++++ server/pom.xml | 4 ++++ 2 files changed, 9 insertions(+) diff --git a/pom.xml b/pom.xml index f79f015c209..6dfdc2d8508 100644 --- a/pom.xml +++ b/pom.xml @@ -256,6 +256,11 @@ jackson-jaxrs-json-provider 2.2.3 + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-smile-provider + 2.2.3 + org.codehaus.jackson jackson-core-asl diff --git a/server/pom.xml b/server/pom.xml index 7382aefb237..a8208e048eb 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -77,6 +77,10 @@ com.fasterxml.jackson.jaxrs jackson-jaxrs-json-provider + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-smile-provider + com.fasterxml.jackson.dataformat jackson-dataformat-smile From 1ce67dcfc473f182b0cf4f34556d4da3938a57e2 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 7 Oct 2014 14:27:47 -0700 Subject: [PATCH 12/53] prepare for next release --- docs/content/Examples.md | 4 ++-- docs/content/Kafka-Eight.md | 4 ++-- docs/content/Production-Cluster-Configuration.md | 6 +++--- docs/content/Realtime-Config.md | 4 ++-- docs/content/Router.md | 2 +- docs/content/Simple-Cluster-Configuration.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-1.md | 2 +- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.md | 2 +- 11 files changed, 20 insertions(+), 20 deletions(-) diff --git a/docs/content/Examples.md b/docs/content/Examples.md index 468bdb8f41a..049218da290 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.156 +git checkout druid-0.6.157 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.156-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Kafka-Eight.md b/docs/content/Kafka-Eight.md index d8b3c80232e..c167aea59ba 100644 --- a/docs/content/Kafka-Eight.md +++ b/docs/content/Kafka-Eight.md @@ -8,9 +8,9 @@ The previous examples are for Kafka 7. To support Kafka 8, a couple changes need - Update realtime node's configs for Kafka 8 extensions - e.g. - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.156",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.157",...]` - becomes - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.156",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.157",...]` - Update realtime task config for changed keys - `firehose.type`, `plumber.rejectionPolicyFactory`, and all of `firehose.consumerProps` changes. diff --git a/docs/content/Production-Cluster-Configuration.md b/docs/content/Production-Cluster-Configuration.md index d8ed57ab048..0cf81020510 100644 --- a/docs/content/Production-Cluster-Configuration.md +++ b/docs/content/Production-Cluster-Configuration.md @@ -57,7 +57,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/overlord -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -139,7 +139,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/middlemanager -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.156","io.druid.extensions:druid-kafka-seven:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157","io.druid.extensions:druid-kafka-seven:0.6.157"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -286,7 +286,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/historical -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Realtime-Config.md b/docs/content/Realtime-Config.md index 6cb566fac16..cf5104d550d 100644 --- a/docs/content/Realtime-Config.md +++ b/docs/content/Realtime-Config.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.157"] druid.zk.service.host=localhost @@ -76,7 +76,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/realtime -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.156","io.druid.extensions:druid-kafka-seven:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157","io.druid.extensions:druid-kafka-seven:0.6.157"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Router.md b/docs/content/Router.md index ce8033836cf..b6638fdadd2 100644 --- a/docs/content/Router.md +++ b/docs/content/Router.md @@ -51,7 +51,7 @@ druid.service=druid/prod/router druid.extensions.remoteRepositories=[] druid.extensions.localRepository=lib -druid.extensions.coordinates=["io.druid.extensions:druid-histogram:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-histogram:0.6.157"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Simple-Cluster-Configuration.md b/docs/content/Simple-Cluster-Configuration.md index 788f2d65af8..e873895f097 100644 --- a/docs/content/Simple-Cluster-Configuration.md +++ b/docs/content/Simple-Cluster-Configuration.md @@ -28,7 +28,7 @@ Configuration: -Ddruid.zk.service.host=localhost --Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.156"] +-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.157"] -Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid -Ddruid.db.connector.user=druid diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index a0c89c0fafb..95f53508448 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.156-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.156 +cd druid-services-0.6.157 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md index 399af3cc786..12a5ea405af 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md @@ -91,7 +91,7 @@ druid.service=overlord druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.157"] druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid druid.db.connector.user=druid diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 13483109143..5673a9f8f1f 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes and external depende If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.156-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -240,7 +240,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.156","io.druid.extensions:druid-kafka-seven:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.157","io.druid.extensions:druid-kafka-seven:0.6.157"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index d28dc27923c..83dfd848924 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.156-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.156 +cd druid-services-0.6.157 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.md b/docs/content/Twitter-Tutorial.md index db0eab430c6..c8f2b2adc0e 100644 --- a/docs/content/Twitter-Tutorial.md +++ b/docs/content/Twitter-Tutorial.md @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. # Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.156-bin.tar.gz). +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz). Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: From 96e29c2fdd4cd1b0231e934908c86a46c9ec087c Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 7 Oct 2014 14:30:20 -0700 Subject: [PATCH 13/53] [maven-release-plugin] prepare release druid-0.6.157 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index e07536e0705..8354210a1bd 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/common/pom.xml b/common/pom.xml index 687d67afe92..0a5c46c4206 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/examples/pom.xml b/examples/pom.xml index ed6bfaadae4..10f3b185d70 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 2ab577dd4ef..8bc6c573262 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/histogram/pom.xml b/histogram/pom.xml index e2420e915b3..7c69040227e 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 32336247e92..60d62150584 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index a398851837c..9d5f9bfae1f 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 2251c71c8cd..701c62cb5b0 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 9f4274211d0..1c3277c51f0 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/pom.xml b/pom.xml index 6dfdc2d8508..01c560ad54f 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.157-SNAPSHOT + 0.6.157 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 - druid-0.6.131-SNAPSHOT + druid-0.6.157 diff --git a/processing/pom.xml b/processing/pom.xml index c00837b2d91..c0f51992ad5 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 2c0590533f9..316fa93c232 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index a512b649ecd..6595ceeb79b 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/server/pom.xml b/server/pom.xml index a8208e048eb..c3dac395c5e 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/services/pom.xml b/services/pom.xml index cfccaaed8de..7f88ab779ff 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 From 8a4b32c482a4821bf82453e67adfdfd88140650a Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 7 Oct 2014 14:30:26 -0700 Subject: [PATCH 14/53] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 8354210a1bd..f976ec19663 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 0a5c46c4206..62a4c7efd0d 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 10f3b185d70..e4cf96e065e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 8bc6c573262..2ef16b75cfe 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index 7c69040227e..dd73d1a7ea3 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 60d62150584..1268759137d 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 9d5f9bfae1f..8031019b137 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 701c62cb5b0..3a30b97b1eb 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 1c3277c51f0..502517995c3 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/pom.xml b/pom.xml index 01c560ad54f..03244fb8736 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.157 + 0.6.158-SNAPSHOT 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 - druid-0.6.157 + druid-0.6.131-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index c0f51992ad5..a6f09b1f608 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 316fa93c232..291e333eab5 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 6595ceeb79b..eeda5228469 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index c3dac395c5e..0fcf50829bd 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 7f88ab779ff..bd4d5d7a6d2 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT From c25203e1409ed2a2474fd267259e85e1a67e7942 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 7 Oct 2014 14:48:49 -0700 Subject: [PATCH 15/53] remove unused jsonspec content type --- pom.xml | 2 +- .../main/java/io/druid/data/input/ProtoBufInputRowParser.java | 3 +-- .../java/io/druid/data/input/ProtoBufInputRowParserTest.java | 3 +-- .../java/io/druid/segment/realtime/FireDepartmentTest.java | 3 +-- .../segment/realtime/plumber/RealtimePlumberSchoolTest.java | 3 +-- 5 files changed, 5 insertions(+), 9 deletions(-) diff --git a/pom.xml b/pom.xml index 6dfdc2d8508..f14cedd7440 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.9 2.6.0 - 0.2.12 + 0.2.13 diff --git a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java index 4a449efbe82..df8bdb5cef8 100644 --- a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java +++ b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java @@ -69,8 +69,7 @@ public class ProtoBufInputRowParser implements ByteBufferInputRowParser if (parseSpec == null) { this.parseSpec = new JSONParseSpec( timestampSpec, - new DimensionsSpec(dimensions, dimensionExclusions, spatialDimensions), - JSONParseSpec.JSON + new DimensionsSpec(dimensions, dimensionExclusions, spatialDimensions) ); } else { this.parseSpec = parseSpec; diff --git a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java index 570045daab0..67d5fdc3574 100644 --- a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java +++ b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java @@ -59,8 +59,7 @@ public class ProtoBufInputRowParserTest ProtoBufInputRowParser parser = new ProtoBufInputRowParser( new JSONParseSpec( new TimestampSpec("timestamp", "iso"), - new DimensionsSpec(Arrays.asList(DIMENSIONS), Arrays.asList(), null), - JSONParseSpec.JSON + new DimensionsSpec(Arrays.asList(DIMENSIONS), Arrays.asList(), null) ), "prototest.desc", null, null, null, null diff --git a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java index d212342abdf..5fca643fc08 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -61,8 +61,7 @@ public class FireDepartmentTest Arrays.asList("dim1", "dim2"), null, null - ), - JSONParseSpec.JSON + ) ), null, null, null, null ), 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 ff4808f6815..3a9cdd3ebe6 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 @@ -125,8 +125,7 @@ public class RealtimePlumberSchoolTest { return new JSONParseSpec( new TimestampSpec("timestamp", "auto"), - new DimensionsSpec(null, null, null), - JSONParseSpec.JSON + new DimensionsSpec(null, null, null) ); } From 8a34a55c0bb6bd9e8e107e9ef35b44972ef1edb3 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 8 Oct 2014 10:51:00 -0700 Subject: [PATCH 16/53] add example hadoop config --- docs/content/Hadoop-Configuration.md | 360 +++++++++++++++++++++++++++ docs/content/toc.textile | 1 + 2 files changed, 361 insertions(+) create mode 100644 docs/content/Hadoop-Configuration.md diff --git a/docs/content/Hadoop-Configuration.md b/docs/content/Hadoop-Configuration.md new file mode 100644 index 00000000000..296558fcee7 --- /dev/null +++ b/docs/content/Hadoop-Configuration.md @@ -0,0 +1,360 @@ +--- +layout: doc_page +--- + +Example Production Hadoop Configuration +======================================= + +The following configuration should work relatively well for Druid indexing and Hadoop. In the example, we are using EC2 cc2.8xlarge nodes. + +### Core-site.xml + +``` + + + + + hadoop.tmp.dir + /mnt/persistent/hadoop + + + + + fs.defaultFS + hdfs://#{IP}:9000 + + + fs.s3.impl + org.apache.hadoop.fs.s3native.NativeS3FileSystem + + + fs.s3.awsAccessKeyId + #{S3_ACCESS_KEY} + + + fs.s3.awsSecretAccessKey + #{S3_SECRET_KEY} + + + fs.s3.buffer.dir + /mnt/persistent/hadoop-s3n + + + fs.s3n.awsAccessKeyId + #{S3N_ACCESS_KEY} + + + fs.s3n.awsSecretAccessKey + #{S3N_SECRET_KEY} + + + + + io.compression.codecs + org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.Lz4Codec,org.apache.hadoop.io.compress.BZip2Codec,org.apache.hadoop.io.compress.SnappyCodec + + + + + io.seqfile.local.dir + /mnt/persistent/hadoop/io/local + + + +``` + +### Mapred-site.xml + +``` + + + + mapreduce.framework.name + yarn + + + + mapreduce.jobtracker.address + #{JT_ADDR}:9001 + + + mapreduce.jobtracker.http.address + #{JT_HTTP_ADDR}:9100 + + + mapreduce.jobhistory.address + #{JH_ADDR}:10020 + + + mapreduce.jobhistory.webapp.address + #{JH_WEBAPP_ADDR}:19888 + + + mapreduce.tasktracker.http.address + #{TT_ADDR}:9103 + + + + + mapreduce.job.reduces + 21 + + + + mapreduce.job.jvm.numtasks + 20 + + + mapreduce.map.memory.mb + 2048 + + + mapreduce.map.java.opts + -server -Xmx1536m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps + + + mapreduce.reduce.memory.mb + 6144 + + + mapreduce.reduce.java.opts + -server -Xmx2560m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps + + + mapreduce.reduce.shuffle.parallelcopies + 50 + + + mapreduce.reduce.shuffle.input.buffer.percent + 0.5 + + + mapreduce.task.io.sort.mb + 256 + + + mapreduce.task.io.sort.factor + 100 + + + mapreduce.jobtracker.handler.count + 64 + + + mapreduce.tasktracker.http.threads + 20 + + + + + mapreduce.cluster.local.dir + /mnt/persistent/hadoop/mapred/local + + + + + mapreduce.jobhistory.recovery.enable + true + + + mapreduce.jobhistory.recovery.store.class + org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService + + + mapreduce.jobhistory.recovery.store.fs.uri + file://${hadoop.tmp.dir}/mapred-jobhistory-state + + + + + + mapreduce.output.fileoutputformat.compress + false + + + mapreduce.map.output.compress + true + + + mapreduce.output.fileoutputformat.compress.type + BLOCK + + + mapreduce.map.output.compress.codec + org.apache.hadoop.io.compress.Lz4Codec + + + mapreduce.output.fileoutputformat.compress.codec + org.apache.hadoop.io.compress.GzipCodec + + + + mapreduce.map.speculative + false + + + mapreduce.reduce.speculative + false + + + + + mapreduce.task.timeout + 1800000 + + + +``` + +### Yarn-site.xml + +``` + + + + yarn.resourcemanager.hostname + #{RM_HOSTNAME} + + + yarn.resourcemanager.scheduler.class + org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler + + + yarn.nodemanager.aux-services + mapreduce_shuffle + + + yarn.log-aggregation-enable + true + + + yarn.log.server.url + http://#{IP_LOG_SERVER}:19888/jobhistory/logs/ + + + yarn.nodemanager.hostname + #{IP_ADDR} + + + yarn.scheduler.minimum-allocation-mb + 512 + + + yarn.nodemanager.resource.memory-mb + 1024 + + + yarn.nodemanager.resource.cpu-vcores + 1 + + + yarn.nodemanager.vmem-check-enabled + false + + + + + yarn.nodemanager.local-dirs + /mnt/persistent/hadoop/nm-local-dir + + + + + yarn.resourcemanager.recovery.enabled + false + + + yarn.resourcemanager.store.class + org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore + + + yarn.resourcemanager.fs.state-store.uri + file://${hadoop.tmp.dir}/yarn-resourcemanager-state + + + + + yarn.resourcemanager.nodes.exclude-path + /mnt/persistent/hadoop/yarn-exclude.txt + + + +``` + +### HDFS-site.xml + +``` + + + + dfs.replication + 3 + + + dfs.namenode.datanode.registration.ip-hostname-check + false + + + dfs.hosts.exclude + /mnt/persistent/hadoop/hdfs-exclude.txt + + + + + dfs.datanode.data.dir + file:///mnt/persistent/hadoop/dfs/data + + + +``` + +### Capacity-scheduler.xml + +``` + + + + yarn.scheduler.capacity.maximum-am-resource-percent + 0.1 + + + yarn.scheduler.capacity.root.queues + default + + + yarn.scheduler.capacity.root.default.capacity + 100 + + + yarn.scheduler.capacity.root.default.user-limit-factor + 1 + + + yarn.scheduler.capacity.root.queues + default + + + yarn.scheduler.capacity.root.default.maximum-capacity + 100 + + + yarn.scheduler.capacity.root.default.state + RUNNING + + + yarn.scheduler.capacity.root.default.acl_submit_applications + * + + + yarn.scheduler.capacity.root.default.acl_administer_queue + * + + + yarn.scheduler.capacity.node-locality-delay + -1 + + + +``` \ No newline at end of file diff --git a/docs/content/toc.textile b/docs/content/toc.textile index d811e68b55a..cc3c86eac99 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -17,6 +17,7 @@ h2. Getting Started h2. Booting a Druid Cluster * "Simple Cluster Configuration":Simple-Cluster-Configuration.html * "Production Cluster Configuration":Production-Cluster-Configuration.html +* "Production Hadoop Configuration":Hadoop-Configuration.html * "Rolling Cluster Updates":Rolling-Updates.html h2. Configuration From 21ac3f2aab1eb7b83533daf9201f0bc2ab0078c3 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 8 Oct 2014 10:56:00 -0700 Subject: [PATCH 17/53] slight clarification about hadoop setup --- docs/content/Hadoop-Configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Hadoop-Configuration.md b/docs/content/Hadoop-Configuration.md index 296558fcee7..ef8c48e167a 100644 --- a/docs/content/Hadoop-Configuration.md +++ b/docs/content/Hadoop-Configuration.md @@ -5,7 +5,7 @@ layout: doc_page Example Production Hadoop Configuration ======================================= -The following configuration should work relatively well for Druid indexing and Hadoop. In the example, we are using EC2 cc2.8xlarge nodes. +The following configuration should work relatively well for Druid indexing and Hadoop. In the example, we are using Hadoop 2.4 with EC2 m1.xlarge nodes for NameNodes and cc2.8xlarge nodes for DataNodes. ### Core-site.xml From 290107ce7b8d1d3401e26c44b860d61b78094d0c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 7 Oct 2014 14:48:49 -0700 Subject: [PATCH 18/53] remove unused jsonspec content type --- pom.xml | 2 +- .../main/java/io/druid/data/input/ProtoBufInputRowParser.java | 3 +-- .../java/io/druid/data/input/ProtoBufInputRowParserTest.java | 3 +-- .../java/io/druid/segment/realtime/FireDepartmentTest.java | 3 +-- .../segment/realtime/plumber/RealtimePlumberSchoolTest.java | 3 +-- 5 files changed, 5 insertions(+), 9 deletions(-) diff --git a/pom.xml b/pom.xml index 03244fb8736..5731b98d72a 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.9 2.6.0 - 0.2.12 + 0.2.13 diff --git a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java index 4a449efbe82..df8bdb5cef8 100644 --- a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java +++ b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java @@ -69,8 +69,7 @@ public class ProtoBufInputRowParser implements ByteBufferInputRowParser if (parseSpec == null) { this.parseSpec = new JSONParseSpec( timestampSpec, - new DimensionsSpec(dimensions, dimensionExclusions, spatialDimensions), - JSONParseSpec.JSON + new DimensionsSpec(dimensions, dimensionExclusions, spatialDimensions) ); } else { this.parseSpec = parseSpec; diff --git a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java index 570045daab0..67d5fdc3574 100644 --- a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java +++ b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java @@ -59,8 +59,7 @@ public class ProtoBufInputRowParserTest ProtoBufInputRowParser parser = new ProtoBufInputRowParser( new JSONParseSpec( new TimestampSpec("timestamp", "iso"), - new DimensionsSpec(Arrays.asList(DIMENSIONS), Arrays.asList(), null), - JSONParseSpec.JSON + new DimensionsSpec(Arrays.asList(DIMENSIONS), Arrays.asList(), null) ), "prototest.desc", null, null, null, null diff --git a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java index d212342abdf..5fca643fc08 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -61,8 +61,7 @@ public class FireDepartmentTest Arrays.asList("dim1", "dim2"), null, null - ), - JSONParseSpec.JSON + ) ), null, null, null, null ), 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 ff4808f6815..3a9cdd3ebe6 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 @@ -125,8 +125,7 @@ public class RealtimePlumberSchoolTest { return new JSONParseSpec( new TimestampSpec("timestamp", "auto"), - new DimensionsSpec(null, null, null), - JSONParseSpec.JSON + new DimensionsSpec(null, null, null) ); } From e2e912cdc37e404e8f9c0dd7cf1192e1e8bec716 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 9 Oct 2014 10:26:39 -0700 Subject: [PATCH 19/53] fix broken link, resolved #782 --- docs/content/Querying.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Querying.md b/docs/content/Querying.md index 2b2981b96f7..df516ac4e05 100644 --- a/docs/content/Querying.md +++ b/docs/content/Querying.md @@ -88,7 +88,7 @@ A groupBy also requires the JSON field "aggregations" (See [Aggregations](Aggreg ], ``` -You can also specify postAggregations, which are applied after data has been aggregated for the current granularity and dimensions bucket. See [Post Aggregations](Post Aggregations.html) for a detailed description. In the rand example, an arithmetic type operation (division, as specified by "fn") is performed with the result "name" of "avg_random". The "fields" field specifies the inputs from the aggregation stage to this expression. Note that identifiers corresponding to "name" JSON field inside the type "fieldAccess" are required but not used outside this expression, so they are prefixed with "dummy" for clarity: +You can also specify postAggregations, which are applied after data has been aggregated for the current granularity and dimensions bucket. See [Post Aggregations](Post-aggregations.html) for a detailed description. In the rand example, an arithmetic type operation (division, as specified by "fn") is performed with the result "name" of "avg_random". The "fields" field specifies the inputs from the aggregation stage to this expression. Note that identifiers corresponding to "name" JSON field inside the type "fieldAccess" are required but not used outside this expression, so they are prefixed with "dummy" for clarity: ```javascript "postAggregations": [{ From fa27908ae41f7438cea6a28718aa1f9b42957d9f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 9 Oct 2014 10:28:33 -0700 Subject: [PATCH 20/53] and another one, fixes #782 --- docs/content/Querying.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Querying.md b/docs/content/Querying.md index df516ac4e05..895a1cf2239 100644 --- a/docs/content/Querying.md +++ b/docs/content/Querying.md @@ -127,7 +127,7 @@ Properties shared by all query types |timeseries, topN, groupBy, search|filter|Specifies the filter (the "WHERE" clause in SQL) for the query. See [Filters](Filters.html)|no| |timeseries, topN, groupBy, search|granularity|the timestamp granularity to bucket results into (i.e. "hour"). See [Granularities](Granularities.html) for more information.|no| |timeseries, topN, groupBy|aggregations|aggregations that combine values in a bucket. See [Aggregations](Aggregations.html).|yes| -|timeseries, topN, groupBy|postAggregations|aggregations of aggregations. See [Post Aggregations](Post Aggregations.html).|yes| +|timeseries, topN, groupBy|postAggregations|aggregations of aggregations. See [Post Aggregations](Post-aggregations.html).|yes| |groupBy|dimensions|constrains the groupings; if empty, then one value per time granularity bucket|yes| |search|limit|maximum number of results (default is 1000), a system-level maximum can also be set via `com.metamx.query.search.maxSearchLimit`|no| |search|searchDimensions|Dimensions to apply the search query to. If not specified, it will search through all dimensions.|no| From 0532208c25ad737adadbbe29670e6dc0d160f42d Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 9 Oct 2014 14:21:38 -0700 Subject: [PATCH 21/53] prepare for next release --- docs/content/Examples.md | 4 ++-- docs/content/Kafka-Eight.md | 4 ++-- docs/content/Production-Cluster-Configuration.md | 6 +++--- docs/content/Realtime-Config.md | 4 ++-- docs/content/Router.md | 2 +- docs/content/Simple-Cluster-Configuration.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-1.md | 2 +- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.md | 2 +- pom.xml | 2 +- 12 files changed, 21 insertions(+), 21 deletions(-) diff --git a/docs/content/Examples.md b/docs/content/Examples.md index 049218da290..b61168e20dc 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.157 +git checkout druid-0.6.158 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Kafka-Eight.md b/docs/content/Kafka-Eight.md index c167aea59ba..b7ae4e5c4ab 100644 --- a/docs/content/Kafka-Eight.md +++ b/docs/content/Kafka-Eight.md @@ -8,9 +8,9 @@ The previous examples are for Kafka 7. To support Kafka 8, a couple changes need - Update realtime node's configs for Kafka 8 extensions - e.g. - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.157",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.158",...]` - becomes - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.157",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.158",...]` - Update realtime task config for changed keys - `firehose.type`, `plumber.rejectionPolicyFactory`, and all of `firehose.consumerProps` changes. diff --git a/docs/content/Production-Cluster-Configuration.md b/docs/content/Production-Cluster-Configuration.md index 0cf81020510..70b4cb4036d 100644 --- a/docs/content/Production-Cluster-Configuration.md +++ b/docs/content/Production-Cluster-Configuration.md @@ -57,7 +57,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/overlord -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -139,7 +139,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/middlemanager -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157","io.druid.extensions:druid-kafka-seven:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158","io.druid.extensions:druid-kafka-seven:0.6.158"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -286,7 +286,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/historical -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Realtime-Config.md b/docs/content/Realtime-Config.md index cf5104d550d..c61a22b736d 100644 --- a/docs/content/Realtime-Config.md +++ b/docs/content/Realtime-Config.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.158"] druid.zk.service.host=localhost @@ -76,7 +76,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/realtime -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157","io.druid.extensions:druid-kafka-seven:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158","io.druid.extensions:druid-kafka-seven:0.6.158"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Router.md b/docs/content/Router.md index b6638fdadd2..2a077e944fb 100644 --- a/docs/content/Router.md +++ b/docs/content/Router.md @@ -51,7 +51,7 @@ druid.service=druid/prod/router druid.extensions.remoteRepositories=[] druid.extensions.localRepository=lib -druid.extensions.coordinates=["io.druid.extensions:druid-histogram:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-histogram:0.6.158"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Simple-Cluster-Configuration.md b/docs/content/Simple-Cluster-Configuration.md index e873895f097..98871836233 100644 --- a/docs/content/Simple-Cluster-Configuration.md +++ b/docs/content/Simple-Cluster-Configuration.md @@ -28,7 +28,7 @@ Configuration: -Ddruid.zk.service.host=localhost --Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.157"] +-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.158"] -Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid -Ddruid.db.connector.user=druid diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 95f53508448..7d07d713fb0 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.157 +cd druid-services-0.6.158 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md index 12a5ea405af..e4242e32cbc 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md @@ -91,7 +91,7 @@ druid.service=overlord druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.158"] druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid druid.db.connector.user=druid diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 5673a9f8f1f..51e585e40ba 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes and external depende If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -240,7 +240,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.157","io.druid.extensions:druid-kafka-seven:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.158","io.druid.extensions:druid-kafka-seven:0.6.158"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index 83dfd848924..f7893348ef6 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.157 +cd druid-services-0.6.158 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.md b/docs/content/Twitter-Tutorial.md index c8f2b2adc0e..dd69a952ae9 100644 --- a/docs/content/Twitter-Tutorial.md +++ b/docs/content/Twitter-Tutorial.md @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. # Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz). +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz). Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: diff --git a/pom.xml b/pom.xml index 5731b98d72a..62408db61aa 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.9 2.6.0 - 0.2.13 + 0.2.14 From dada60a301d3147c5bb483f8f4d0a8eb1e598e74 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 9 Oct 2014 14:24:48 -0700 Subject: [PATCH 22/53] [maven-release-plugin] prepare release druid-0.6.158 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index f976ec19663..24ba056e724 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/common/pom.xml b/common/pom.xml index 62a4c7efd0d..38be89a6821 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/examples/pom.xml b/examples/pom.xml index e4cf96e065e..2c4721fe102 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 2ef16b75cfe..83130405c55 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/histogram/pom.xml b/histogram/pom.xml index dd73d1a7ea3..7b357b03bf6 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 1268759137d..267451d224f 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 8031019b137..4d0f44893f6 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 3a30b97b1eb..86952798fa4 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 502517995c3..fdb82e37fcd 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/pom.xml b/pom.xml index 62408db61aa..2cc2e7754bd 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.158-SNAPSHOT + 0.6.158 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 - druid-0.6.131-SNAPSHOT + druid-0.6.158 diff --git a/processing/pom.xml b/processing/pom.xml index a6f09b1f608..ee57f65523d 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 291e333eab5..09dc0c7d46d 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index eeda5228469..0b58c6f5661 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/server/pom.xml b/server/pom.xml index 0fcf50829bd..9b6c8952ba0 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/services/pom.xml b/services/pom.xml index bd4d5d7a6d2..03bd30a8695 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 From d0fbd5f729c142f9482360bf41d3b969073e2330 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 9 Oct 2014 14:24:53 -0700 Subject: [PATCH 23/53] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 24ba056e724..795430b3d7a 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 38be89a6821..8b657d43b7b 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 2c4721fe102..e8e39269e2e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 83130405c55..a1fbad02890 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index 7b357b03bf6..2d13ca77b1b 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 267451d224f..7ae737f572f 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 4d0f44893f6..bb18a62d312 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 86952798fa4..d6fa98d745a 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index fdb82e37fcd..66f80d94371 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/pom.xml b/pom.xml index 2cc2e7754bd..f8c672c1c41 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.158 + 0.6.159-SNAPSHOT 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 - druid-0.6.158 + druid-0.6.131-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index ee57f65523d..adfec8434f8 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 09dc0c7d46d..c7ffa9270ae 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 0b58c6f5661..9fcaf6349dd 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 9b6c8952ba0..fe3b96333e2 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 03bd30a8695..f9c32a441a2 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT From 592611f2ee72e5de037f7c38e7d46927ece32560 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 9 Oct 2014 17:04:36 -0700 Subject: [PATCH 24/53] fix cardinality aggregator serde --- .../CardinalityAggregatorFactory.java | 10 ++++++++-- .../cardinality/CardinalityAggregatorTest.java | 17 ++++++++++++++++- 2 files changed, 24 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java index b1db05a2284..071a91b3c3e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java @@ -63,12 +63,12 @@ public class CardinalityAggregatorFactory implements AggregatorFactory public CardinalityAggregatorFactory( @JsonProperty("name") String name, @JsonProperty("fieldNames") final List fieldNames, - @JsonProperty("byRow") final Boolean byRow + @JsonProperty("byRow") final boolean byRow ) { this.name = name; this.fieldNames = fieldNames; - this.byRow = byRow == null ? false : byRow; + this.byRow = byRow; } @Override @@ -203,6 +203,12 @@ public class CardinalityAggregatorFactory implements AggregatorFactory return fieldNames; } + @JsonProperty + public boolean isByRow() + { + return byRow; + } + @Override public byte[] getCacheKey() { diff --git a/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java index 4d27df07303..9d8ac7c721f 100644 --- a/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java @@ -19,16 +19,20 @@ package io.druid.query.aggregation.cardinality; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.BufferAggregator; import io.druid.segment.DimensionSelector; import io.druid.segment.data.IndexedInts; -import junit.framework.Assert; +import org.junit.Assert; import org.junit.Test; import javax.annotation.Nullable; @@ -378,4 +382,15 @@ public class CardinalityAggregatorTest 0.05 ); } + + @Test + public void testSerde() throws Exception + { + CardinalityAggregatorFactory factory = new CardinalityAggregatorFactory("billy", ImmutableList.of("b", "a", "c"), true); + ObjectMapper objectMapper = new DefaultObjectMapper(); + Assert.assertEquals( + factory, + objectMapper.readValue(objectMapper.writeValueAsString(factory), AggregatorFactory.class) + ); + } } From 830436594831724e676874df901c8f6645dcbd23 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 10 Oct 2014 10:19:33 -0700 Subject: [PATCH 25/53] move thanks page to main site --- docs/content/Thanks.md | 12 ------------ docs/content/toc.textile | 2 +- 2 files changed, 1 insertion(+), 13 deletions(-) delete mode 100644 docs/content/Thanks.md diff --git a/docs/content/Thanks.md b/docs/content/Thanks.md deleted file mode 100644 index 97ec7e0904a..00000000000 --- a/docs/content/Thanks.md +++ /dev/null @@ -1,12 +0,0 @@ ---- -layout: doc_page ---- - -YourKit supports the Druid open source projects with its -full-featured Java Profiler. -YourKit, LLC is the creator of innovative and intelligent tools for profiling -Java and .NET applications. Take a look at YourKit's software products: -YourKit Java -Profiler and -YourKit .NET -Profiler. diff --git a/docs/content/toc.textile b/docs/content/toc.textile index cc3c86eac99..62a9ec0776a 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -93,4 +93,4 @@ h2. Development * "Libraries":./Libraries.html h2. Misc -* "Thanks":./Thanks.html +* "Thanks":/thanks.html From e1fedbe7415aedc092d624abcef1c91cff3af194 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Sun, 12 Oct 2014 11:26:46 -0700 Subject: [PATCH 26/53] RemoteTaskRunner should respect worker version changes (fixes #787). --- .../indexing/overlord/ImmutableZkWorker.java | 2 +- .../indexing/overlord/RemoteTaskRunner.java | 28 ++++++++++++++++- .../io/druid/indexing/overlord/ZkWorker.java | 28 ++++++++--------- .../overlord/RemoteTaskRunnerTest.java | 31 +++++++++++++++++++ 4 files changed, 72 insertions(+), 17 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ImmutableZkWorker.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ImmutableZkWorker.java index f657b6a2435..f78f576de67 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ImmutableZkWorker.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ImmutableZkWorker.java @@ -32,7 +32,7 @@ public class ImmutableZkWorker { private final Worker worker; private final int currCapacityUsed; - private final Set availabilityGroups; + private final ImmutableSet availabilityGroups; public ImmutableZkWorker(Worker worker, int currCapacityUsed, Set availabilityGroups) { 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 e9bba479c64..e382c6dee3c 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 @@ -164,7 +164,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer @Override public void childEvent(CuratorFramework client, final PathChildrenCacheEvent event) throws Exception { - Worker worker; + final Worker worker; switch (event.getType()) { case CHILD_ADDED: worker = jsonMapper.readValue( @@ -198,6 +198,14 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer } ); break; + case CHILD_UPDATED: + worker = jsonMapper.readValue( + event.getData().getData(), + Worker.class + ); + updateWorker(worker); + break; + case CHILD_REMOVED: worker = jsonMapper.readValue( event.getData().getData(), @@ -745,6 +753,24 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer } } + /** + * We allow workers to change their own capacities and versions. They cannot change their own hosts or ips without + * dropping themselves and re-announcing. + */ + private void updateWorker(final Worker worker) + { + final ZkWorker zkWorker = zkWorkers.get(worker.getHost()); + if (zkWorker != null) { + log.info("Worker[%s] updated its announcement from[%s] to[%s].", worker.getHost(), zkWorker.getWorker(), worker); + zkWorker.setWorker(worker); + } else { + log.warn( + "WTF, worker[%s] updated its announcement but we didn't have a ZkWorker for it. Ignoring.", + worker.getHost() + ); + } + } + /** * When a ephemeral worker node disappears from ZK, incomplete running tasks will be retried by * the logic in the status listener. We still have to make sure there are no tasks assigned diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java index abc4da0ad57..54b09da2b29 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java @@ -22,11 +22,11 @@ package io.druid.indexing.overlord; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; +import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import io.druid.indexing.common.task.Task; import io.druid.indexing.worker.TaskAnnouncement; import io.druid.indexing.worker.Worker; import org.apache.curator.framework.recipes.cache.ChildData; @@ -46,15 +46,15 @@ import java.util.concurrent.atomic.AtomicReference; */ public class ZkWorker implements Closeable { - private final Worker worker; private final PathChildrenCache statusCache; private final Function cacheConverter; + private AtomicReference worker; private AtomicReference lastCompletedTaskTime = new AtomicReference(new DateTime()); public ZkWorker(Worker worker, PathChildrenCache statusCache, final ObjectMapper jsonMapper) { - this.worker = worker; + this.worker = new AtomicReference<>(worker); this.statusCache = statusCache; this.cacheConverter = new Function() { @@ -84,7 +84,7 @@ public class ZkWorker implements Closeable @JsonProperty("worker") public Worker getWorker() { - return worker; + return worker.get(); } @JsonProperty("runningTasks") @@ -137,30 +137,28 @@ public class ZkWorker implements Closeable return getRunningTasks().containsKey(taskId); } - public boolean isAtCapacity() - { - return getCurrCapacityUsed() >= worker.getCapacity(); - } - public boolean isValidVersion(String minVersion) { - return worker.getVersion().compareTo(minVersion) >= 0; + return worker.get().getVersion().compareTo(minVersion) >= 0; } - public boolean canRunTask(Task task) + public void setWorker(Worker newWorker) { - return (worker.getCapacity() - getCurrCapacityUsed() >= task.getTaskResource().getRequiredCapacity() - && !getAvailabilityGroups().contains(task.getTaskResource().getAvailabilityGroup())); + final Worker oldWorker = worker.get(); + Preconditions.checkArgument(newWorker.getHost().equals(oldWorker.getHost()), "Cannot change Worker host"); + Preconditions.checkArgument(newWorker.getIp().equals(oldWorker.getIp()), "Cannot change Worker ip"); + + worker.set(newWorker); } public void setLastCompletedTaskTime(DateTime completedTaskTime) { - lastCompletedTaskTime.getAndSet(completedTaskTime); + lastCompletedTaskTime.set(completedTaskTime); } public ImmutableZkWorker toImmutable() { - return new ImmutableZkWorker(worker, getCurrCapacityUsed(), getAvailabilityGroups()); + return new ImmutableZkWorker(worker.get(), getCurrCapacityUsed(), getAvailabilityGroups()); } @Override 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 11c7c85c639..b5f86ea7494 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 @@ -361,6 +361,29 @@ public class RemoteTaskRunnerTest Assert.assertEquals(TaskStatus.Status.FAILED, status.getStatusCode()); } + @Test + public void testWorkerDisabled() throws Exception + { + doSetup(); + final ListenableFuture result = remoteTaskRunner.run(task); + + Assert.assertTrue(taskAnnounced(task.getId())); + mockWorkerRunningTask(task); + Assert.assertTrue(workerRunningTask(task.getId())); + + // Disable while task running + disableWorker(); + + // Continue test + mockWorkerCompleteSuccessfulTask(task); + Assert.assertTrue(workerCompletedTask(result)); + Assert.assertEquals(task.getId(), result.get().getId()); + Assert.assertEquals(TaskStatus.Status.SUCCESS, result.get().getStatusCode()); + + // Confirm RTR thinks the worker is disabled. + Assert.assertEquals("", Iterables.getOnlyElement(remoteTaskRunner.getWorkers()).getWorker().getVersion()); + } + private void doSetup() throws Exception { makeWorker(); @@ -405,6 +428,14 @@ public class RemoteTaskRunnerTest ); } + private void disableWorker() throws Exception + { + cf.setData().forPath( + announcementsPath, + jsonMapper.writeValueAsBytes(new Worker(worker.getHost(), worker.getIp(), worker.getCapacity(), "")) + ); + } + private boolean taskAnnounced(final String taskId) { return pathExists(joiner.join(tasksPath, taskId)); From 282e40e07ce7ebb6de4d03f199e1dd9b1e89176f Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 13 Oct 2014 11:10:17 -0700 Subject: [PATCH 27/53] update bytebuffer collections dep to fix issue with empty spatial concise sets --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index f8c672c1c41..d1caceabeac 100644 --- a/pom.xml +++ b/pom.xml @@ -89,7 +89,7 @@ com.metamx bytebuffer-collections - 0.0.2 + 0.0.4 com.metamx @@ -194,7 +194,7 @@ it.uniroma3.mat extendedset - 1.3.4 + 1.3.7 com.google.guava From d4f5a71dedb93a7f9ce82a25a6139d87dd9a3cbc Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 13 Oct 2014 12:48:12 -0700 Subject: [PATCH 28/53] prepare for next release --- docs/content/Examples.md | 4 ++-- docs/content/Kafka-Eight.md | 4 ++-- docs/content/Production-Cluster-Configuration.md | 6 +++--- docs/content/Realtime-Config.md | 4 ++-- docs/content/Router.md | 2 +- docs/content/Simple-Cluster-Configuration.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-1.md | 2 +- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.md | 2 +- 11 files changed, 20 insertions(+), 20 deletions(-) diff --git a/docs/content/Examples.md b/docs/content/Examples.md index b61168e20dc..dfd44ffe927 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.158 +git checkout druid-0.6.159 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.159-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Kafka-Eight.md b/docs/content/Kafka-Eight.md index b7ae4e5c4ab..5819a931b2a 100644 --- a/docs/content/Kafka-Eight.md +++ b/docs/content/Kafka-Eight.md @@ -8,9 +8,9 @@ The previous examples are for Kafka 7. To support Kafka 8, a couple changes need - Update realtime node's configs for Kafka 8 extensions - e.g. - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.158",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.159",...]` - becomes - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.158",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.159",...]` - Update realtime task config for changed keys - `firehose.type`, `plumber.rejectionPolicyFactory`, and all of `firehose.consumerProps` changes. diff --git a/docs/content/Production-Cluster-Configuration.md b/docs/content/Production-Cluster-Configuration.md index 70b4cb4036d..6fc319712de 100644 --- a/docs/content/Production-Cluster-Configuration.md +++ b/docs/content/Production-Cluster-Configuration.md @@ -57,7 +57,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/overlord -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.159"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -139,7 +139,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/middlemanager -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158","io.druid.extensions:druid-kafka-seven:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.159","io.druid.extensions:druid-kafka-seven:0.6.159"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -286,7 +286,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/historical -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.159"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Realtime-Config.md b/docs/content/Realtime-Config.md index c61a22b736d..12ab55ff8e3 100644 --- a/docs/content/Realtime-Config.md +++ b/docs/content/Realtime-Config.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.159"] druid.zk.service.host=localhost @@ -76,7 +76,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/realtime -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158","io.druid.extensions:druid-kafka-seven:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.159","io.druid.extensions:druid-kafka-seven:0.6.159"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Router.md b/docs/content/Router.md index 2a077e944fb..73849164216 100644 --- a/docs/content/Router.md +++ b/docs/content/Router.md @@ -51,7 +51,7 @@ druid.service=druid/prod/router druid.extensions.remoteRepositories=[] druid.extensions.localRepository=lib -druid.extensions.coordinates=["io.druid.extensions:druid-histogram:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-histogram:0.6.159"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Simple-Cluster-Configuration.md b/docs/content/Simple-Cluster-Configuration.md index 98871836233..8bfade9ab9a 100644 --- a/docs/content/Simple-Cluster-Configuration.md +++ b/docs/content/Simple-Cluster-Configuration.md @@ -28,7 +28,7 @@ Configuration: -Ddruid.zk.service.host=localhost --Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.158"] +-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.159"] -Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid -Ddruid.db.connector.user=druid diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 7d07d713fb0..4a74b4f5b20 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.159-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.158 +cd druid-services-0.6.159 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md index e4242e32cbc..7ef8648cc0f 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md @@ -91,7 +91,7 @@ druid.service=overlord druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.159"] druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid druid.db.connector.user=druid diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 51e585e40ba..3797746ca20 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes and external depende If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.159-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.159"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -240,7 +240,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.158","io.druid.extensions:druid-kafka-seven:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.159","io.druid.extensions:druid-kafka-seven:0.6.159"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index f7893348ef6..47ebec7e049 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.159-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.158 +cd druid-services-0.6.159 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.md b/docs/content/Twitter-Tutorial.md index dd69a952ae9..95993732449 100644 --- a/docs/content/Twitter-Tutorial.md +++ b/docs/content/Twitter-Tutorial.md @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. # Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz). +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.159-bin.tar.gz). Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: From 7fd1747ffa8eade09704e32eaf3a6d80a19c33d6 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 13 Oct 2014 12:50:41 -0700 Subject: [PATCH 29/53] [maven-release-plugin] prepare release druid-0.6.159 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 795430b3d7a..74863effc38 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/common/pom.xml b/common/pom.xml index 8b657d43b7b..d48011e3434 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/examples/pom.xml b/examples/pom.xml index e8e39269e2e..c59a68a7772 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index a1fbad02890..546a3ec9a27 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/histogram/pom.xml b/histogram/pom.xml index 2d13ca77b1b..5159554cea2 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 7ae737f572f..32a00aa3a37 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index bb18a62d312..4a085554c10 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index d6fa98d745a..aa519cb387c 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 66f80d94371..9e491f6548c 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/pom.xml b/pom.xml index d1caceabeac..5c597957197 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.159-SNAPSHOT + 0.6.159 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 - druid-0.6.131-SNAPSHOT + druid-0.6.159 diff --git a/processing/pom.xml b/processing/pom.xml index adfec8434f8..f001fad2c83 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index c7ffa9270ae..13946216979 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 9fcaf6349dd..c5ce3df8727 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/server/pom.xml b/server/pom.xml index fe3b96333e2..be472dc473a 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/services/pom.xml b/services/pom.xml index f9c32a441a2..c7dfe120611 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 From a4c8f0440949c474bed0b7d4862aafd04380185f Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 13 Oct 2014 12:50:45 -0700 Subject: [PATCH 30/53] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 74863effc38..5682138b49d 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index d48011e3434..da437e99e1f 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index c59a68a7772..fb2597326e7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 546a3ec9a27..6cb8f84483e 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index 5159554cea2..a26ce5f5601 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 32a00aa3a37..cfd2e8da529 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 4a085554c10..5fe0ebae569 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index aa519cb387c..be87d5b4d79 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 9e491f6548c..a26e940b725 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/pom.xml b/pom.xml index 5c597957197..8c1202d7b79 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.159 + 0.6.160-SNAPSHOT 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 - druid-0.6.159 + druid-0.6.131-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index f001fad2c83..867b720ca57 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 13946216979..745f10e88ac 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index c5ce3df8727..e4914713493 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index be472dc473a..740e06d56ad 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index c7dfe120611..9968d0b6146 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT From 35716e8deef7428a4933c587453585bc1532269d Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 13 Oct 2014 14:33:45 -0700 Subject: [PATCH 31/53] add a best practices doc --- docs/content/Best-Practices.md | 23 +++++++++++++++++++++++ docs/content/toc.textile | 1 + 2 files changed, 24 insertions(+) create mode 100644 docs/content/Best-Practices.md diff --git a/docs/content/Best-Practices.md b/docs/content/Best-Practices.md new file mode 100644 index 00000000000..f4cd8f32595 --- /dev/null +++ b/docs/content/Best-Practices.md @@ -0,0 +1,23 @@ +--- +layout: doc_page +--- + +Best Practices +============== + +# Use UTC Timezone + +We recommend using UTC timezone for all your events and across on your nodes, not just for Druid, but for all data infrastructure. This can greatly mitigate potential query problems with inconsistent timezones. + +# Use Lowercase Strings for Column Names + +Druid is not perfect in how it handles mix-cased dimension and metric names. This will hopefully change very soon but for the time being, lower casing your column names is recommended. + +# 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 for Best Results + +Although Druid supports schemaless ingestion of dimensions, because of 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. This may require some ETL processing on your data however. + diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 62a9ec0776a..21f867bca36 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -19,6 +19,7 @@ h2. Booting a Druid Cluster * "Production Cluster Configuration":Production-Cluster-Configuration.html * "Production Hadoop Configuration":Hadoop-Configuration.html * "Rolling Cluster Updates":Rolling-Updates.html +* "Best Practices":Best-Practices.html h2. Configuration * "Common Configuration":Configuration.html From 62f77fabc46f51870588c93ae9de6a10876c506b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 13 Oct 2014 17:45:34 -0400 Subject: [PATCH 32/53] simplify test for approx histogram --- .../ApproximateHistogramGroupByQueryTest.java | 60 +++++++------------ 1 file changed, 23 insertions(+), 37 deletions(-) diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index cd7aa5a70c1..6d6aade79cb 100644 --- a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -30,12 +30,9 @@ import io.druid.data.input.Row; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; -import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.MaxAggregatorFactory; -import io.druid.query.aggregation.MinAggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; -import io.druid.query.dimension.LegacyDimensionSpec; import io.druid.query.groupby.GroupByQuery; import io.druid.query.groupby.GroupByQueryConfig; import io.druid.query.groupby.GroupByQueryEngine; @@ -165,13 +162,20 @@ public class ApproximateHistogramGroupByQueryTest GroupByQuery query = new GroupByQuery.Builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setGranularity(QueryRunnerTestHelper.allGran) - .setDimensions(Arrays.asList(new LegacyDimensionSpec(QueryRunnerTestHelper.providerDimension))) + .setDimensions( + Arrays.asList( + new DefaultDimensionSpec( + QueryRunnerTestHelper.providerDimension, + "provider" + ) + ) + ) .setInterval(QueryRunnerTestHelper.fullOnInterval) .setLimitSpec( new DefaultLimitSpec( Lists.newArrayList( new OrderByColumnSpec( - QueryRunnerTestHelper.providerDimension, + "provider", OrderByColumnSpec.Direction.DESCENDING ) ), 1 @@ -179,20 +183,12 @@ public class ApproximateHistogramGroupByQueryTest ) .setAggregatorSpecs( Lists.newArrayList( - Iterables.concat( - QueryRunnerTestHelper.commonAggregators, - Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index"), - aggFactory - ) - ) + QueryRunnerTestHelper.rowsCount, + aggFactory ) ) .setPostAggregatorSpecs( - Arrays.asList( - QueryRunnerTestHelper.addRowsIndexConstant, - QueryRunnerTestHelper.dependentPostAgg, + Arrays.asList( new QuantilePostAggregator("quantile", "apphisto", 0.5f) ) ) @@ -201,31 +197,21 @@ public class ApproximateHistogramGroupByQueryTest List expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01T00:00:00.000Z", - "provider", "spot", - "rows", 837L, - "addRowsIndexConstant", 96444.5703125, - "dependentPostAgg", 97282.5703125, - "index", 95606.5703125, - "maxIndex", 277.2735290527344, - "minIndex", 59.02102279663086, - "quantile", 101.78856f, - "uniques", QueryRunnerTestHelper.UNIQUES_9, + "provider", "upfront", + "rows", 186L, + "quantile", 880.9881f, "apphisto", new Histogram( new float[]{ - 4.457897186279297f, - 59.02102279663086f, - 113.58415222167969f, - 168.14727783203125f, - 222.7104034423828f, - 277.2735290527344f + 214.97299194335938f, + 545.9906005859375f, + 877.0081787109375f, + 1208.0257568359375f, + 1539.0433349609375f, + 1870.06103515625f }, new double[]{ - 0.0, - 462.4309997558594, - 357.5404968261719, - 15.022850036621094, - 2.0056631565093994 + 0.0, 67.53287506103516, 72.22068786621094, 31.984678268432617, 14.261756896972656 } ) ) From c18f156cd8ab710eb8ceaabce99aa283815efcc1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 13 Oct 2014 17:53:24 -0400 Subject: [PATCH 33/53] remove unused test code --- .../histogram/ApproximateHistogramGroupByQueryTest.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index 6d6aade79cb..cfd1b37835e 100644 --- a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -42,7 +42,6 @@ import io.druid.query.groupby.GroupByQueryRunnerTestHelper; import io.druid.query.groupby.orderby.DefaultLimitSpec; import io.druid.query.groupby.orderby.OrderByColumnSpec; import io.druid.segment.TestHelper; -import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -61,13 +60,6 @@ public class ApproximateHistogramGroupByQueryTest { private final QueryRunner runner; private GroupByQueryRunnerFactory factory; - private Supplier configSupplier; - - @Before - public void setUp() throws Exception - { - configSupplier = Suppliers.ofInstance(new GroupByQueryConfig()); - } @Parameterized.Parameters public static Collection constructorFeeder() throws IOException From 5b194ee5fe154bfa462fc8d65a03ce361f919cc9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 14 Oct 2014 09:22:19 -0400 Subject: [PATCH 34/53] fix groupBy ordering casing bug --- .../ApproximateHistogramGroupByQueryTest.java | 6 +-- .../groupby/orderby/DefaultLimitSpec.java | 2 +- .../query/groupby/GroupByQueryRunnerTest.java | 42 +++++++++++++++++++ 3 files changed, 46 insertions(+), 4 deletions(-) diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index cfd1b37835e..d623b9b6649 100644 --- a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -158,7 +158,7 @@ public class ApproximateHistogramGroupByQueryTest Arrays.asList( new DefaultDimensionSpec( QueryRunnerTestHelper.providerDimension, - "provider" + "proViderAlias" ) ) ) @@ -167,7 +167,7 @@ public class ApproximateHistogramGroupByQueryTest new DefaultLimitSpec( Lists.newArrayList( new OrderByColumnSpec( - "provider", + "proViderAlias", OrderByColumnSpec.Direction.DESCENDING ) ), 1 @@ -189,7 +189,7 @@ public class ApproximateHistogramGroupByQueryTest List expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01T00:00:00.000Z", - "provider", "upfront", + "provideralias", "upfront", "rows", 186L, "quantile", 880.9881f, "apphisto", diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java index bf6fc138ad4..ad96280a925 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java @@ -173,7 +173,7 @@ public class DefaultLimitSpec implements LimitSpec public String apply(Row input) { // Multi-value dimensions have all been flattened at this point; - final List dimList = input.getDimension(dimension); + final List dimList = input.getDimension(dimension.toLowerCase()); return dimList.isEmpty() ? null : dimList.get(0); } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 6beab76793d..1456df7fcb9 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -733,6 +733,48 @@ public class GroupByQueryRunnerTest ); } + @Test + public void testGroupByWithMixedCasingOrdering() + { + GroupByQuery query = new GroupByQuery.Builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setGranularity(QueryRunnerTestHelper.allGran) + .setDimensions( + Arrays.asList( + new DefaultDimensionSpec( + QueryRunnerTestHelper.providerDimension, + "ProviderAlias" + ) + ) + ) + .setInterval(QueryRunnerTestHelper.fullOnInterval) + .setLimitSpec( + new DefaultLimitSpec( + Lists.newArrayList( + new OrderByColumnSpec( + "providerALIAS", + OrderByColumnSpec.Direction.DESCENDING + ) + ), 3 + ) + ) + .setAggregatorSpecs( + Lists.newArrayList( + QueryRunnerTestHelper.rowsCount + ) + ) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "provideralias", "upfront", "rows", 186L), + GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "provideralias", "total_market", "rows", 186L), + GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "provideralias", "spot", "rows", 837L) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, "order-limit"); + } + @Test public void testHavingSpec() { From f4a97aebbc2605798637285c123a709008a9f527 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 15 Oct 2014 22:32:56 +0530 Subject: [PATCH 35/53] fix rollup for hashed partitions truncate timestamp while calculating the partitionNumber --- .../io/druid/indexer/HadoopDruidIndexerConfig.java | 5 ++++- .../io/druid/indexing/common/task/IndexTask.java | 12 +++++++----- pom.xml | 2 +- .../partition/HashBasedNumberedShardSpec.java | 12 ++++++------ .../io/druid/timeline/partition/LinearShardSpec.java | 4 ++-- .../druid/timeline/partition/NumberedShardSpec.java | 4 ++-- .../timeline/partition/SingleDimensionShardSpec.java | 6 +++--- .../server/shard/HashBasedNumberedShardSpecTest.java | 5 +++-- .../server/shard/SingleDimensionShardSpecTest.java | 2 +- 9 files changed, 29 insertions(+), 23 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 e6402221e45..335c8c1858b 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -41,6 +41,7 @@ import com.metamx.common.logger.Logger; import io.druid.common.utils.JodaUtils; import io.druid.data.input.InputRow; import io.druid.data.input.impl.StringInputRowParser; +import io.druid.granularity.QueryGranularity; import io.druid.guice.GuiceInjectors; import io.druid.guice.JsonConfigProvider; import io.druid.guice.annotations.Self; @@ -172,6 +173,7 @@ public class HadoopDruidIndexerConfig private volatile PathSpec pathSpec; private volatile Map shardSpecLookups = Maps.newHashMap(); private volatile Map hadoopShardSpecLookup = Maps.newHashMap(); + private final QueryGranularity rollupGran; @JsonCreator public HadoopDruidIndexerConfig( @@ -203,6 +205,7 @@ public class HadoopDruidIndexerConfig hadoopShardSpecLookup.put(hadoopyShardSpec.getActualSpec(), hadoopyShardSpec); } } + this.rollupGran = schema.getDataSchema().getGranularitySpec().getQueryGranularity(); } @JsonProperty @@ -326,7 +329,7 @@ public class HadoopDruidIndexerConfig return Optional.absent(); } - final ShardSpec actualSpec = shardSpecLookups.get(timeBucket.get().getStart()).getShardSpec(inputRow); + final ShardSpec actualSpec = shardSpecLookups.get(timeBucket.get().getStart()).getShardSpec(rollupGran.truncate(inputRow.getTimestampFromEpoch()), inputRow); final HadoopyShardSpec hadoopyShardSpec = hadoopShardSpecLookup.get(actualSpec); return Optional.of( diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index b7496c70782..f04736a66e3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -407,14 +407,14 @@ public class IndexTask extends AbstractFixedIntervalTask final int myRowFlushBoundary = rowFlushBoundary > 0 ? rowFlushBoundary : toolbox.getConfig().getDefaultRowFlushBoundary(); - + final QueryGranularity rollupGran = ingestionSchema.getDataSchema().getGranularitySpec().getQueryGranularity(); try { plumber.startJob(); while (firehose.hasMore()) { final InputRow inputRow = firehose.nextRow(); - if (shouldIndex(shardSpec, interval, inputRow)) { + if (shouldIndex(shardSpec, interval, inputRow, rollupGran)) { int numRows = plumber.add(inputRow); if (numRows == -1) { throw new ISE( @@ -469,13 +469,15 @@ public class IndexTask extends AbstractFixedIntervalTask * * @return true or false */ - private boolean shouldIndex( + private static boolean shouldIndex( final ShardSpec shardSpec, final Interval interval, - final InputRow inputRow + final InputRow inputRow, + final QueryGranularity rollupGran ) { - return interval.contains(inputRow.getTimestampFromEpoch()) && shardSpec.isInChunk(inputRow); + return interval.contains(inputRow.getTimestampFromEpoch()) + && shardSpec.isInChunk(rollupGran.truncate(inputRow.getTimestampFromEpoch()), inputRow); } public static class IndexIngestionSpec extends IngestionSpec diff --git a/pom.xml b/pom.xml index 8c1202d7b79..c8eca1e8738 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.9 2.6.0 - 0.2.14 + 0.2.14.1 diff --git a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java index be640a03545..56ff5159c62 100644 --- a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java @@ -49,14 +49,14 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec } @Override - public boolean isInChunk(InputRow inputRow) + public boolean isInChunk(long timestamp, InputRow inputRow) { - return (((long) hash(inputRow)) - getPartitionNum()) % getPartitions() == 0; + return (((long) hash(timestamp, inputRow)) - getPartitionNum()) % getPartitions() == 0; } - protected int hash(InputRow inputRow) + protected int hash(long timestamp, InputRow inputRow) { - final List groupKey = Rows.toGroupKey(inputRow.getTimestampFromEpoch(), inputRow); + final List groupKey = Rows.toGroupKey(timestamp, inputRow); try { return hashFunction.hashBytes(jsonMapper.writeValueAsBytes(groupKey)).asInt(); } @@ -80,9 +80,9 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec return new ShardSpecLookup() { @Override - public ShardSpec getShardSpec(InputRow row) + public ShardSpec getShardSpec(long timestamp, InputRow row) { - int index = Math.abs(hash(row) % getPartitions()); + int index = Math.abs(hash(timestamp, row) % getPartitions()); return shardSpecs.get(index); } }; diff --git a/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java b/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java index d3ab608a328..b095f7135c4 100644 --- a/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java @@ -50,7 +50,7 @@ public class LinearShardSpec implements ShardSpec return new ShardSpecLookup() { @Override - public ShardSpec getShardSpec(InputRow row) + public ShardSpec getShardSpec(long timestamp, InputRow row) { return shardSpecs.get(0); } @@ -63,7 +63,7 @@ public class LinearShardSpec implements ShardSpec } @Override - public boolean isInChunk(InputRow inputRow) { + public boolean isInChunk(long timestamp, InputRow inputRow) { return true; } diff --git a/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java b/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java index 65399f1009f..7c90e66c711 100644 --- a/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java @@ -60,7 +60,7 @@ public class NumberedShardSpec implements ShardSpec return new ShardSpecLookup() { @Override - public ShardSpec getShardSpec(InputRow row) + public ShardSpec getShardSpec(long timestamp, InputRow row) { return shardSpecs.get(0); } @@ -80,7 +80,7 @@ public class NumberedShardSpec implements ShardSpec } @Override - public boolean isInChunk(InputRow inputRow) + public boolean isInChunk(long timestamp, InputRow inputRow) { return true; } diff --git a/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java b/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java index 1cf232a4b29..2350dc559ef 100644 --- a/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java @@ -100,10 +100,10 @@ public class SingleDimensionShardSpec implements ShardSpec return new ShardSpecLookup() { @Override - public ShardSpec getShardSpec(InputRow row) + public ShardSpec getShardSpec(long timestamp, InputRow row) { for (ShardSpec spec : shardSpecs) { - if (spec.isInChunk(row)) { + if (spec.isInChunk(timestamp, row)) { return spec; } } @@ -124,7 +124,7 @@ public class SingleDimensionShardSpec implements ShardSpec } @Override - public boolean isInChunk(InputRow inputRow) + public boolean isInChunk(long timestamp, InputRow inputRow) { final List values = inputRow.getDimension(dimension); diff --git a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java index a5e3ff1181d..47c78724010 100644 --- a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java +++ b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java @@ -127,7 +127,7 @@ public class HashBasedNumberedShardSpecTest public boolean assertExistsInOneSpec(List specs, InputRow row) { for (ShardSpec spec : specs) { - if (spec.isInChunk(row)) { + if (spec.isInChunk(row.getTimestampFromEpoch(), row)) { return true; } } @@ -145,7 +145,7 @@ public class HashBasedNumberedShardSpecTest } @Override - protected int hash(InputRow inputRow) + protected int hash(long timestamp, InputRow inputRow) { return inputRow.hashCode(); } @@ -208,4 +208,5 @@ public class HashBasedNumberedShardSpecTest return 0; } } + } diff --git a/server/src/test/java/io/druid/server/shard/SingleDimensionShardSpecTest.java b/server/src/test/java/io/druid/server/shard/SingleDimensionShardSpecTest.java index d07b52d9fcb..5bb49b38e5c 100644 --- a/server/src/test/java/io/druid/server/shard/SingleDimensionShardSpecTest.java +++ b/server/src/test/java/io/druid/server/shard/SingleDimensionShardSpecTest.java @@ -111,7 +111,7 @@ public class SingleDimensionShardSpecTest } ) ); - Assert.assertEquals(String.format("spec[%s], row[%s]", spec, inputRow), pair.lhs, spec.isInChunk(inputRow)); + Assert.assertEquals(String.format("spec[%s], row[%s]", spec, inputRow), pair.lhs, spec.isInChunk(inputRow.getTimestampFromEpoch(), inputRow)); } } } From 41e88baecaec5a97f34655a3c93a61283e90f3a0 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 15 Oct 2014 23:09:28 +0530 Subject: [PATCH 36/53] Add test for bucket selection --- .../indexer/HadoopDruidIndexerConfigTest.java | 83 +++++++++++++++++++ 1 file changed, 83 insertions(+) diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java index ee8fa9315c5..1fe6068f2ad 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -20,15 +20,34 @@ package io.druid.indexer; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.client.util.Lists; +import com.google.common.base.Optional; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.metamx.common.Granularity; +import io.druid.data.input.InputRow; +import io.druid.data.input.MapBasedInputRow; +import io.druid.data.input.impl.JSONDataSpec; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.indexer.rollup.DataRollupSpec; import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import io.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.joda.time.DateTime; +import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; +import java.util.Arrays; +import java.util.List; + /** */ public class HadoopDruidIndexerConfigTest @@ -125,4 +144,68 @@ public class HadoopDruidIndexerConfigTest ); } + + @Test + public void testHashedBucketSelection() { + List specs = Lists.newArrayList(); + final int partitionCount = 10; + for (int i = 0; i < partitionCount; i++) { + specs.add(new HadoopyShardSpec(new HashBasedNumberedShardSpec(i, partitionCount, new DefaultObjectMapper()), i)); + } + HadoopIngestionSpec spec = new HadoopIngestionSpec( + null, null, null, + "foo", + new TimestampSpec("timestamp", "auto"), + new JSONDataSpec(ImmutableList.of("foo"), null), + new UniformGranularitySpec( + Granularity.HOUR, + QueryGranularity.MINUTE, + ImmutableList.of(new Interval("2010-01-01/P1D")), + Granularity.HOUR + ), + null, + null, + null, + null, + null, + false, + true, + ImmutableMap.of(new DateTime("2010-01-01T01:00:00"), specs), + false, + new DataRollupSpec(ImmutableList.of(), QueryGranularity.MINUTE), + null, + false, + ImmutableMap.of("foo", "bar"), + false, + null, + null, + null, + null, + null, + null + ); + HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromSchema(spec); + final List dims = Arrays.asList("diM1", "dIM2"); + final ImmutableMap values = ImmutableMap.of( + "Dim1", + "1", + "DiM2", + "2", + "dim1", + "3", + "dim2", + "4" + ); + final long timestamp = new DateTime("2010-01-01T01:00:01").getMillis(); + final Bucket expectedBucket = config.getBucket(new MapBasedInputRow(timestamp, dims, values)).get(); + final long nextBucketTimestamp = QueryGranularity.MINUTE.next(QueryGranularity.MINUTE.truncate(timestamp)); + // check that all rows having same set of dims and truncated timestamp hash to same bucket + for (int i = 0; timestamp + i < nextBucketTimestamp; i++) { + Assert.assertEquals( + expectedBucket.partitionNum, + config.getBucket(new MapBasedInputRow(timestamp + i, dims, values)).get().partitionNum + ); + } + + } } From 20d29666af5d984d85de04ce18abb48410e346c6 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 16 Oct 2014 13:17:04 -0700 Subject: [PATCH 37/53] New PartitionsSpec docs. --- docs/content/Batch-ingestion.md | 61 ++++++++++++++++++++++----------- 1 file changed, 41 insertions(+), 20 deletions(-) diff --git a/docs/content/Batch-ingestion.md b/docs/content/Batch-ingestion.md index 2f53eb48b4e..0326d1e1a66 100644 --- a/docs/content/Batch-ingestion.md +++ b/docs/content/Batch-ingestion.md @@ -162,37 +162,58 @@ The indexing process has the ability to roll data up as it processes the incomin ### Partitioning specification -Segments are always partitioned based on timestamp (according to the granularitySpec) and may be further partitioned in some other way depending on partition type. -Druid supports two types of partitions spec - singleDimension and hashed. +Segments are always partitioned based on timestamp (according to the granularitySpec) and may be further partitioned in +some other way depending on partition type. Druid supports two types of partitioning strategies: "hashed" (based on the +hash of all dimensions in each row), and "dimension" (based on ranges of a single dimension). -In SingleDimension partition type data is partitioned based on the values in that dimension. -For example, data for a day may be split by the dimension "last\_name" into two segments: one with all values from A-M and one with all values from N-Z. +Hashed partitioning is recommended in most cases, as it will improve indexing performance and create more uniformly +sized data segments relative to single-dimension partitioning. -In hashed partition type, the number of partitions is determined based on the targetPartitionSize and cardinality of input set and the data is partitioned based on the hashcode of the row. - -It is recommended to use Hashed partition as it is more efficient than singleDimension since it does not need to determine the dimension for creating partitions. -Hashing also gives better distribution of data resulting in equal sized partitions and improving query performance - -To use this druid to automatically determine optimal partitions indexer must be given a target partition size. It can then find a good set of partition ranges on its own. - -#### Configuration for disabling auto-sharding and creating Fixed number of partitions - Druid can be configured to NOT run determine partitions and create a fixed number of shards by specifying numShards in hashed partitionsSpec. - e.g This configuration will skip determining optimal partitions and always create 4 shards for every segment granular interval +#### Hash-based partitioning ```json "partitionsSpec": { - "type": "hashed" - "numShards": 4 + "type": "hashed", + "targetPartitionSize": 5000000 } ``` +Hashed partitioning works by first selecting a number of segments, and then partitioning rows across those segments +according to the hash of all dimensions in each row. The number of segments is determined automatically based on the +cardinality of the input set and a target partition size. + +The configuration options are: + |property|description|required?| |--------|-----------|---------| -|type|type of partitionSpec to be used |no, default : singleDimension| -|targetPartitionSize|target number of rows to include in a partition, should be a number that targets segments of 700MB\~1GB.|yes| +|type|type of partitionSpec to be used |"hashed"| +|targetPartitionSize|target number of rows to include in a partition, should be a number that targets segments of 500MB\~1GB.|either this or numShards| +|numShards|specify the number of partitions directly, instead of a target partition size. Ingestion will run faster, since it can skip the step necessary to select a number of partitions automatically.|either this or targetPartitionSize| + +#### Single-dimension partitioning + +```json + "partitionsSpec": { + "type": "dimension", + "targetPartitionSize": 5000000 + } +``` + +Single-dimension partitioning works by first selecting a dimension to partition on, and then separating that dimension +into contiguous ranges. Each segment will contain all rows with values of that dimension in that range. For example, +your segments may be partitioned on the dimension "host" using the ranges "a.example.com" to "f.example.com" and +"f.example.com" to "z.example.com". By default, the dimension to use is determined automatically, although you can +override it with a specific dimension. + +The configuration options are: + +|property|description|required?| +|--------|-----------|---------| +|type|type of partitionSpec to be used |"dimension"| +|targetPartitionSize|target number of rows to include in a partition, should be a number that targets segments of 500MB\~1GB.|yes| +|maxPartitionSize|maximum number of rows to include in a partition. Defaults to 50% larger than the targetPartitionSize.|no| |partitionDimension|the dimension to partition on. Leave blank to select a dimension automatically.|no| -|assumeGrouped|assume input data has already been grouped on time and dimensions. This is faster, but can choose suboptimal partitions if the assumption is violated.|no| -|numShards|provides a way to manually override druid-auto sharding and specify the number of shards to create for each segment granular interval.It is only supported by hashed partitionSpec and targetPartitionSize must be set to -1|no| +|assumeGrouped|assume input data has already been grouped on time and dimensions. Ingestion will run faster, but can choose suboptimal partitions if the assumption is violated.|no| ### Updater job spec From 3d09edaae661e0139c9cca112ab588f770db68f4 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 17 Oct 2014 11:31:18 -0400 Subject: [PATCH 38/53] fix rabbit serde and add tests; fixes #794 --- rabbitmq/pom.xml | 8 ++- .../JacksonifiedConnectionFactory.java | 14 +++- .../rabbitmq/RabbitMQDruidModule.java | 2 +- .../rabbitmq/RabbitMQFirehoseConfig.java | 18 +++-- .../rabbitmq/RabbitMQFirehoseFactory.java | 57 +++++++++------ .../rabbitmq/RabbitMQFirehoseFactoryTest.java | 69 +++++++++++++++++++ 6 files changed, 139 insertions(+), 29 deletions(-) create mode 100644 rabbitmq/src/test/java/io/druid/examples/rabbitmq/RabbitMQFirehoseFactoryTest.java diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 745f10e88ac..e0d2e693414 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -1,5 +1,6 @@ - + 4.0.0 io.druid.extensions druid-rabbitmq @@ -17,6 +18,11 @@ io.druid druid-api + + io.druid + druid-processing + ${project.parent.version} + com.rabbitmq amqp-client diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java index 132fe3b6179..9f6e6f67ec8 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java @@ -20,8 +20,10 @@ package io.druid.firehose.rabbitmq; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Maps; import com.rabbitmq.client.ConnectionFactory; +import javax.annotation.Nullable; import java.net.URISyntaxException; import java.security.KeyManagementException; import java.security.NoSuchAlgorithmException; @@ -161,7 +163,17 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory @JsonProperty public Map getClientProperties() { - return super.getClientProperties(); + return Maps.transformEntries( + super.getClientProperties(), + new Maps.EntryTransformer() + { + @Override + public Object transformEntry(String key, Object value) + { + return value.toString(); + } + } + ); } @Override diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQDruidModule.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQDruidModule.java index 548cbcc1d1a..8ca79bbea99 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQDruidModule.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQDruidModule.java @@ -29,7 +29,7 @@ import java.util.List; /** */ -public class RabbitMQDruidModule implements DruidModule +public class RabbitMQDruidModule implements DruidModule { @Override public List getJacksonModules() diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseConfig.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseConfig.java index 7bae291c8a3..0b17e533144 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseConfig.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseConfig.java @@ -105,29 +105,35 @@ public class RabbitMQFirehoseConfig } @JsonProperty - public int getMaxRetries() { + public int getMaxRetries() + { return maxRetries; } - public void setMaxRetries(int maxRetries) { + public void setMaxRetries(int maxRetries) + { this.maxRetries = maxRetries; } @JsonProperty - public int getRetryIntervalSeconds() { + public int getRetryIntervalSeconds() + { return retryIntervalSeconds; } - public void setRetryIntervalSeconds(int retryIntervalSeconds) { + public void setRetryIntervalSeconds(int retryIntervalSeconds) + { this.retryIntervalSeconds = retryIntervalSeconds; } @JsonProperty - public long getMaxDurationSeconds() { + public long getMaxDurationSeconds() + { return maxDurationSeconds; } - public void setMaxDurationSeconds(int maxDurationSeconds) { + public void setMaxDurationSeconds(int maxDurationSeconds) + { this.maxDurationSeconds = maxDurationSeconds; } } diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java index 8504c70ed9d..ec650829e5a 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java @@ -50,14 +50,14 @@ import java.util.concurrent.LinkedBlockingQueue; /** * A FirehoseFactory for RabbitMQ. - * + *

* It will receive it's configuration through the realtime.spec file and expects to find a * consumerProps element in the firehose definition with values for a number of configuration options. * Below is a complete example for a RabbitMQ firehose configuration with some explanation. Options * that have defaults can be skipped but options with no defaults must be specified with the exception * of the URI property. If the URI property is set, it will override any other property that was also * set. - * + *

* File: realtime.spec *

  *   "firehose" : {
@@ -89,7 +89,7 @@ import java.util.concurrent.LinkedBlockingQueue;
  *     }
  *   },
  * 
- * + *

* Limitations: This implementation will not attempt to reconnect to the MQ broker if the * connection to it is lost. Furthermore it does not support any automatic failover on high availability * RabbitMQ clusters. This is not supported by the underlying AMQP client library and while the behavior @@ -97,7 +97,7 @@ import java.util.concurrent.LinkedBlockingQueue; * the RabbitMQ cluster that sets the "ha-mode" and "ha-sync-mode" properly on the queue that this * Firehose connects to, messages should survive an MQ broker node failure and be delivered once a * connection to another node is set up. - * + *

* For more information on RabbitMQ high availability please see: * http://www.rabbitmq.com/ha.html. */ @@ -105,14 +105,9 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory _queue; - public QueueingConsumer(Channel ch) { + public QueueingConsumer(Channel ch) + { this(ch, new LinkedBlockingQueue()); } - public QueueingConsumer(Channel ch, BlockingQueue q) { + public QueueingConsumer(Channel ch, BlockingQueue q) + { super(ch); this._queue = q; } - @Override public void handleShutdownSignal(String consumerTag, ShutdownSignalException sig) { + @Override + public void handleShutdownSignal(String consumerTag, ShutdownSignalException sig) + { _queue.clear(); } - @Override public void handleCancel(String consumerTag) throws IOException { + @Override + public void handleCancel(String consumerTag) throws IOException + { _queue.clear(); } - @Override public void handleDelivery(String consumerTag, - Envelope envelope, - AMQP.BasicProperties properties, - byte[] body) - throws IOException + @Override + public void handleDelivery( + String consumerTag, + Envelope envelope, + AMQP.BasicProperties properties, + byte[] body + ) + throws IOException { this._queue.add(new Delivery(envelope, properties, body)); } public Delivery nextDelivery() - throws InterruptedException, ShutdownSignalException, ConsumerCancelledException + throws InterruptedException, ShutdownSignalException, ConsumerCancelledException { return _queue.take(); } diff --git a/rabbitmq/src/test/java/io/druid/examples/rabbitmq/RabbitMQFirehoseFactoryTest.java b/rabbitmq/src/test/java/io/druid/examples/rabbitmq/RabbitMQFirehoseFactoryTest.java new file mode 100644 index 00000000000..dc52cc30c37 --- /dev/null +++ b/rabbitmq/src/test/java/io/druid/examples/rabbitmq/RabbitMQFirehoseFactoryTest.java @@ -0,0 +1,69 @@ +/* + * 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.examples.rabbitmq; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.JSONParseSpec; +import io.druid.data.input.impl.SpatialDimensionSchema; +import io.druid.data.input.impl.StringInputRowParser; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.firehose.rabbitmq.JacksonifiedConnectionFactory; +import io.druid.firehose.rabbitmq.RabbitMQFirehoseConfig; +import io.druid.firehose.rabbitmq.RabbitMQFirehoseFactory; +import io.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; + +/** + */ +public class RabbitMQFirehoseFactoryTest +{ + private static final ObjectMapper mapper = new DefaultObjectMapper(); + + @Test + public void testSerde() throws Exception + { + RabbitMQFirehoseFactory factory = new RabbitMQFirehoseFactory( + new JacksonifiedConnectionFactory(), + new RabbitMQFirehoseConfig(), + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "auto"), + new DimensionsSpec( + Arrays.asList("dim"), + Lists.newArrayList(), + Lists.newArrayList() + ) + ), + null, null, null, null + ) + ); + + byte[] bytes = mapper.writeValueAsBytes(factory); + RabbitMQFirehoseFactory factory2 = mapper.readValue(bytes, RabbitMQFirehoseFactory.class); + byte[] bytes2 = mapper.writeValueAsBytes(factory2); + + Assert.assertArrayEquals(bytes, bytes2); + } +} From bf3d31e5cc350779823001cf3e0edf68fc1285bc Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 17 Oct 2014 12:32:37 -0400 Subject: [PATCH 39/53] address cr --- .../JacksonifiedConnectionFactory.java | 209 +++++++++++------- .../rabbitmq/RabbitMQFirehoseConfig.java | 151 ++++++++----- .../rabbitmq/RabbitMQFirehoseFactory.java | 9 +- .../rabbitmq/RabbitMQFirehoseFactoryTest.java | 36 ++- 4 files changed, 272 insertions(+), 133 deletions(-) diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java index 9f6e6f67ec8..4bd8ccb24a4 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java @@ -19,14 +19,11 @@ package io.druid.firehose.rabbitmq; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Maps; import com.rabbitmq.client.ConnectionFactory; -import javax.annotation.Nullable; -import java.net.URISyntaxException; -import java.security.KeyManagementException; -import java.security.NoSuchAlgorithmException; import java.util.Map; /** @@ -35,133 +32,137 @@ import java.util.Map; */ public class JacksonifiedConnectionFactory extends ConnectionFactory { + public static JacksonifiedConnectionFactory makeDefaultConnectionFactory() throws Exception + { + return new JacksonifiedConnectionFactory(null, 0, null, null, null, null, 0, 0, 0, 0, null); + } + + private final String host; + private final int port; + private final String username; + private final String password; + private final String virtualHost; + private final String uri; + private final int requestedChannelMax; + private final int requestedFrameMax; + private final int requestedHeartbeat; + private final int connectionTimeout; + private final Map clientProperties; + + @JsonCreator + public JacksonifiedConnectionFactory( + @JsonProperty("host") String host, + @JsonProperty("port") int port, + @JsonProperty("username") String username, + @JsonProperty("password") String password, + @JsonProperty("virtualHost") String virtualHost, + @JsonProperty("uri") String uri, + @JsonProperty("requestedChannelMax") int requestedChannelMax, + @JsonProperty("requestedFrameMax") int requestedFrameMax, + @JsonProperty("requestedHeartbeat") int requestedHeartbeat, + @JsonProperty("connectionTimeout") int connectionTimeout, + @JsonProperty("clientProperties") Map clientProperties + ) throws Exception + { + this.host = host == null ? super.getHost() : host; + this.port = port == 0 ? super.getPort() : port; + this.username = username == null ? super.getUsername() : username; + this.password = password == null ? super.getPassword() : password; + this.virtualHost = virtualHost == null ? super.getVirtualHost() : virtualHost; + this.uri = uri; + this.requestedChannelMax = requestedChannelMax == 0 ? super.getRequestedChannelMax() : requestedChannelMax; + this.requestedFrameMax = requestedFrameMax == 0 ? super.getRequestedFrameMax() : requestedFrameMax; + this.requestedHeartbeat = requestedHeartbeat == 0 ? super.getRequestedHeartbeat() : requestedHeartbeat; + this.connectionTimeout = connectionTimeout == 0 ? super.getConnectionTimeout() : connectionTimeout; + this.clientProperties = clientProperties == null ? super.getClientProperties() : clientProperties; + + super.setHost(this.host); + super.setPort(this.port); + super.setUsername(this.username); + super.setPassword(this.password); + super.setVirtualHost(this.virtualHost); + if (this.uri != null) { + super.setUri(this.uri); + } + super.setRequestedChannelMax(this.requestedChannelMax); + super.setRequestedFrameMax(this.requestedFrameMax); + super.setRequestedHeartbeat(this.requestedHeartbeat); + super.setConnectionTimeout(this.connectionTimeout); + super.setClientProperties(this.clientProperties); + } + @Override @JsonProperty public String getHost() { - return super.getHost(); - } - - @Override - public void setHost(String host) - { - super.setHost(host); + return host; } @Override @JsonProperty public int getPort() { - return super.getPort(); + return port; } - @Override - public void setPort(int port) - { - super.setPort(port); - } @Override @JsonProperty public String getUsername() { - return super.getUsername(); - } - - @Override - public void setUsername(String username) - { - super.setUsername(username); + return username; } @Override @JsonProperty public String getPassword() { - return super.getPassword(); - } - - @Override - public void setPassword(String password) - { - super.setPassword(password); + return password; } @Override @JsonProperty public String getVirtualHost() { - return super.getVirtualHost(); + return virtualHost; } - @Override - public void setVirtualHost(String virtualHost) - { - super.setVirtualHost(virtualHost); - } - - @Override @JsonProperty - public void setUri(String uriString) throws URISyntaxException, NoSuchAlgorithmException, KeyManagementException + public String getUri() { - super.setUri(uriString); + return uri; } @Override @JsonProperty public int getRequestedChannelMax() { - return super.getRequestedChannelMax(); - } - - @Override - public void setRequestedChannelMax(int requestedChannelMax) - { - super.setRequestedChannelMax(requestedChannelMax); + return requestedChannelMax; } @Override @JsonProperty public int getRequestedFrameMax() { - return super.getRequestedFrameMax(); - } - - @Override - public void setRequestedFrameMax(int requestedFrameMax) - { - super.setRequestedFrameMax(requestedFrameMax); + return requestedFrameMax; } @Override @JsonProperty public int getRequestedHeartbeat() { - return super.getRequestedHeartbeat(); - } - - @Override - public void setConnectionTimeout(int connectionTimeout) - { - super.setConnectionTimeout(connectionTimeout); + return requestedHeartbeat; } @Override @JsonProperty public int getConnectionTimeout() { - return super.getConnectionTimeout(); + return connectionTimeout; } - @Override - public void setRequestedHeartbeat(int requestedHeartbeat) - { - super.setRequestedHeartbeat(requestedHeartbeat); - } - - @Override - @JsonProperty - public Map getClientProperties() + @JsonProperty("clientProperties") + public Map getClientPropertiesForSerde() { return Maps.transformEntries( super.getClientProperties(), @@ -177,8 +178,68 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory } @Override - public void setClientProperties(Map clientProperties) + public boolean equals(Object o) { - super.setClientProperties(clientProperties); + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + JacksonifiedConnectionFactory that = (JacksonifiedConnectionFactory) o; + + if (connectionTimeout != that.connectionTimeout) { + return false; + } + if (port != that.port) { + return false; + } + if (requestedChannelMax != that.requestedChannelMax) { + return false; + } + if (requestedFrameMax != that.requestedFrameMax) { + return false; + } + if (requestedHeartbeat != that.requestedHeartbeat) { + return false; + } + if (clientProperties != null ? !clientProperties.equals(that.clientProperties) : that.clientProperties != null) { + return false; + } + if (host != null ? !host.equals(that.host) : that.host != null) { + return false; + } + if (password != null ? !password.equals(that.password) : that.password != null) { + return false; + } + if (uri != null ? !uri.equals(that.uri) : that.uri != null) { + return false; + } + if (username != null ? !username.equals(that.username) : that.username != null) { + return false; + } + if (virtualHost != null ? !virtualHost.equals(that.virtualHost) : that.virtualHost != null) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = host != null ? host.hashCode() : 0; + result = 31 * result + port; + result = 31 * result + (username != null ? username.hashCode() : 0); + result = 31 * result + (password != null ? password.hashCode() : 0); + result = 31 * result + (virtualHost != null ? virtualHost.hashCode() : 0); + result = 31 * result + (uri != null ? uri.hashCode() : 0); + result = 31 * result + requestedChannelMax; + result = 31 * result + requestedFrameMax; + result = 31 * result + requestedHeartbeat; + result = 31 * result + connectionTimeout; + result = 31 * result + (clientProperties != null ? clientProperties.hashCode() : 0); + return result; } } diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseConfig.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseConfig.java index 0b17e533144..325bc41cd92 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseConfig.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseConfig.java @@ -19,6 +19,7 @@ package io.druid.firehose.rabbitmq; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; /** @@ -26,17 +27,50 @@ import com.fasterxml.jackson.annotation.JsonProperty; */ public class RabbitMQFirehoseConfig { - private String queue = null; - private String exchange = null; - private String routingKey = null; - private boolean durable = false; - private boolean exclusive = false; - private boolean autoDelete = false; - // Lyra (auto reconnect) properties - private int maxRetries = 100; - private int retryIntervalSeconds = 2; - private long maxDurationSeconds = 5 * 60; + private static final int defaultMaxRetries = 100; + private static final int defaultRetryIntervalSeconds = 2; + private static final long defaultMaxDurationSeconds = 5 * 60; + + public static RabbitMQFirehoseConfig makeDefaultConfig() + { + return new RabbitMQFirehoseConfig(null, null, null, false, false, false, 0, 0, 0); + } + + private final String queue; + private final String exchange; + private final String routingKey; + private final boolean durable; + private final boolean exclusive; + private final boolean autoDelete; + private final int maxRetries; + private final int retryIntervalSeconds; + private final long maxDurationSeconds; + + @JsonCreator + public RabbitMQFirehoseConfig( + @JsonProperty("queue") String queue, + @JsonProperty("exchange") String exchange, + @JsonProperty("routingKey") String routingKey, + @JsonProperty("durable") boolean durable, + @JsonProperty("exclusive") boolean exclusive, + @JsonProperty("autoDelete") boolean autoDelete, + @JsonProperty("maxRetries") int maxRetries, + @JsonProperty("retryIntervalSeconds") int retryIntervalSeconds, + @JsonProperty("maxDurationSeconds") long maxDurationSeconds + ) + { + this.queue = queue; + this.exchange = exchange; + this.routingKey = routingKey; + this.durable = durable; + this.exclusive = exclusive; + this.autoDelete = autoDelete; + + this.maxRetries = maxRetries == 0 ? defaultMaxRetries : maxRetries; + this.retryIntervalSeconds = retryIntervalSeconds == 0 ? defaultRetryIntervalSeconds : retryIntervalSeconds; + this.maxDurationSeconds = maxDurationSeconds == 0 ? defaultMaxDurationSeconds : maxDurationSeconds; + } @JsonProperty public String getQueue() @@ -44,96 +78,109 @@ public class RabbitMQFirehoseConfig return queue; } - public void setQueue(String queue) - { - this.queue = queue; - } - @JsonProperty public String getExchange() { return exchange; } - public void setExchange(String exchange) - { - this.exchange = exchange; - } - @JsonProperty public String getRoutingKey() { return routingKey; } - public void setRoutingKey(String routingKey) - { - this.routingKey = routingKey; - } - @JsonProperty public boolean isDurable() { return durable; } - public void setDurable(boolean durable) - { - this.durable = durable; - } - @JsonProperty public boolean isExclusive() { return exclusive; } - public void setExclusive(boolean exclusive) - { - this.exclusive = exclusive; - } - @JsonProperty public boolean isAutoDelete() { return autoDelete; } - public void setAutoDelete(boolean autoDelete) - { - this.autoDelete = autoDelete; - } - @JsonProperty public int getMaxRetries() { return maxRetries; } - public void setMaxRetries(int maxRetries) - { - this.maxRetries = maxRetries; - } - @JsonProperty public int getRetryIntervalSeconds() { return retryIntervalSeconds; } - public void setRetryIntervalSeconds(int retryIntervalSeconds) - { - this.retryIntervalSeconds = retryIntervalSeconds; - } - @JsonProperty public long getMaxDurationSeconds() { return maxDurationSeconds; } - public void setMaxDurationSeconds(int maxDurationSeconds) + @Override + public boolean equals(Object o) { - this.maxDurationSeconds = maxDurationSeconds; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + RabbitMQFirehoseConfig that = (RabbitMQFirehoseConfig) o; + + if (autoDelete != that.autoDelete) { + return false; + } + if (durable != that.durable) { + return false; + } + if (exclusive != that.exclusive) { + return false; + } + if (maxDurationSeconds != that.maxDurationSeconds) { + return false; + } + if (maxRetries != that.maxRetries) { + return false; + } + if (retryIntervalSeconds != that.retryIntervalSeconds) { + return false; + } + if (exchange != null ? !exchange.equals(that.exchange) : that.exchange != null) { + return false; + } + if (queue != null ? !queue.equals(that.queue) : that.queue != null) { + return false; + } + if (routingKey != null ? !routingKey.equals(that.routingKey) : that.routingKey != null) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = queue != null ? queue.hashCode() : 0; + result = 31 * result + (exchange != null ? exchange.hashCode() : 0); + result = 31 * result + (routingKey != null ? routingKey.hashCode() : 0); + result = 31 * result + (durable ? 1 : 0); + result = 31 * result + (exclusive ? 1 : 0); + result = 31 * result + (autoDelete ? 1 : 0); + result = 31 * result + maxRetries; + result = 31 * result + retryIntervalSeconds; + result = 31 * result + (int) (maxDurationSeconds ^ (maxDurationSeconds >>> 32)); + return result; } } diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java index ec650829e5a..f38398bcc84 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java @@ -26,7 +26,6 @@ import com.metamx.common.logger.Logger; import com.rabbitmq.client.AMQP; import com.rabbitmq.client.Channel; import com.rabbitmq.client.Connection; -import com.rabbitmq.client.ConnectionFactory; import com.rabbitmq.client.ConsumerCancelledException; import com.rabbitmq.client.DefaultConsumer; import com.rabbitmq.client.Envelope; @@ -114,10 +113,12 @@ public class RabbitMQFirehoseFactory implements FirehoseFactoryof("hi", "bye") + ); + RabbitMQFirehoseFactory factory = new RabbitMQFirehoseFactory( - new JacksonifiedConnectionFactory(), - new RabbitMQFirehoseConfig(), + connectionFactory, + config, new StringInputRowParser( new JSONParseSpec( new TimestampSpec("timestamp", "auto"), new DimensionsSpec( - Arrays.asList("dim"), + Arrays.asList("dim"), Lists.newArrayList(), Lists.newArrayList() ) @@ -65,5 +92,8 @@ public class RabbitMQFirehoseFactoryTest byte[] bytes2 = mapper.writeValueAsBytes(factory2); Assert.assertArrayEquals(bytes, bytes2); + + Assert.assertEquals(factory.getConfig(), factory2.getConfig()); + Assert.assertEquals(factory.getConnectionFactory(), factory2.getConnectionFactory()); } } From 56b8ef4629073451cd6a0fd0bb10b65900dbb5a8 Mon Sep 17 00:00:00 2001 From: fjy Date: Sun, 19 Oct 2014 16:04:52 -0700 Subject: [PATCH 40/53] one more comment --- .../druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java index 4bd8ccb24a4..56cb040f89f 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java @@ -64,6 +64,8 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory @JsonProperty("clientProperties") Map clientProperties ) throws Exception { + super(); + this.host = host == null ? super.getHost() : host; this.port = port == 0 ? super.getPort() : port; this.username = username == null ? super.getUsername() : username; From cc18bdeb0d157f3cffdb223236d22488cc323a33 Mon Sep 17 00:00:00 2001 From: fjy Date: Sun, 19 Oct 2014 16:14:26 -0700 Subject: [PATCH 41/53] another test for defaults --- .../JacksonifiedConnectionFactory.java | 4 +- .../rabbitmq/RabbitMQFirehoseFactoryTest.java | 40 +++++++++++++++++++ 2 files changed, 42 insertions(+), 2 deletions(-) diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java index 56cb040f89f..a4a39bcc9cd 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java @@ -65,7 +65,7 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory ) throws Exception { super(); - + this.host = host == null ? super.getHost() : host; this.port = port == 0 ? super.getPort() : port; this.username = username == null ? super.getUsername() : username; @@ -206,7 +206,7 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory if (requestedHeartbeat != that.requestedHeartbeat) { return false; } - if (clientProperties != null ? !clientProperties.equals(that.clientProperties) : that.clientProperties != null) { + if (clientProperties != null ? Maps.difference(clientProperties, that.clientProperties).areEqual() : that.clientProperties != null) { return false; } if (host != null ? !host.equals(that.host) : that.host != null) { diff --git a/rabbitmq/src/test/java/io/druid/examples/rabbitmq/RabbitMQFirehoseFactoryTest.java b/rabbitmq/src/test/java/io/druid/examples/rabbitmq/RabbitMQFirehoseFactoryTest.java index 09bd205df25..eca04717fc2 100644 --- a/rabbitmq/src/test/java/io/druid/examples/rabbitmq/RabbitMQFirehoseFactoryTest.java +++ b/rabbitmq/src/test/java/io/druid/examples/rabbitmq/RabbitMQFirehoseFactoryTest.java @@ -22,6 +22,7 @@ package io.druid.examples.rabbitmq; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.rabbitmq.client.ConnectionFactory; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.SpatialDimensionSchema; @@ -96,4 +97,43 @@ public class RabbitMQFirehoseFactoryTest Assert.assertEquals(factory.getConfig(), factory2.getConfig()); Assert.assertEquals(factory.getConnectionFactory(), factory2.getConnectionFactory()); } + + @Test + public void testDefaultSerde() throws Exception + { + RabbitMQFirehoseConfig config = RabbitMQFirehoseConfig.makeDefaultConfig(); + + JacksonifiedConnectionFactory connectionFactory = JacksonifiedConnectionFactory.makeDefaultConnectionFactory(); + + RabbitMQFirehoseFactory factory = new RabbitMQFirehoseFactory( + connectionFactory, + config, + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "auto"), + new DimensionsSpec( + Arrays.asList("dim"), + Lists.newArrayList(), + Lists.newArrayList() + ) + ), + null, null, null, null + ) + ); + + byte[] bytes = mapper.writeValueAsBytes(factory); + RabbitMQFirehoseFactory factory2 = mapper.readValue(bytes, RabbitMQFirehoseFactory.class); + byte[] bytes2 = mapper.writeValueAsBytes(factory2); + + Assert.assertArrayEquals(bytes, bytes2); + + Assert.assertEquals(factory.getConfig(), factory2.getConfig()); + Assert.assertEquals(factory.getConnectionFactory(), factory2.getConnectionFactory()); + + Assert.assertEquals(300, factory2.getConfig().getMaxDurationSeconds()); + + Assert.assertEquals(ConnectionFactory.DEFAULT_HOST, factory2.getConnectionFactory().getHost()); + Assert.assertEquals(ConnectionFactory.DEFAULT_USER, factory2.getConnectionFactory().getUsername()); + Assert.assertEquals(ConnectionFactory.DEFAULT_AMQP_PORT, factory2.getConnectionFactory().getPort()); + } } From 9f75df63f34d469ae27ab965425f0f0cebc6b984 Mon Sep 17 00:00:00 2001 From: fjy Date: Sun, 19 Oct 2014 16:38:21 -0700 Subject: [PATCH 42/53] fix more things --- rabbitmq/pom.xml | 11 ++--- .../JacksonifiedConnectionFactory.java | 40 +++++++++++++------ 2 files changed, 33 insertions(+), 18 deletions(-) diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index e0d2e693414..a35f3b16c15 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -18,11 +18,6 @@ io.druid druid-api - - io.druid - druid-processing - ${project.parent.version} - com.rabbitmq amqp-client @@ -45,5 +40,11 @@ commons-cli test + + io.druid + druid-processing + ${project.parent.version} + test + \ No newline at end of file diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java index a4a39bcc9cd..28f1d0d14d1 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Maps; import com.rabbitmq.client.ConnectionFactory; +import com.rabbitmq.client.LongString; import java.util.Map; @@ -37,6 +38,24 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory return new JacksonifiedConnectionFactory(null, 0, null, null, null, null, 0, 0, 0, 0, null); } + private static Map getSerializableClientProperties(final Map clientProperties) + { + return Maps.transformEntries( + clientProperties, + new Maps.EntryTransformer() + { + @Override + public Object transformEntry(String key, Object value) + { + if (value instanceof LongString) { + return value.toString(); + } + return value; + } + } + ); + } + private final String host; private final int port; private final String username; @@ -164,19 +183,9 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory } @JsonProperty("clientProperties") - public Map getClientPropertiesForSerde() + public Map getSerializableClientProperties() { - return Maps.transformEntries( - super.getClientProperties(), - new Maps.EntryTransformer() - { - @Override - public Object transformEntry(String key, Object value) - { - return value.toString(); - } - } - ); + return getSerializableClientProperties(clientProperties); } @Override @@ -206,7 +215,12 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory if (requestedHeartbeat != that.requestedHeartbeat) { return false; } - if (clientProperties != null ? Maps.difference(clientProperties, that.clientProperties).areEqual() : that.clientProperties != null) { + if (clientProperties != null + ? !Maps.difference( + getSerializableClientProperties(clientProperties), + getSerializableClientProperties(that.clientProperties) + ).areEqual() + : that.clientProperties != null) { return false; } if (host != null ? !host.equals(that.host) : that.host != null) { From 63a4f2dc055598aab2bf36bc4d55769c9ec13a49 Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Sun, 19 Oct 2014 18:29:34 -0700 Subject: [PATCH 43/53] fixed broken internal link --- docs/content/Querying.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Querying.md b/docs/content/Querying.md index 895a1cf2239..8fc926b426d 100644 --- a/docs/content/Querying.md +++ b/docs/content/Querying.md @@ -133,7 +133,7 @@ Properties shared by all query types |search|searchDimensions|Dimensions to apply the search query to. If not specified, it will search through all dimensions.|no| |search|query|The query portion of the search query. This is essentially a predicate that specifies if something matches.|yes| -Query Context +Query Context ------------- |property |default | description | From b7c4983a918edfc1dde39add84f4e35376dcb698 Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Sun, 19 Oct 2014 18:38:20 -0700 Subject: [PATCH 44/53] added syntax highlighting --- docs/content/Granularities.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/content/Granularities.md b/docs/content/Granularities.md index d4f3ea73141..e8a3e4fc2bf 100644 --- a/docs/content/Granularities.md +++ b/docs/content/Granularities.md @@ -21,13 +21,13 @@ Duration granularities are specified as an exact duration in milliseconds and ti They also support specifying an optional origin, which defines where to start counting time buckets from (defaults to 1970-01-01T00:00:00Z). -``` +```javascript {"type": "duration", "duration": "7200000"} ``` This chunks up every 2 hours. -``` +```javascript {"type": "duration", "duration": "3600000", "origin": "2012-01-01T00:30:00Z"} ``` @@ -39,13 +39,13 @@ Period granularities are specified as arbitrary period combinations of years, mo Time zone is optional (defaults to UTC). Origin is optional (defaults to 1970-01-01T00:00:00 in the given time zone). -``` +```javascript {"type": "period", "period": "P2D", "timeZone": "America/Los_Angeles"} ``` This will bucket by two-day chunks in the Pacific timezone. -``` +```javascript {"type": "period", "period": "P3M", "timeZone": "America/Los_Angeles", "origin": "2012-02-01T00:00:00-08:00"} ``` From 1dd9aeba4f22db24d12984b27976d846ddc978f0 Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Sun, 19 Oct 2014 21:22:10 -0700 Subject: [PATCH 45/53] removed link with no target --- docs/content/Querying.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Querying.md b/docs/content/Querying.md index 8fc926b426d..dd556b2740f 100644 --- a/docs/content/Querying.md +++ b/docs/content/Querying.md @@ -66,7 +66,7 @@ The dataSource JSON field shown next identifies where to apply the query. In thi "dataSource": "randSeq", ``` -The granularity JSON field specifies the bucket size for values. It could be a built-in time interval like "second", "minute", "fifteen_minute", "thirty_minute", "hour" or "day". It can also be an expression like `{"type": "period", "period":"PT6m"}` meaning "6 minute buckets". See [Granularities](Granularities.html) for more information on the different options for this field. In this example, it is set to the special value "all" which means [bucket all data points together into the same time bucket]() +The granularity JSON field specifies the bucket size for values. It could be a built-in time interval like "second", "minute", "fifteen_minute", "thirty_minute", "hour" or "day". It can also be an expression like `{"type": "period", "period":"PT6m"}` meaning "6 minute buckets". See [Granularities](Granularities.html) for more information on the different options for this field. In this example, it is set to the special value "all" which means bucket all data points together into the same time bucket. ```javascript "granularity": "all", From 38cb73e0cccc61f00b43ef4bb4944c0c1550787c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 20 Oct 2014 13:19:21 -0700 Subject: [PATCH 46/53] javacript tiered broker selector strategy --- ...avascriptTieredBrokerSelectorStrategy.java | 60 ++++++++++ .../router/TieredBrokerSelectorStrategy.java | 3 +- ...criptTieredBrokerSelectorStrategyTest.java | 103 ++++++++++++++++++ 3 files changed, 165 insertions(+), 1 deletion(-) create mode 100644 server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java create mode 100644 server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java diff --git a/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java b/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java new file mode 100644 index 00000000000..cce7fb25554 --- /dev/null +++ b/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java @@ -0,0 +1,60 @@ +/* + * 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.router; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.api.client.repackaged.com.google.common.base.Throwables; +import com.google.common.base.Optional; +import io.druid.query.Query; + +import javax.script.Compilable; +import javax.script.Invocable; +import javax.script.ScriptEngine; +import javax.script.ScriptEngineManager; +import javax.script.ScriptException; + +public class JavascriptTieredBrokerSelectorStrategy implements TieredBrokerSelectorStrategy +{ + private final SelectorFunction function; + + public JavascriptTieredBrokerSelectorStrategy(@JsonProperty("function") String function) + { + final ScriptEngine engine = new ScriptEngineManager().getEngineByName("javascript"); + try { + ((Compilable)engine).compile("var apply = " + function).eval(); + } catch(ScriptException e) { + Throwables.propagate(e); + } + this.function = ((Invocable)engine).getInterface(SelectorFunction.class); + } + + @Override + public Optional getBrokerServiceName( + TieredBrokerConfig config, Query query + ) + { + return Optional.fromNullable(function.apply(config, query)); + } + + private static interface SelectorFunction + { + public String apply(TieredBrokerConfig config, Query query); + } +} diff --git a/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java b/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java index 40a7714d870..74c1091397e 100644 --- a/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java +++ b/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java @@ -29,7 +29,8 @@ import io.druid.query.Query; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = "timeBoundary", value = TimeBoundaryTieredBrokerSelectorStrategy.class), - @JsonSubTypes.Type(name = "priority", value = PriorityTieredBrokerSelectorStrategy.class) + @JsonSubTypes.Type(name = "priority", value = PriorityTieredBrokerSelectorStrategy.class), + @JsonSubTypes.Type(name = "javascript", value = JavascriptTieredBrokerSelectorStrategy.class) }) public interface TieredBrokerSelectorStrategy diff --git a/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java b/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java new file mode 100644 index 00000000000..78a6db3c59d --- /dev/null +++ b/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java @@ -0,0 +1,103 @@ +/* + * 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.router; + +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import io.druid.query.Druids; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import org.junit.Assert; +import org.junit.Test; + +import java.util.LinkedHashMap; + +public class JavascriptTieredBrokerSelectorStrategyTest +{ + + @Test + public void testGetBrokerServiceName() throws Exception + { + TieredBrokerSelectorStrategy jsStrategy = new JavascriptTieredBrokerSelectorStrategy( + "function (config, query) { if (config.getTierToBrokerMap().values().size() > 0 && query.getAggregatorSpecs && query.getAggregatorSpecs().size() <= 2) { return config.getTierToBrokerMap().values().toArray()[0] } else { return config.getDefaultBrokerServiceName() } }" + ); + + final LinkedHashMap tierBrokerMap = new LinkedHashMap<>(); + tierBrokerMap.put("fast", "druid/fastBroker"); + tierBrokerMap.put("slow", "druid/broker"); + + final TieredBrokerConfig tieredBrokerConfig = new TieredBrokerConfig() + { + @Override + public String getDefaultBrokerServiceName() + { + return "druid/broker"; + } + + @Override + public LinkedHashMap getTierToBrokerMap() + { + return tierBrokerMap; + } + }; + + final Druids.TimeseriesQueryBuilder queryBuilder = Druids.newTimeseriesQueryBuilder().dataSource("test") + .intervals("2014/2015") + .aggregators( + ImmutableList.of( + new CountAggregatorFactory("count") + ) + ); + + Assert.assertEquals( + Optional.of("druid/fastBroker"), + jsStrategy.getBrokerServiceName( + tieredBrokerConfig, + queryBuilder.build() + ) + ); + + + Assert.assertEquals( + Optional.of("druid/broker"), + jsStrategy.getBrokerServiceName( + tieredBrokerConfig, + Druids.newTimeBoundaryQueryBuilder().dataSource("test").bound("maxTime").build() + ) + ); + + Assert.assertEquals( + Optional.of("druid/broker"), + jsStrategy.getBrokerServiceName( + tieredBrokerConfig, + queryBuilder.aggregators( + ImmutableList.of( + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("longSum", "a"), + new DoubleSumAggregatorFactory("doubleSum", "b") + ) + ).build() + ) + ); + + } +} From b32ce31eff5beeca3e890b461ce88dacc1e13440 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 20 Oct 2014 14:11:49 -0700 Subject: [PATCH 47/53] add documentation --- docs/content/Router.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/docs/content/Router.md b/docs/content/Router.md index 73849164216..f046fe4b468 100644 --- a/docs/content/Router.md +++ b/docs/content/Router.md @@ -118,3 +118,16 @@ Including this strategy means all timeBoundary queries are always routed to the ``` Queries with a priority set to less than minPriority are routed to the lowest priority broker. Queries with priority set to greater than maxPriority are routed to the highest priority broker. By default, minPriority is 0 and maxPriority is 1. Using these default values, if a query with priority 0 (the default query priority is 0) is sent, the query skips the priority selection logic. + +### javascript + +Allows defining arbitrary routing rules using a JavaScript function. The function is passed the configuration and the query to be executed, and returns the tier it should be routed to, or null for the default tier. + +*Example*: a function that return the highest priority broker unless the given query has more than two aggregators. + +```json +{ + "type" : "javascript", + "function" : "function (config, query) { if (config.getTierToBrokerMap().values().size() > 0 && query.getAggregatorSpecs && query.getAggregatorSpecs().size() <= 2) { return config.getTierToBrokerMap().values().toArray()[0] } else { return config.getDefaultBrokerServiceName() } }" +} +``` From 23a59db566bc054b3231f7c027239f0239bdaebb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 20 Oct 2014 16:23:45 -0700 Subject: [PATCH 48/53] fix ordering for partitions with same start time, fixes #796 Partitions with same start time may produce results out of order, especially in the presence of data gaps. Results for those segments must be re-ordered prior to being merged --- .../druid/client/CachingClusteredClient.java | 27 ++++++--- .../client/CachingClusteredClientTest.java | 57 +++++++++++++++++++ 2 files changed, 75 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index d0135eb7f4a..f189e060670 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -25,7 +25,6 @@ import com.google.common.base.Function; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -261,14 +260,22 @@ public class CachingClusteredClient implements QueryRunner Ordering.natural().onResultOf(Pair.>lhsFn()) ); - final Sequence> seq = Sequences.simple( - Iterables.transform(listOfSequences, Pair.>rhsFn()) - ); - if (strategy == null) { - return toolChest.mergeSequences(seq); - } else { - return strategy.mergeSequences(seq); + final List> orderedSequences = Lists.newLinkedList(); + DateTime unorderedStart = null; + List> unordered = Lists.newLinkedList(); + for (Pair> sequencePair : listOfSequences) { + if (unorderedStart != null && unorderedStart.getMillis() != sequencePair.lhs.getMillis()) { + orderedSequences.add(toolChest.mergeSequencesUnordered(Sequences.simple(unordered))); + unordered = Lists.newLinkedList(); + } + unorderedStart = sequencePair.lhs; + unordered.add(sequencePair.rhs); } + if(!unordered.isEmpty()) { + orderedSequences.add(toolChest.mergeSequencesUnordered(Sequences.simple(unordered))); + } + + return toolChest.mergeSequences(Sequences.simple(orderedSequences)); } private void addSequencesFromCache(ArrayList>> listOfSequences) @@ -332,7 +339,9 @@ public class CachingClusteredClient implements QueryRunner if (!server.isAssignable() || !populateCache || isBySegment) { resultSeqToAdd = clientQueryable.run(query.withQuerySegmentSpec(segmentSpec)); } else { - resultSeqToAdd = toolChest.mergeSequences( + // this could be more efficient, since we only need to reorder results + // for batches of segments with the same segment start time. + resultSeqToAdd = toolChest.mergeSequencesUnordered( Sequences.map( clientQueryable.run(rewrittenQuery.withQuerySegmentSpec(segmentSpec)), new Function>() diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index f99f3d4346b..c59da0c9906 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -309,6 +309,63 @@ public class CachingClusteredClientTest ); } + @Test + public void testTimeseriesMergingOutOfOrderPartitions() throws Exception + { + final Druids.TimeseriesQueryBuilder builder = Druids.newTimeseriesQueryBuilder() + .dataSource(DATA_SOURCE) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .aggregators(AGGS) + .postAggregators(POST_AGGS) + .context(CONTEXT); + + QueryRunner runner = new FinalizeResultsQueryRunner(client, new TimeseriesQueryQueryToolChest(new QueryConfig())); + + testQueryCaching( + runner, + builder.build(), + new Interval("2011-01-05/2011-01-10"), + makeTimeResults( + new DateTime("2011-01-05T02"), 80, 100, + new DateTime("2011-01-06T02"), 420, 520, + new DateTime("2011-01-07T02"), 12, 2194, + new DateTime("2011-01-08T02"), 59, 201, + new DateTime("2011-01-09T02"), 181, 52 + ), + new Interval("2011-01-05/2011-01-10"), + makeTimeResults( + new DateTime("2011-01-05T00"), 85, 102, + new DateTime("2011-01-06T00"), 412, 521, + new DateTime("2011-01-07T00"), 122, 21894, + new DateTime("2011-01-08T00"), 5, 20, + new DateTime("2011-01-09T00"), 18, 521 + ) + ); + + TestHelper.assertExpectedResults( + makeRenamedTimeResults( + new DateTime("2011-01-05T00"), 85, 102, + new DateTime("2011-01-05T02"), 80, 100, + new DateTime("2011-01-06T00"), 412, 521, + new DateTime("2011-01-06T02"), 420, 520, + new DateTime("2011-01-07T00"), 122, 21894, + new DateTime("2011-01-07T02"), 12, 2194, + new DateTime("2011-01-08T00"), 5, 20, + new DateTime("2011-01-08T02"), 59, 201, + new DateTime("2011-01-09T00"), 18, 521, + new DateTime("2011-01-09T02"), 181, 52 + ), + runner.run( + builder.intervals("2011-01-05/2011-01-10") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS) + .build() + ) + ); + } + @Test @SuppressWarnings("unchecked") public void testTimeseriesCachingTimeZone() throws Exception From 9d5e65913da00f280c38424d70d0e7d9601129fe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 21 Oct 2014 14:09:11 -0700 Subject: [PATCH 49/53] fix serde --- docs/content/Aggregations.md | 2 +- ...avascriptTieredBrokerSelectorStrategy.java | 66 ++++++++++++++++--- .../router/TieredBrokerSelectorStrategy.java | 2 +- ...criptTieredBrokerSelectorStrategyTest.java | 24 +++++-- 4 files changed, 78 insertions(+), 16 deletions(-) diff --git a/docs/content/Aggregations.md b/docs/content/Aggregations.md index 29740a2858c..abd4780b025 100644 --- a/docs/content/Aggregations.md +++ b/docs/content/Aggregations.md @@ -159,4 +159,4 @@ Uses [HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf) to ```json { "type" : "hyperUnique", "name" : , "fieldName" : } -``` \ No newline at end of file +``` diff --git a/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java b/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java index cce7fb25554..629c6a6873e 100644 --- a/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java +++ b/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java @@ -19,9 +19,11 @@ package io.druid.server.router; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.api.client.repackaged.com.google.common.base.Throwables; import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; import io.druid.query.Query; import javax.script.Compilable; @@ -30,19 +32,29 @@ import javax.script.ScriptEngine; import javax.script.ScriptEngineManager; import javax.script.ScriptException; -public class JavascriptTieredBrokerSelectorStrategy implements TieredBrokerSelectorStrategy +public class JavaScriptTieredBrokerSelectorStrategy implements TieredBrokerSelectorStrategy { - private final SelectorFunction function; - - public JavascriptTieredBrokerSelectorStrategy(@JsonProperty("function") String function) + private static interface SelectorFunction { + public String apply(TieredBrokerConfig config, Query query); + } + + private final SelectorFunction fnSelector; + private final String function; + + @JsonCreator + public JavaScriptTieredBrokerSelectorStrategy(@JsonProperty("function") String fn) + { + Preconditions.checkNotNull(fn, "function must not be null"); + final ScriptEngine engine = new ScriptEngineManager().getEngineByName("javascript"); try { - ((Compilable)engine).compile("var apply = " + function).eval(); + ((Compilable)engine).compile("var apply = " + fn).eval(); } catch(ScriptException e) { Throwables.propagate(e); } - this.function = ((Invocable)engine).getInterface(SelectorFunction.class); + this.function = fn; + this.fnSelector = ((Invocable)engine).getInterface(SelectorFunction.class); } @Override @@ -50,11 +62,45 @@ public class JavascriptTieredBrokerSelectorStrategy implements TieredBrokerSelec TieredBrokerConfig config, Query query ) { - return Optional.fromNullable(function.apply(config, query)); + return Optional.fromNullable(fnSelector.apply(config, query)); } - private static interface SelectorFunction + @JsonProperty + public String getFunction() { - public String apply(TieredBrokerConfig config, Query query); + return function; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + JavaScriptTieredBrokerSelectorStrategy that = (JavaScriptTieredBrokerSelectorStrategy) o; + + if (!function.equals(that.function)) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + return function.hashCode(); + } + + @Override + public String toString() + { + return "JavascriptTieredBrokerSelectorStrategy{" + + "function='" + function + '\'' + + '}'; } } diff --git a/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java b/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java index 74c1091397e..10bffe4fb37 100644 --- a/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java +++ b/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java @@ -30,7 +30,7 @@ import io.druid.query.Query; @JsonSubTypes(value = { @JsonSubTypes.Type(name = "timeBoundary", value = TimeBoundaryTieredBrokerSelectorStrategy.class), @JsonSubTypes.Type(name = "priority", value = PriorityTieredBrokerSelectorStrategy.class), - @JsonSubTypes.Type(name = "javascript", value = JavascriptTieredBrokerSelectorStrategy.class) + @JsonSubTypes.Type(name = "javascript", value = JavaScriptTieredBrokerSelectorStrategy.class) }) public interface TieredBrokerSelectorStrategy diff --git a/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java b/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java index 78a6db3c59d..3f7e9f14947 100644 --- a/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java +++ b/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java @@ -19,8 +19,10 @@ package io.druid.server.router; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; +import io.druid.jackson.DefaultObjectMapper; import io.druid.query.Druids; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -31,15 +33,29 @@ import org.junit.Test; import java.util.LinkedHashMap; -public class JavascriptTieredBrokerSelectorStrategyTest +public class JavaScriptTieredBrokerSelectorStrategyTest { + final TieredBrokerSelectorStrategy jsStrategy = new JavaScriptTieredBrokerSelectorStrategy( + "function (config, query) { if (config.getTierToBrokerMap().values().size() > 0 && query.getAggregatorSpecs && query.getAggregatorSpecs().size() <= 2) { return config.getTierToBrokerMap().values().toArray()[0] } else { return config.getDefaultBrokerServiceName() } }" + ); + + @Test + public void testSerde() throws Exception + { + ObjectMapper mapper = new DefaultObjectMapper(); + Assert.assertEquals( + jsStrategy, + mapper.readValue( + mapper.writeValueAsString(jsStrategy), + JavaScriptTieredBrokerSelectorStrategy.class + ) + ); + } @Test public void testGetBrokerServiceName() throws Exception { - TieredBrokerSelectorStrategy jsStrategy = new JavascriptTieredBrokerSelectorStrategy( - "function (config, query) { if (config.getTierToBrokerMap().values().size() > 0 && query.getAggregatorSpecs && query.getAggregatorSpecs().size() <= 2) { return config.getTierToBrokerMap().values().toArray()[0] } else { return config.getDefaultBrokerServiceName() } }" - ); + final LinkedHashMap tierBrokerMap = new LinkedHashMap<>(); tierBrokerMap.put("fast", "druid/fastBroker"); From fd7d3e5aceabb568c37f7eb31db2a4f48b0dffe9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 21 Oct 2014 15:30:29 -0700 Subject: [PATCH 50/53] make it work with Java 8 --- .../server/router/JavascriptTieredBrokerSelectorStrategy.java | 2 +- .../router/JavascriptTieredBrokerSelectorStrategyTest.java | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java b/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java index 629c6a6873e..afc7997de43 100644 --- a/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java +++ b/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java @@ -34,7 +34,7 @@ import javax.script.ScriptException; public class JavaScriptTieredBrokerSelectorStrategy implements TieredBrokerSelectorStrategy { - private static interface SelectorFunction + public static interface SelectorFunction { public String apply(TieredBrokerConfig config, Query query); } diff --git a/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java b/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java index 3f7e9f14947..2b2162148ed 100644 --- a/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java +++ b/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java @@ -55,8 +55,6 @@ public class JavaScriptTieredBrokerSelectorStrategyTest @Test public void testGetBrokerServiceName() throws Exception { - - final LinkedHashMap tierBrokerMap = new LinkedHashMap<>(); tierBrokerMap.put("fast", "druid/fastBroker"); tierBrokerMap.put("slow", "druid/broker"); From 8735272ce39c35cfee3e79e2455b798fa6685855 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 21 Oct 2014 15:36:49 -0700 Subject: [PATCH 51/53] fix file casing --- ...rStrategy.java => JavaScriptTieredBrokerSelectorStrategy.java} | 0 ...yTest.java => JavaScriptTieredBrokerSelectorStrategyTest.java} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename server/src/main/java/io/druid/server/router/{JavascriptTieredBrokerSelectorStrategy.java => JavaScriptTieredBrokerSelectorStrategy.java} (100%) rename server/src/test/java/io/druid/server/router/{JavascriptTieredBrokerSelectorStrategyTest.java => JavaScriptTieredBrokerSelectorStrategyTest.java} (100%) diff --git a/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java b/server/src/main/java/io/druid/server/router/JavaScriptTieredBrokerSelectorStrategy.java similarity index 100% rename from server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java rename to server/src/main/java/io/druid/server/router/JavaScriptTieredBrokerSelectorStrategy.java diff --git a/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java b/server/src/test/java/io/druid/server/router/JavaScriptTieredBrokerSelectorStrategyTest.java similarity index 100% rename from server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java rename to server/src/test/java/io/druid/server/router/JavaScriptTieredBrokerSelectorStrategyTest.java From 2d96bc5f1feef16d682cc34764d59c049ae8b101 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 21 Oct 2014 16:08:56 -0700 Subject: [PATCH 52/53] more FAQ docs --- docs/content/Best-Practices.md | 5 +++ docs/content/Coordinator.md | 4 +-- docs/content/Ingestion-FAQ.md | 9 +++++ docs/content/Rule-Configuration.md | 55 ++++++++++++++++++++++++++---- 4 files changed, 64 insertions(+), 9 deletions(-) diff --git a/docs/content/Best-Practices.md b/docs/content/Best-Practices.md index f4cd8f32595..9688a52f966 100644 --- a/docs/content/Best-Practices.md +++ b/docs/content/Best-Practices.md @@ -21,3 +21,8 @@ SSDs are highly recommended for historical and real-time nodes if you are not ru Although Druid supports schemaless ingestion of dimensions, because of 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. This may require some ETL processing on your data however. +# 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 diff --git a/docs/content/Coordinator.md b/docs/content/Coordinator.md index 9021cbe1dff..42d1d17041d 100644 --- a/docs/content/Coordinator.md +++ b/docs/content/Coordinator.md @@ -20,9 +20,7 @@ io.druid.cli.Main server coordinator Rules ----- -Segments are loaded and dropped from the cluster based on a set of rules. Rules indicate how segments should be assigned to different historical node tiers and how many replicants of a segment should exist in each tier. Rules may also indicate when segments should be dropped entirely from the cluster. The coordinator loads a set of rules from the database. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The coordinator will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule. - -For more information on rules, see [Rule Configuration](Rule-Configuration.html). +Segments can be automatically loaded and dropped from the cluster based on a set of rules. For more information on rules, see [Rule Configuration](Rule-Configuration.html). Cleaning Up Segments -------------------- diff --git a/docs/content/Ingestion-FAQ.md b/docs/content/Ingestion-FAQ.md index fc83907c47a..ecf6b2ccdac 100644 --- a/docs/content/Ingestion-FAQ.md +++ b/docs/content/Ingestion-FAQ.md @@ -21,6 +21,12 @@ druid.storage.bucket=druid druid.storage.baseKey=sample ``` +Other common reasons that hand-off fails are as follows: + +1) Historical nodes are out of capacity and cannot download any more segments. You'll see exceptions in the coordinator logs if this occurs. +2) Segments are corrupt and cannot download. You'll see exceptions in your historical nodes if this occurs. +3) Deep storage is improperly configured. Make sure that your segment actually exists in deep storage and that the coordinator logs have no errors. + ## How do I get HDFS to work? Make sure to include the `druid-hdfs-storage` module as one of your extensions and set `druid.storage.type=hdfs`. @@ -50,6 +56,9 @@ To do this use the IngestSegmentFirehose and run an indexer task. The IngestSegm Typically the above will be run as a batch job to say everyday feed in a chunk of data and aggregate it. +## Real-time ingestion seems to be stuck + +There are a few ways this can occur. Druid will throttle ingestion to prevent out of memory problems if the intermediate persists are taking too long or if hand-off is taking too long. If your node logs indicate certain columns are taking a very long time to build (for example, if your segment granularity is hourly, but creating a single column takes 30 minutes), you should re-evaluate your configuration or scale up your real-time ingestion. ## More information diff --git a/docs/content/Rule-Configuration.md b/docs/content/Rule-Configuration.md index bf8b8a9792d..c25c9e62b70 100644 --- a/docs/content/Rule-Configuration.md +++ b/docs/content/Rule-Configuration.md @@ -2,12 +2,34 @@ layout: doc_page --- # Configuring Rules for Coordinator Nodes + +Rules indicate how segments should be assigned to different historical node tiers and how many replicas of a segment should exist in each tier. Rules may also indicate when segments should be dropped entirely from the cluster. The coordinator loads a set of rules from the metadata storage. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The coordinator will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule. + Note: It is recommended that the coordinator console is used to configure rules. However, the coordinator node does have HTTP endpoints to programmatically configure rules. + Load Rules ---------- -Load rules indicate how many replicants of a segment should exist in a server tier. +Load rules indicate how many replicas of a segment should exist in a server tier. + +### Forever Load Rule + +Forever load rules are of the form: + +```json +{ + "type" : "loadForever", + "tieredReplicants": { + "hot": 1, + "_default_tier" : 1 + } +} +``` + +* `type` - this should always be "loadByInterval" +* `tieredReplicants` - A JSON Object where the keys are the tier names and values are the number of replicas for that tier. + ### Interval Load Rule @@ -16,14 +38,17 @@ Interval load rules are of the form: ```json { "type" : "loadByInterval", - "interval" : "2012-01-01/2013-01-01", - "tier" : "hot" + "interval": "2012-01-01/2013-01-01", + "tieredReplicants": { + "hot": 1, + "_default_tier" : 1 + } } ``` * `type` - this should always be "loadByInterval" * `interval` - A JSON Object representing ISO-8601 Intervals -* `tier` - the configured historical node tier +* `tieredReplicants` - A JSON Object where the keys are the tier names and values are the number of replicas for that tier. ### Period Load Rule @@ -33,13 +58,16 @@ Period load rules are of the form: { "type" : "loadByPeriod", "period" : "P1M", - "tier" : "hot" + "tieredReplicants": { + "hot": 1, + "_default_tier" : 1 + } } ``` * `type` - this should always be "loadByPeriod" * `period` - A JSON Object representing ISO-8601 Periods -* `tier` - the configured historical node tier +* `tieredReplicants` - A JSON Object where the keys are the tier names and values are the number of replicas for that tier. The interval of a segment will be compared against the specified period. The rule matches if the period overlaps the interval. @@ -48,6 +76,21 @@ Drop Rules Drop rules indicate when segments should be dropped from the cluster. +### Forever Drop Rule + +Forever drop rules are of the form: + +```json +{ + "type" : "dropForever" +} +``` + +* `type` - this should always be "dropByPeriod" + +All segments that match this rule are dropped from the cluster. + + ### Interval Drop Rule Interval drop rules are of the form: From ee392b6064c7ec04783f348de068cf513249d65b Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 21 Oct 2014 16:26:17 -0700 Subject: [PATCH 53/53] more updates to docs --- docs/content/Ingestion-FAQ.md | 9 ++++++++- .../{Best-Practices.md => Recommendations.md} | 15 +++++++++++---- docs/content/index.md | 13 +------------ docs/content/toc.textile | 2 +- 4 files changed, 21 insertions(+), 18 deletions(-) rename docs/content/{Best-Practices.md => Recommendations.md} (53%) diff --git a/docs/content/Ingestion-FAQ.md b/docs/content/Ingestion-FAQ.md index ecf6b2ccdac..972e62a6a23 100644 --- a/docs/content/Ingestion-FAQ.md +++ b/docs/content/Ingestion-FAQ.md @@ -1,6 +1,11 @@ --- layout: doc_page --- + +## What types of data does Druid support? + +Druid can ingest JSON, CSV, TSV and other delimited data out of the box. Druid supports single dimension values, or multiple dimension values (an array of strings). Druid supports long and float numeric columns. + ## Where do my Druid segments end up after ingestion? Depending on what `druid.storage.type` is set to, Druid will upload segments to some [Deep Storage](Deep-Storage.html). Local disk is used as the default deep storage. @@ -24,7 +29,9 @@ druid.storage.baseKey=sample Other common reasons that hand-off fails are as follows: 1) Historical nodes are out of capacity and cannot download any more segments. You'll see exceptions in the coordinator logs if this occurs. + 2) Segments are corrupt and cannot download. You'll see exceptions in your historical nodes if this occurs. + 3) Deep storage is improperly configured. Make sure that your segment actually exists in deep storage and that the coordinator logs have no errors. ## How do I get HDFS to work? @@ -41,7 +48,7 @@ You can check the coordinator console located at `:/cluste ## My queries are returning empty results -You can check `:/druid/v2/datasources/?interval=0/3000` for the dimensions and metrics that have been created for your datasource. Make sure that the name of the aggregators you use in your query match one of these metrics. Also make sure that the query interval you specify match a valid time range where data exists. Note: the broker endpoint will only return valid results on historical segments. +You can check `:/druid/v2/datasources/?interval=0/3000` for the dimensions and metrics that have been created for your datasource. Make sure that the name of the aggregators you use in your query match one of these metrics. Also make sure that the query interval you specify match a valid time range where data exists. Note: the broker endpoint will only return valid results on historical segments and not segments served by real-time nodes. ## How can I Reindex existing data in Druid with schema changes? diff --git a/docs/content/Best-Practices.md b/docs/content/Recommendations.md similarity index 53% rename from docs/content/Best-Practices.md rename to docs/content/Recommendations.md index 9688a52f966..bf764ffe6c2 100644 --- a/docs/content/Best-Practices.md +++ b/docs/content/Recommendations.md @@ -2,8 +2,8 @@ layout: doc_page --- -Best Practices -============== +Recommendations +=============== # Use UTC Timezone @@ -17,12 +17,19 @@ Druid is not perfect in how it handles mix-cased dimension and metric names. Thi 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 for Best Results +# Provide Columns Names in Lexicographic Order -Although Druid supports schemaless ingestion of dimensions, because of 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. This may require some ETL processing on your data however. +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. + + +# 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. # 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 diff --git a/docs/content/index.md b/docs/content/index.md index 529a2325436..3c236cc81be 100644 --- a/docs/content/index.md +++ b/docs/content/index.md @@ -37,17 +37,6 @@ When Druid? * You want to do your analysis on data as it’s happening (in real-time) * You need a data store that is always available, 24x7x365, and years into the future. - -Not Druid? ----------- - -* The amount of data you have can easily be handled by MySQL -* You're querying for individual entries or doing lookups (not analytics) -* Batch ingestion is good enough -* Canned queries are good enough -* Downtime is no big deal - - Druid vs… ---------- @@ -60,7 +49,7 @@ Druid vs… About This Page ---------- -The data store world is vast, confusing and constantly in flux. This page is meant to help potential evaluators decide whether Druid is a good fit for the problem one needs to solve. If anything about it is incorrect please provide that feedback on the mailing list or via some other means so we can fix it. +The data infrastructure world is vast, confusing and constantly in flux. This page is meant to help potential evaluators decide whether Druid is a good fit for the problem one needs to solve. If anything about it is incorrect please provide that feedback on the mailing list or via some other means so we can fix it. diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 21f867bca36..29c08ab4d0e 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -19,7 +19,7 @@ h2. Booting a Druid Cluster * "Production Cluster Configuration":Production-Cluster-Configuration.html * "Production Hadoop Configuration":Hadoop-Configuration.html * "Rolling Cluster Updates":Rolling-Updates.html -* "Best Practices":Best-Practices.html +* "Recommendations":Recommendations.html h2. Configuration * "Common Configuration":Configuration.html