From d15cd8fde7fc762f5c52c68d6b01516992c2cdcd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 26 Aug 2014 17:15:24 -0700 Subject: [PATCH 01/46] add some comments about string interning --- .../java/io/druid/segment/incremental/IncrementalIndex.java | 4 ++++ .../segment/incremental/IncrementalIndexStorageAdapter.java | 6 +++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 4f6f6a0f67f..ca521a4f1f9 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -673,6 +673,10 @@ 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) + */ 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 1c452496ffb..55e283fbb0d 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -525,6 +525,10 @@ 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} + */ if (id == dimVal) { return true; } @@ -597,4 +601,4 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter }; } } -} \ No newline at end of file +} From 4a23cd94eff96b9a66283e8c58b02ebaca695a29 Mon Sep 17 00:00:00 2001 From: pdeva Date: Sun, 14 Sep 2014 11:22:36 -0700 Subject: [PATCH 02/46] Corrected documentation for IngestSegmentFirehose Ingest Segment firehose had incorrect documentation. --- docs/content/Firehose.md | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/docs/content/Firehose.md b/docs/content/Firehose.md index ebe15b3632a..065ba2472c1 100644 --- a/docs/content/Firehose.md +++ b/docs/content/Firehose.md @@ -50,9 +50,7 @@ A sample ingest firehose spec is shown below - { "type" : "ingestSegment", "dataSource" : "wikipedia", - "interval" : "2013-01-01/2013-01-02", - "dimensions":[], - "metrics":[] + "interval" : "2013-01-01/2013-01-02" } ``` @@ -61,8 +59,8 @@ A sample ingest firehose spec is shown below - |type|ingestSegment. Type of firehose|yes| |dataSource|A String defining the data source to fetch rows from, very similar to a table in a relational database|yes| |interval|A String representing ISO-8601 Interval. This defines the time range to fetch the data over.|yes| -|dimensions|The list of dimensions to select. If left empty, all dimensions are selected.|no| -|metrics|The list of metrics to select. If left empty, all metrics are returned.|no| +|dimensions|The list of dimensions to select. If left empty, no dimensions are returned. If left null or not defined, all dimensions are returned. |no| +|metrics|The list of metrics to select. If left empty, no metrics are returned. If left null or not defined, all metrics are selected.|no| |filter| See [Filters](Filters.html)|yes| From cfa92e82172edb1642de99c0ff44411f3fb51885 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 15 Sep 2014 12:13:52 -0700 Subject: [PATCH 03/46] fix incorrect nullable annotations --- .../main/java/io/druid/collections/OrderedMergeIterator.java | 4 ++-- .../main/java/io/druid/collections/OrderedMergeSequence.java | 2 +- common/src/main/java/io/druid/common/utils/JodaUtils.java | 2 +- .../main/java/io/druid/indexing/common/task/AppendTask.java | 2 +- .../java/io/druid/indexing/common/task/MergeTaskBase.java | 2 +- .../druid/indexing/firehose/IngestSegmentFirehoseFactory.java | 4 ++-- .../main/java/io/druid/query/IntervalChunkingQueryRunner.java | 2 +- processing/src/main/java/io/druid/query/Queries.java | 2 +- .../src/main/java/io/druid/query/QueryRunnerHelper.java | 2 +- server/src/main/java/io/druid/server/http/DBResource.java | 4 ++-- .../src/main/java/io/druid/server/http/ServersResource.java | 2 +- 11 files changed, 14 insertions(+), 14 deletions(-) diff --git a/common/src/main/java/io/druid/collections/OrderedMergeIterator.java b/common/src/main/java/io/druid/collections/OrderedMergeIterator.java index 7cfc72adefc..a1214379e84 100644 --- a/common/src/main/java/io/druid/collections/OrderedMergeIterator.java +++ b/common/src/main/java/io/druid/collections/OrderedMergeIterator.java @@ -75,7 +75,7 @@ public class OrderedMergeIterator implements Iterator new Predicate>() { @Override - public boolean apply(@Nullable Iterator input) + public boolean apply(Iterator input) { return input.hasNext(); } @@ -85,7 +85,7 @@ public class OrderedMergeIterator implements Iterator new Function, PeekingIterator>() { @Override - public PeekingIterator apply(@Nullable Iterator input) + public PeekingIterator apply(Iterator input) { return Iterators.peekingIterator(input); } diff --git a/common/src/main/java/io/druid/collections/OrderedMergeSequence.java b/common/src/main/java/io/druid/collections/OrderedMergeSequence.java index 00b1f12d3de..c24404c9a57 100644 --- a/common/src/main/java/io/druid/collections/OrderedMergeSequence.java +++ b/common/src/main/java/io/druid/collections/OrderedMergeSequence.java @@ -83,7 +83,7 @@ public class OrderedMergeSequence implements Sequence new Function, T>() { @Override - public T apply(@Nullable Yielder input) + public T apply(Yielder input) { return input.get(); } diff --git a/common/src/main/java/io/druid/common/utils/JodaUtils.java b/common/src/main/java/io/druid/common/utils/JodaUtils.java index b488f1e5a7d..db6da04e875 100644 --- a/common/src/main/java/io/druid/common/utils/JodaUtils.java +++ b/common/src/main/java/io/druid/common/utils/JodaUtils.java @@ -91,7 +91,7 @@ public class JodaUtils intervals, new Predicate() { @Override - public boolean apply(@Nullable Interval input) + public boolean apply(Interval input) { return input.overlaps(i); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java index 40807073874..d2a3c93594c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java @@ -97,7 +97,7 @@ public class AppendTask extends MergeTaskBase new Predicate() { @Override - public boolean apply(@Nullable Rowboat input) + public boolean apply(Rowboat input) { return holder.getInterval().contains(input.getTimestamp()); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java index 29749501227..7e66991e188 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java @@ -133,7 +133,7 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask new Function() { @Override - public String apply(@Nullable DataSegment input) + public String apply(DataSegment input) { return input.getIdentifier(); } diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index 6987a24e97f..ca496d346ba 100644 --- a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -236,7 +236,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory apply(@Nullable StorageAdapter adapter) + public Sequence apply(StorageAdapter adapter) { return Sequences.concat( Sequences.map( @@ -248,7 +248,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory apply(@Nullable final Cursor cursor) + public Sequence apply(final Cursor cursor) { final TimestampColumnSelector timestampColumnSelector = cursor.makeTimestampColumnSelector(); diff --git a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java index 0bf4c3928b0..df4c8596217 100644 --- a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java @@ -61,7 +61,7 @@ public class IntervalChunkingQueryRunner implements QueryRunner new Function>() { @Override - public Iterable apply(@Nullable Interval input) + public Iterable apply(Interval input) { return splitInterval(input); } diff --git a/processing/src/main/java/io/druid/query/Queries.java b/processing/src/main/java/io/druid/query/Queries.java index 3fd1f802189..7691a8f7eb4 100644 --- a/processing/src/main/java/io/druid/query/Queries.java +++ b/processing/src/main/java/io/druid/query/Queries.java @@ -49,7 +49,7 @@ public class Queries new Function() { @Override - public String apply(@Nullable AggregatorFactory input) + public String apply(AggregatorFactory input) { return input.getName(); } diff --git a/processing/src/main/java/io/druid/query/QueryRunnerHelper.java b/processing/src/main/java/io/druid/query/QueryRunnerHelper.java index 2765cf0e35c..d9c2e147dda 100644 --- a/processing/src/main/java/io/druid/query/QueryRunnerHelper.java +++ b/processing/src/main/java/io/druid/query/QueryRunnerHelper.java @@ -71,7 +71,7 @@ public class QueryRunnerHelper new Function>() { @Override - public Result apply(@Nullable Cursor input) + public Result apply(Cursor input) { log.debug("Running over cursor[%s]", adapter.getInterval(), input.getTime()); return mapFn.apply(input); diff --git a/server/src/main/java/io/druid/server/http/DBResource.java b/server/src/main/java/io/druid/server/http/DBResource.java index 02277f9e79d..24c96d2b8b9 100644 --- a/server/src/main/java/io/druid/server/http/DBResource.java +++ b/server/src/main/java/io/druid/server/http/DBResource.java @@ -74,7 +74,7 @@ public class DBResource new Function() { @Override - public String apply(@Nullable DruidDataSource dataSource) + public String apply(DruidDataSource dataSource) { return dataSource.getName(); } @@ -126,7 +126,7 @@ public class DBResource new Function() { @Override - public Object apply(@Nullable DataSegment segment) + public Object apply(DataSegment segment) { return segment.getIdentifier(); } diff --git a/server/src/main/java/io/druid/server/http/ServersResource.java b/server/src/main/java/io/druid/server/http/ServersResource.java index ed43c340e8d..3a7b325d202 100644 --- a/server/src/main/java/io/druid/server/http/ServersResource.java +++ b/server/src/main/java/io/druid/server/http/ServersResource.java @@ -158,7 +158,7 @@ public class ServersResource new Function() { @Override - public String apply(@Nullable DataSegment segment) + public String apply(DataSegment segment) { return segment.getIdentifier(); } From d3f04d4fbe04e0444200dd1d59a878b11501cd57 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 15 Sep 2014 12:43:57 -0700 Subject: [PATCH 04/46] update druid api ver --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 50cb60eab37..80f3fad6004 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.6 2.6.0 - 0.2.8 + 0.2.9 From d4795ce9272f2fb71cb79bd91278d98fbf501328 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 15 Sep 2014 12:15:11 -0700 Subject: [PATCH 05/46] fix missing charsets --- .../io/druid/examples/web/WebJsonSupplier.java | 3 ++- .../io/druid/granularity/PeriodGranularity.java | 3 ++- .../aggregation/DoubleSumAggregatorFactory.java | 3 ++- .../aggregation/HistogramAggregatorFactory.java | 2 +- .../aggregation/LongSumAggregatorFactory.java | 3 ++- .../query/aggregation/MaxAggregatorFactory.java | 3 ++- .../query/aggregation/MinAggregatorFactory.java | 3 ++- .../query/dimension/DefaultDimensionSpec.java | 3 ++- .../query/dimension/ExtractionDimensionSpec.java | 3 ++- .../extraction/JavascriptDimExtractionFn.java | 3 ++- .../query/extraction/PartialDimExtractionFn.java | 3 ++- .../query/extraction/RegexDimExtractionFn.java | 3 ++- .../query/extraction/TimeDimExtractionFn.java | 3 ++- .../druid/query/filter/ExtractionDimFilter.java | 5 +++-- .../io/druid/query/filter/SelectorDimFilter.java | 5 +++-- .../io/druid/query/filter/SpatialDimFilter.java | 3 ++- .../query/groupby/having/EqualToHavingSpec.java | 3 ++- .../groupby/having/GreaterThanHavingSpec.java | 3 ++- .../query/groupby/having/LessThanHavingSpec.java | 3 ++- .../query/groupby/orderby/OrderByColumnSpec.java | 5 +++-- .../query/search/SearchQueryQueryToolChest.java | 3 ++- .../search/search/FragmentSearchQuerySpec.java | 3 ++- .../InsensitiveContainsSearchQuerySpec.java | 3 ++- .../java/io/druid/query/select/PagingSpec.java | 3 ++- .../query/select/SelectQueryQueryToolChest.java | 5 +++-- .../rabbitmq/RabbitMQFirehoseFactory.java | 3 ++- .../src/main/java/io/druid/client/CacheUtil.java | 3 ++- .../io/druid/initialization/Initialization.java | 8 +++++++- .../io/druid/server/log/FileRequestLogger.java | 16 ++++++++++++---- .../main/java/io/druid/server/sql/SQLRunner.java | 2 +- 30 files changed, 77 insertions(+), 37 deletions(-) diff --git a/examples/src/main/java/io/druid/examples/web/WebJsonSupplier.java b/examples/src/main/java/io/druid/examples/web/WebJsonSupplier.java index 9372f721145..39d0fae1882 100644 --- a/examples/src/main/java/io/druid/examples/web/WebJsonSupplier.java +++ b/examples/src/main/java/io/druid/examples/web/WebJsonSupplier.java @@ -20,6 +20,7 @@ package io.druid.examples.web; import com.google.api.client.repackaged.com.google.common.base.Throwables; +import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import com.google.common.io.InputSupplier; import com.metamx.emitter.EmittingLogger; @@ -55,6 +56,6 @@ public class WebJsonSupplier implements InputSupplier { URLConnection connection = url.openConnection(); connection.setDoInput(true); - return new BufferedReader(new InputStreamReader(url.openStream())); + return new BufferedReader(new InputStreamReader(url.openStream(), Charsets.UTF_8)); } } diff --git a/processing/src/main/java/io/druid/granularity/PeriodGranularity.java b/processing/src/main/java/io/druid/granularity/PeriodGranularity.java index aa8b41439a3..2aef4ff4044 100644 --- a/processing/src/main/java/io/druid/granularity/PeriodGranularity.java +++ b/processing/src/main/java/io/druid/granularity/PeriodGranularity.java @@ -21,6 +21,7 @@ package io.druid.granularity; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import org.joda.time.Chronology; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; @@ -325,7 +326,7 @@ public class PeriodGranularity extends BaseQueryGranularity @Override public byte[] cacheKey() { - return (period.toString() + ":" + chronology.getZone().toString()).getBytes(); + return (period.toString() + ":" + chronology.getZone().toString()).getBytes(Charsets.UTF_8); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java index 083a16589dc..713022efade 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java @@ -21,6 +21,7 @@ package io.druid.query.aggregation; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import com.google.common.primitives.Doubles; import io.druid.segment.ColumnSelectorFactory; @@ -129,7 +130,7 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory @Override public byte[] getCacheKey() { - byte[] fieldNameBytes = fieldName.getBytes(); + byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8); return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); } diff --git a/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java index 422d8279a2e..5a99ac9d18c 100644 --- a/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java @@ -154,7 +154,7 @@ public class HistogramAggregatorFactory implements AggregatorFactory @Override public byte[] getCacheKey() { - byte[] fieldNameBytes = fieldName.getBytes(); + byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8); return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); } diff --git a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java index 963d9458beb..249d37045a0 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java @@ -21,6 +21,7 @@ package io.druid.query.aggregation; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import com.google.common.primitives.Longs; import io.druid.segment.ColumnSelectorFactory; @@ -125,7 +126,7 @@ public class LongSumAggregatorFactory implements AggregatorFactory @Override public byte[] getCacheKey() { - byte[] fieldNameBytes = fieldName.getBytes(); + byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8); return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); } diff --git a/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java index 56e6de5f56e..23c8b066c25 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java @@ -21,6 +21,7 @@ package io.druid.query.aggregation; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import com.google.common.primitives.Doubles; import io.druid.segment.ColumnSelectorFactory; @@ -126,7 +127,7 @@ public class MaxAggregatorFactory implements AggregatorFactory @Override public byte[] getCacheKey() { - byte[] fieldNameBytes = fieldName.getBytes(); + byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8); return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); } diff --git a/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java index e7256d0ccb9..859ed2b23cd 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java @@ -21,6 +21,7 @@ package io.druid.query.aggregation; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import com.google.common.primitives.Doubles; import io.druid.segment.ColumnSelectorFactory; @@ -126,7 +127,7 @@ public class MinAggregatorFactory implements AggregatorFactory @Override public byte[] getCacheKey() { - byte[] fieldNameBytes = fieldName.getBytes(); + byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8); return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); } diff --git a/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java index ee5ee674293..cb44fe59c6c 100644 --- a/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java @@ -21,6 +21,7 @@ package io.druid.query.dimension; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import io.druid.query.extraction.DimExtractionFn; import java.nio.ByteBuffer; @@ -68,7 +69,7 @@ public class DefaultDimensionSpec implements DimensionSpec @Override public byte[] getCacheKey() { - byte[] dimensionBytes = dimension.getBytes(); + byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8); return ByteBuffer.allocate(1 + dimensionBytes.length) .put(CACHE_TYPE_ID) diff --git a/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java index 878de5bbad5..c692f1399fb 100644 --- a/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java @@ -21,6 +21,7 @@ package io.druid.query.dimension; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import io.druid.query.extraction.DimExtractionFn; import java.nio.ByteBuffer; @@ -73,7 +74,7 @@ public class ExtractionDimensionSpec implements DimensionSpec @Override public byte[] getCacheKey() { - byte[] dimensionBytes = dimension.getBytes(); + byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8); byte[] dimExtractionFnBytes = dimExtractionFn.getCacheKey(); return ByteBuffer.allocate(1 + dimensionBytes.length + dimExtractionFnBytes.length) diff --git a/processing/src/main/java/io/druid/query/extraction/JavascriptDimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/JavascriptDimExtractionFn.java index 3935e07c293..e8099e643a4 100644 --- a/processing/src/main/java/io/druid/query/extraction/JavascriptDimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/JavascriptDimExtractionFn.java @@ -21,6 +21,7 @@ package io.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import com.google.common.base.Function; import org.mozilla.javascript.Context; import org.mozilla.javascript.ContextFactory; @@ -80,7 +81,7 @@ public class JavascriptDimExtractionFn implements DimExtractionFn @Override public byte[] getCacheKey() { - byte[] bytes = function.getBytes(); + byte[] bytes = function.getBytes(Charsets.UTF_8); return ByteBuffer.allocate(1 + bytes.length) .put(CACHE_TYPE_ID) .put(bytes) diff --git a/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java index 8b6a3b58017..323c7989387 100644 --- a/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java @@ -21,6 +21,7 @@ package io.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import java.nio.ByteBuffer; import java.util.regex.Matcher; @@ -47,7 +48,7 @@ public class PartialDimExtractionFn implements DimExtractionFn @Override public byte[] getCacheKey() { - byte[] exprBytes = expr.getBytes(); + byte[] exprBytes = expr.getBytes(Charsets.UTF_8); return ByteBuffer.allocate(1 + exprBytes.length) .put(CACHE_TYPE_ID) .put(exprBytes) diff --git a/processing/src/main/java/io/druid/query/extraction/RegexDimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/RegexDimExtractionFn.java index 50a418afed3..636507c1273 100644 --- a/processing/src/main/java/io/druid/query/extraction/RegexDimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/RegexDimExtractionFn.java @@ -21,6 +21,7 @@ package io.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import java.nio.ByteBuffer; import java.util.regex.Matcher; @@ -47,7 +48,7 @@ public class RegexDimExtractionFn implements DimExtractionFn @Override public byte[] getCacheKey() { - byte[] exprBytes = expr.getBytes(); + byte[] exprBytes = expr.getBytes(Charsets.UTF_8); return ByteBuffer.allocate(1 + exprBytes.length) .put(CACHE_TYPE_ID) .put(exprBytes) diff --git a/processing/src/main/java/io/druid/query/extraction/TimeDimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/TimeDimExtractionFn.java index 215f8c32aee..25c7bde0cfa 100644 --- a/processing/src/main/java/io/druid/query/extraction/TimeDimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/TimeDimExtractionFn.java @@ -21,6 +21,7 @@ package io.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import com.ibm.icu.text.SimpleDateFormat; import java.nio.ByteBuffer; @@ -55,7 +56,7 @@ public class TimeDimExtractionFn implements DimExtractionFn @Override public byte[] getCacheKey() { - byte[] timeFormatBytes = timeFormat.getBytes(); + byte[] timeFormatBytes = timeFormat.getBytes(Charsets.UTF_8); return ByteBuffer.allocate(1 + timeFormatBytes.length) .put(CACHE_TYPE_ID) .put(timeFormatBytes) diff --git a/processing/src/main/java/io/druid/query/filter/ExtractionDimFilter.java b/processing/src/main/java/io/druid/query/filter/ExtractionDimFilter.java index 8ce26f71f99..65fa0e4494c 100644 --- a/processing/src/main/java/io/druid/query/filter/ExtractionDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/ExtractionDimFilter.java @@ -21,6 +21,7 @@ package io.druid.query.filter; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import io.druid.query.extraction.DimExtractionFn; @@ -71,8 +72,8 @@ public class ExtractionDimFilter implements DimFilter @Override public byte[] getCacheKey() { - byte[] dimensionBytes = dimension.getBytes(); - byte[] valueBytes = value.getBytes(); + byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8); + byte[] valueBytes = value.getBytes(Charsets.UTF_8); return ByteBuffer.allocate(1 + dimensionBytes.length + valueBytes.length) .put(DimFilterCacheHelper.EXTRACTION_CACHE_ID) diff --git a/processing/src/main/java/io/druid/query/filter/SelectorDimFilter.java b/processing/src/main/java/io/druid/query/filter/SelectorDimFilter.java index b8d1580a169..414b475d1fe 100644 --- a/processing/src/main/java/io/druid/query/filter/SelectorDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/SelectorDimFilter.java @@ -21,6 +21,7 @@ package io.druid.query.filter; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import java.nio.ByteBuffer; @@ -47,8 +48,8 @@ public class SelectorDimFilter implements DimFilter @Override public byte[] getCacheKey() { - byte[] dimensionBytes = dimension.getBytes(); - byte[] valueBytes = (value == null) ? new byte[]{} : value.getBytes(); + byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8); + byte[] valueBytes = (value == null) ? new byte[]{} : value.getBytes(Charsets.UTF_8); return ByteBuffer.allocate(1 + dimensionBytes.length + valueBytes.length) .put(DimFilterCacheHelper.SELECTOR_CACHE_ID) diff --git a/processing/src/main/java/io/druid/query/filter/SpatialDimFilter.java b/processing/src/main/java/io/druid/query/filter/SpatialDimFilter.java index 2abcc9282b3..7148ebf55e7 100644 --- a/processing/src/main/java/io/druid/query/filter/SpatialDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/SpatialDimFilter.java @@ -20,6 +20,7 @@ package io.druid.query.filter; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import com.metamx.collections.spatial.search.Bound; @@ -48,7 +49,7 @@ public class SpatialDimFilter implements DimFilter @Override public byte[] getCacheKey() { - byte[] dimBytes = dimension.getBytes(); + byte[] dimBytes = dimension.getBytes(Charsets.UTF_8); byte[] boundBytes = bound.getCacheKey(); return ByteBuffer.allocate(1 + dimBytes.length + boundBytes.length) diff --git a/processing/src/main/java/io/druid/query/groupby/having/EqualToHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/EqualToHavingSpec.java index bc26524e60b..b0389edfca7 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/EqualToHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/EqualToHavingSpec.java @@ -21,6 +21,7 @@ package io.druid.query.groupby.having; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import com.google.common.primitives.Bytes; import io.druid.data.input.Row; @@ -71,7 +72,7 @@ public class EqualToHavingSpec implements HavingSpec @Override public byte[] getCacheKey() { - final byte[] aggBytes = aggregationName.getBytes(); + final byte[] aggBytes = aggregationName.getBytes(Charsets.UTF_8); final byte[] valBytes = Bytes.toArray(Arrays.asList(value)); return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length) .put(CACHE_KEY) diff --git a/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java index 68a8113bf55..19349a2098b 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java @@ -21,6 +21,7 @@ package io.druid.query.groupby.having; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import com.google.common.primitives.Bytes; import io.druid.data.input.Row; @@ -71,7 +72,7 @@ public class GreaterThanHavingSpec implements HavingSpec @Override public byte[] getCacheKey() { - final byte[] aggBytes = aggregationName.getBytes(); + final byte[] aggBytes = aggregationName.getBytes(Charsets.UTF_8); final byte[] valBytes = Bytes.toArray(Arrays.asList(value)); return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length) .put(CACHE_KEY) diff --git a/processing/src/main/java/io/druid/query/groupby/having/LessThanHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/LessThanHavingSpec.java index 4ee782734f2..284ddc7042c 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/LessThanHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/LessThanHavingSpec.java @@ -20,6 +20,7 @@ package io.druid.query.groupby.having; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import com.google.common.primitives.Bytes; import io.druid.data.input.Row; @@ -69,7 +70,7 @@ public class LessThanHavingSpec implements HavingSpec @Override public byte[] getCacheKey() { - final byte[] aggBytes = aggregationName.getBytes(); + final byte[] aggBytes = aggregationName.getBytes(Charsets.UTF_8); final byte[] valBytes = Bytes.toArray(Arrays.asList(value)); return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length) .put(CACHE_KEY) diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/OrderByColumnSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/OrderByColumnSpec.java index efb91a9c3c0..2c471d62507 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/OrderByColumnSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/OrderByColumnSpec.java @@ -21,6 +21,7 @@ package io.druid.query.groupby.orderby; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; @@ -182,8 +183,8 @@ public class OrderByColumnSpec public byte[] getCacheKey() { - final byte[] dimensionBytes = dimension.getBytes(); - final byte[] directionBytes = direction.name().getBytes(); + final byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8); + final byte[] directionBytes = direction.name().getBytes(Charsets.UTF_8); return ByteBuffer.allocate(dimensionBytes.length + directionBytes.length) .put(dimensionBytes) diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index 558d7ee3f13..74fc8d55f17 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -20,6 +20,7 @@ package io.druid.query.search; import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Joiner; @@ -172,7 +173,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest entry : pagingIdentifiers.entrySet()) { - pagingKeys[index] = entry.getKey().getBytes(); + pagingKeys[index] = entry.getKey().getBytes(Charsets.UTF_8); pagingValues[index] = ByteBuffer.allocate(Ints.BYTES).putInt(entry.getValue()).array(); pagingKeysSize += pagingKeys[index].length; pagingValuesSize += Ints.BYTES; diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java index 345aeff1d5b..6f08110f781 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -21,6 +21,7 @@ package io.druid.query.select; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Joiner; @@ -173,7 +174,7 @@ public class SelectQueryQueryToolChest extends QueryToolChest Date: Mon, 15 Sep 2014 12:16:31 -0700 Subject: [PATCH 06/46] make constants final --- indexing-hadoop/src/main/java/io/druid/indexer/Bucket.java | 2 +- .../main/java/io/druid/indexing/common/config/TaskConfig.java | 2 +- processing/src/main/java/io/druid/query/BaseQuery.java | 2 +- server/src/main/java/io/druid/client/cache/CacheConfig.java | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/Bucket.java b/indexing-hadoop/src/main/java/io/druid/indexer/Bucket.java index ed164a676a9..ddb3455b8de 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/Bucket.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/Bucket.java @@ -26,7 +26,7 @@ import java.nio.ByteBuffer; public class Bucket { - public static int PREAMBLE_BYTES = 16; + public static final int PREAMBLE_BYTES = 16; /** ID for this bucket, unique for this indexer run. Used for grouping and partitioning. */ private final int shardNum; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java index 1671fb84bd3..a688d2603f2 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java @@ -28,7 +28,7 @@ import java.util.List; public class TaskConfig { - public static List DEFAULT_DEFAULT_HADOOP_COORDINATES = ImmutableList.of( + public static final List DEFAULT_DEFAULT_HADOOP_COORDINATES = ImmutableList.of( "org.apache.hadoop:hadoop-client:2.3.0" ); diff --git a/processing/src/main/java/io/druid/query/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java index e2052f3ba64..a648f484c78 100644 --- a/processing/src/main/java/io/druid/query/BaseQuery.java +++ b/processing/src/main/java/io/druid/query/BaseQuery.java @@ -36,7 +36,7 @@ import java.util.Map; */ public abstract class BaseQuery implements Query { - public static String QUERYID = "queryId"; + public static final String QUERYID = "queryId"; private final DataSource dataSource; private final Map context; private final QuerySegmentSpec querySegmentSpec; diff --git a/server/src/main/java/io/druid/client/cache/CacheConfig.java b/server/src/main/java/io/druid/client/cache/CacheConfig.java index 26c5f383f2d..25c397b7a83 100644 --- a/server/src/main/java/io/druid/client/cache/CacheConfig.java +++ b/server/src/main/java/io/druid/client/cache/CacheConfig.java @@ -27,8 +27,8 @@ import java.util.List; public class CacheConfig { - public static String USE_CACHE = "useCache"; - public static String POPULATE_CACHE = "populateCache"; + public static final String USE_CACHE = "useCache"; + public static final String POPULATE_CACHE = "populateCache"; @JsonProperty private boolean useCache = true; From 137ad50bf14be2c3b17312571914bcf305724938 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 15 Sep 2014 12:16:56 -0700 Subject: [PATCH 07/46] classes that should be static --- .../main/java/io/druid/indexing/common/task/AppendTask.java | 2 +- .../main/java/io/druid/query/groupby/GroupByQueryEngine.java | 4 ++-- .../main/java/io/druid/server/coordinator/LoadQueuePeon.java | 4 ++-- .../main/java/io/druid/server/log/EmittingRequestLogger.java | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java index d2a3c93594c..51fb358ac8c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java @@ -115,7 +115,7 @@ public class AppendTask extends MergeTaskBase return "append"; } - private class SegmentToMergeHolder + private static class SegmentToMergeHolder { private final DataSegment segment; private final Interval interval; diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java index a3cc06c0b9a..341090f3176 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java @@ -236,7 +236,7 @@ public class GroupByQueryEngine } } - private class PositionMaintainer + private static class PositionMaintainer { private final int[] increments; private final int increment; @@ -284,7 +284,7 @@ public class GroupByQueryEngine } } - private class RowIterator implements CloseableIterator + private static class RowIterator implements CloseableIterator { private final GroupByQuery query; private final Cursor cursor; diff --git a/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java b/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java index 7a591862d11..73cdad0726a 100644 --- a/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java +++ b/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java @@ -384,7 +384,7 @@ public class LoadQueuePeon } } - private class SegmentHolder + private static class SegmentHolder { private final DataSegment segment; private final DataSegmentChangeRequest changeRequest; @@ -457,4 +457,4 @@ public class LoadQueuePeon return changeRequest.toString(); } } -} \ No newline at end of file +} diff --git a/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java b/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java index 798978d6d2f..21a8bde1599 100644 --- a/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java +++ b/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java @@ -125,7 +125,7 @@ public class EmittingRequestLogger implements RequestLogger } } - private class RequestLogEventBuilder implements ServiceEventBuilder + private static class RequestLogEventBuilder implements ServiceEventBuilder { private final String feed; private final RequestLogLine requestLogLine; From c8b8e3f6e9dbdad83ad2fc0972380e3dfd60be01 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 15 Sep 2014 12:17:32 -0700 Subject: [PATCH 08/46] negating compare is bad --- .../main/java/io/druid/indexing/overlord/RemoteTaskRunner.java | 2 +- .../main/java/io/druid/query/PrioritizedExecutorService.java | 2 +- .../client/selector/HighestPriorityTierSelectorStrategy.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index a36e6381678..3d8de3c7001 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 @@ -799,7 +799,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer ZkWorker zkWorker, ZkWorker zkWorker2 ) { - int retVal = -Ints.compare(zkWorker.getCurrCapacityUsed(), zkWorker2.getCurrCapacityUsed()); + int retVal = Ints.compare(zkWorker2.getCurrCapacityUsed(), zkWorker.getCurrCapacityUsed()); if (retVal == 0) { retVal = zkWorker.getWorker().getHost().compareTo(zkWorker2.getWorker().getHost()); } diff --git a/processing/src/main/java/io/druid/query/PrioritizedExecutorService.java b/processing/src/main/java/io/druid/query/PrioritizedExecutorService.java index c0dd98b6718..b1a0463d7b0 100644 --- a/processing/src/main/java/io/druid/query/PrioritizedExecutorService.java +++ b/processing/src/main/java/io/druid/query/PrioritizedExecutorService.java @@ -249,7 +249,7 @@ public class PrioritizedExecutorService extends AbstractExecutorService implemen @Override public int compareTo(PrioritizedListenableFutureTask otherTask) { - return -Ints.compare(getPriority(), otherTask.getPriority()); + return Ints.compare(otherTask.getPriority(), getPriority()); } } } diff --git a/server/src/main/java/io/druid/client/selector/HighestPriorityTierSelectorStrategy.java b/server/src/main/java/io/druid/client/selector/HighestPriorityTierSelectorStrategy.java index 13995c610f5..db0226e59cd 100644 --- a/server/src/main/java/io/druid/client/selector/HighestPriorityTierSelectorStrategy.java +++ b/server/src/main/java/io/druid/client/selector/HighestPriorityTierSelectorStrategy.java @@ -34,7 +34,7 @@ public class HighestPriorityTierSelectorStrategy extends AbstractTierSelectorStr @Override public int compare(Integer o1, Integer o2) { - return -Ints.compare(o1, o2); + return Ints.compare(o2, o1); } }; From 0c35fe2cf428278cdab234cd57fd14a012860542 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 15 Sep 2014 12:17:45 -0700 Subject: [PATCH 09/46] properly close reader --- .../src/main/java/io/druid/guice/PropertiesModule.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/guice/PropertiesModule.java b/processing/src/main/java/io/druid/guice/PropertiesModule.java index bd0868e2414..2314134894b 100644 --- a/processing/src/main/java/io/druid/guice/PropertiesModule.java +++ b/processing/src/main/java/io/druid/guice/PropertiesModule.java @@ -33,6 +33,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; +import java.io.Reader; import java.util.Properties; /** @@ -68,8 +69,8 @@ public class PropertiesModule implements Module if (stream != null) { log.info("Loading properties from %s", propertiesFile); - try { - fileProps.load(new InputStreamReader(stream, Charsets.UTF_8)); + try(Reader reader = new InputStreamReader(stream, Charsets.UTF_8)) { + fileProps.load(reader); } catch (IOException e) { throw Throwables.propagate(e); From 0712c666cbfa5d12cbe0300c4182d2159878ecd8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 15 Sep 2014 12:18:01 -0700 Subject: [PATCH 10/46] null check for something that cannot be null --- .../src/main/java/io/druid/server/http/DatasourcesResource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/server/http/DatasourcesResource.java b/server/src/main/java/io/druid/server/http/DatasourcesResource.java index f99586c1056..4203c372ef5 100644 --- a/server/src/main/java/io/druid/server/http/DatasourcesResource.java +++ b/server/src/main/java/io/druid/server/http/DatasourcesResource.java @@ -261,7 +261,7 @@ public class DatasourcesResource final DruidDataSource dataSource = getDataSource(dataSourceName.toLowerCase()); final Interval theInterval = new Interval(interval.replace("_", "/")); - if (dataSource == null || interval == null) { + if (dataSource == null) { return Response.noContent().build(); } From d501b052ea3e7a314a0b1c7c887e85224b6cf548 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 15 Sep 2014 13:02:47 -0700 Subject: [PATCH 11/46] remove unused columnConfig --- .../java/io/druid/indexer/HadoopDruidIndexerConfig.java | 7 ------- 1 file changed, 7 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 1ff6e5a4482..e6402221e45 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -170,7 +170,6 @@ public class HadoopDruidIndexerConfig private volatile HadoopIngestionSpec schema; private volatile PathSpec pathSpec; - private volatile ColumnConfig columnConfig; private volatile Map shardSpecLookups = Maps.newHashMap(); private volatile Map hadoopShardSpecLookup = Maps.newHashMap(); @@ -179,7 +178,6 @@ public class HadoopDruidIndexerConfig final @JsonProperty("schema") HadoopIngestionSpec schema ) { - this.columnConfig = columnConfig; this.schema = schema; this.pathSpec = jsonMapper.convertValue(schema.getIOConfig().getPathSpec(), PathSpec.class); for (Map.Entry> entry : schema.getTuningConfig().getShardSpecs().entrySet()) { @@ -213,11 +211,6 @@ public class HadoopDruidIndexerConfig return schema; } - public ColumnConfig getColumnConfig() - { - return columnConfig; - } - public String getDataSource() { return schema.getDataSchema().getDataSource(); From 02892233acd262547da37fd4d95c52c418df0da3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 15 Sep 2014 13:56:31 -0700 Subject: [PATCH 12/46] fix synchronized on field getting updated --- .../io/druid/segment/realtime/plumber/Sink.java | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java index 04e485793c1..8b4afa7904b 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java @@ -50,6 +50,7 @@ public class Sink implements Iterable private static final Logger log = new Logger(Sink.class); private volatile FireHydrant currHydrant; + private final transient Object hydrantLock = new Object(); private final Interval interval; private final DataSchema schema; @@ -117,7 +118,7 @@ public class Sink implements Iterable throw new IAE("No currHydrant but given row[%s]", row); } - synchronized (currHydrant) { + synchronized (hydrantLock) { IncrementalIndex index = currHydrant.getIndex(); if (index == null) { return -1; // the hydrant was swapped without being replaced @@ -128,7 +129,7 @@ public class Sink implements Iterable public boolean isEmpty() { - synchronized (currHydrant) { + synchronized (hydrantLock) { return hydrants.size() == 1 && currHydrant.getIndex().isEmpty(); } } @@ -145,7 +146,7 @@ public class Sink implements Iterable public boolean swappable() { - synchronized (currHydrant) { + synchronized (hydrantLock) { return currHydrant.getIndex() != null && currHydrant.getIndex().size() != 0; } } @@ -185,17 +186,11 @@ public class Sink implements Iterable .build() ); - FireHydrant old; - if (currHydrant == null) { // Only happens on initialization, cannot synchronize on null + final FireHydrant old; + synchronized (hydrantLock) { old = currHydrant; currHydrant = new FireHydrant(newIndex, hydrants.size(), getSegment().getIdentifier()); hydrants.add(currHydrant); - } else { - synchronized (currHydrant) { - old = currHydrant; - currHydrant = new FireHydrant(newIndex, hydrants.size(), getSegment().getIdentifier()); - hydrants.add(currHydrant); - } } return old; From d674e611de72e8d7e65bb44ffdd4552ae3ee42e6 Mon Sep 17 00:00:00 2001 From: pdeva Date: Mon, 15 Sep 2014 14:41:57 -0700 Subject: [PATCH 13/46] described how to enable experimental features --- docs/content/About-Experimental-Features.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docs/content/About-Experimental-Features.md b/docs/content/About-Experimental-Features.md index 5fe74527bc3..e99dbc60828 100644 --- a/docs/content/About-Experimental-Features.md +++ b/docs/content/About-Experimental-Features.md @@ -3,3 +3,12 @@ layout: doc_page --- # About Experimental Features Experimental features are features we have developed but have not fully tested in a production environment. If you choose to try them out, there will likely be edge cases that we have not covered. We would love feedback on any of these features, whether they are bug reports, suggestions for improvement, or letting us know they work as intended. + + +To enable experimental features, include their artifacts in the configuration runtime.properties file. Eg- + +``` +druid.extensions.coordinates=["io.druid.extensions:druid-histogram:{VERSION}"] +``` + +The configuration for all the indexer and query nodes need to be updated with this. From 1ab266277288bed015e0673fa5e58f1bc7d43b2d Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 15 Sep 2014 16:52:07 -0700 Subject: [PATCH 14/46] have better dimension exclusions --- pom.xml | 2 +- .../io/druid/segment/indexing/DataSchema.java | 26 ++++++++++++++++++- 2 files changed, 26 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 80f3fad6004..50cb60eab37 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.6 2.6.0 - 0.2.9 + 0.2.8 diff --git a/server/src/main/java/io/druid/segment/indexing/DataSchema.java b/server/src/main/java/io/druid/segment/indexing/DataSchema.java index b0b4d074a8d..63e2377412b 100644 --- a/server/src/main/java/io/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/io/druid/segment/indexing/DataSchema.java @@ -21,11 +21,14 @@ package io.druid.segment.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.api.client.util.Sets; import io.druid.data.input.impl.InputRowParser; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import java.util.Set; + /** */ public class DataSchema @@ -44,7 +47,28 @@ public class DataSchema ) { this.dataSource = dataSource; - this.parser = parser; + + final Set dimensionExclusions = Sets.newHashSet(); + if (parser != null && parser.getParseSpec() != null && parser.getParseSpec().getTimestampSpec() != null) { + dimensionExclusions.add(parser.getParseSpec().getTimestampSpec().getTimestampColumn()); + for (AggregatorFactory aggregator : aggregators) { + dimensionExclusions.add(aggregator.getName()); + } + } + + if (parser != null && parser.getParseSpec() != null && parser.getParseSpec().getDimensionsSpec() != null) { + this.parser = parser.withParseSpec( + parser.getParseSpec() + .withDimensionsSpec( + parser.getParseSpec() + .getDimensionsSpec() + .withDimensionExclusions(dimensionExclusions) + ) + ); + } else { + this.parser = parser; + } + this.aggregators = aggregators; this.granularitySpec = granularitySpec == null ? new UniformGranularitySpec(null, null, null, null) From 34ae426ffaef3a24d3974735f0e9a264be918d2c Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 15 Sep 2014 17:08:02 -0700 Subject: [PATCH 15/46] address cr --- .../io/druid/segment/indexing/DataSchema.java | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/io/druid/segment/indexing/DataSchema.java b/server/src/main/java/io/druid/segment/indexing/DataSchema.java index 63e2377412b..726112de62f 100644 --- a/server/src/main/java/io/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/io/druid/segment/indexing/DataSchema.java @@ -49,22 +49,23 @@ public class DataSchema this.dataSource = dataSource; final Set dimensionExclusions = Sets.newHashSet(); + for (AggregatorFactory aggregator : aggregators) { + dimensionExclusions.add(aggregator.getName()); + } if (parser != null && parser.getParseSpec() != null && parser.getParseSpec().getTimestampSpec() != null) { dimensionExclusions.add(parser.getParseSpec().getTimestampSpec().getTimestampColumn()); - for (AggregatorFactory aggregator : aggregators) { - dimensionExclusions.add(aggregator.getName()); + if (parser.getParseSpec().getDimensionsSpec() != null) { + this.parser = parser.withParseSpec( + parser.getParseSpec() + .withDimensionsSpec( + parser.getParseSpec() + .getDimensionsSpec() + .withDimensionExclusions(dimensionExclusions) + ) + ); + } else { + this.parser = parser; } - } - - if (parser != null && parser.getParseSpec() != null && parser.getParseSpec().getDimensionsSpec() != null) { - this.parser = parser.withParseSpec( - parser.getParseSpec() - .withDimensionsSpec( - parser.getParseSpec() - .getDimensionsSpec() - .withDimensionExclusions(dimensionExclusions) - ) - ); } else { this.parser = parser; } From f2d6360d4d6148b40bef6011c88b9f814310c844 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 15 Sep 2014 17:08:45 -0700 Subject: [PATCH 16/46] cleanup --- .../src/main/java/io/druid/segment/indexing/DataSchema.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/io/druid/segment/indexing/DataSchema.java b/server/src/main/java/io/druid/segment/indexing/DataSchema.java index 726112de62f..fb5a60af1e1 100644 --- a/server/src/main/java/io/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/io/druid/segment/indexing/DataSchema.java @@ -52,8 +52,10 @@ public class DataSchema for (AggregatorFactory aggregator : aggregators) { dimensionExclusions.add(aggregator.getName()); } - if (parser != null && parser.getParseSpec() != null && parser.getParseSpec().getTimestampSpec() != null) { - dimensionExclusions.add(parser.getParseSpec().getTimestampSpec().getTimestampColumn()); + if (parser != null && parser.getParseSpec() != null) { + if (parser.getParseSpec().getTimestampSpec() != null) { + dimensionExclusions.add(parser.getParseSpec().getTimestampSpec().getTimestampColumn()); + } if (parser.getParseSpec().getDimensionsSpec() != null) { this.parser = parser.withParseSpec( parser.getParseSpec() From 77dbac373928243eea06b183bfee3dd522f5ea5a Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 15 Sep 2014 17:10:34 -0700 Subject: [PATCH 17/46] fix import --- server/src/main/java/io/druid/segment/indexing/DataSchema.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/segment/indexing/DataSchema.java b/server/src/main/java/io/druid/segment/indexing/DataSchema.java index fb5a60af1e1..3013c9728f7 100644 --- a/server/src/main/java/io/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/io/druid/segment/indexing/DataSchema.java @@ -21,7 +21,7 @@ package io.druid.segment.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.api.client.util.Sets; +import com.google.common.collect.Sets; import io.druid.data.input.impl.InputRowParser; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.indexing.granularity.GranularitySpec; From f1e3518f37bd0399246a638adc3790ab07d3a2ca Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 16 Sep 2014 00:07:23 -0700 Subject: [PATCH 18/46] initialize segment view before announcing broker --- .../client/BatchServerInventoryView.java | 6 + .../io/druid/client/BrokerServerView.java | 6 + .../io/druid/client/ServerInventoryView.java | 16 +++ .../main/java/io/druid/client/ServerView.java | 8 ++ .../client/SingleServerInventoryView.java | 6 + .../inventory/CuratorInventoryManager.java | 112 ++++++++++++++---- .../CuratorInventoryManagerStrategy.java | 1 + .../coordination/broker/DruidBroker.java | 67 +++++++++++ .../CuratorInventoryManagerTest.java | 7 ++ .../src/main/java/io/druid/cli/CliBroker.java | 5 +- 10 files changed, 206 insertions(+), 28 deletions(-) create mode 100644 server/src/main/java/io/druid/server/coordination/broker/DruidBroker.java diff --git a/server/src/main/java/io/druid/client/BatchServerInventoryView.java b/server/src/main/java/io/druid/client/BatchServerInventoryView.java index a2a3c422aa1..e15e85e399a 100644 --- a/server/src/main/java/io/druid/client/BatchServerInventoryView.java +++ b/server/src/main/java/io/druid/client/BatchServerInventoryView.java @@ -174,6 +174,12 @@ public class BatchServerInventoryView extends ServerInventoryView implements ServerView, { return removeInnerInventory(container, inventoryKey); } + + @Override + public void inventoryInitialized() + { + log.info("Inventory Initialized"); + runSegmentCallbacks( + new Function() + { + @Override + public CallbackAction apply(SegmentCallback input) + { + return input.segmentViewInitialized(); + } + } + ); + } } ); } diff --git a/server/src/main/java/io/druid/client/ServerView.java b/server/src/main/java/io/druid/client/ServerView.java index 67ad13efab4..0eb6e392d46 100644 --- a/server/src/main/java/io/druid/client/ServerView.java +++ b/server/src/main/java/io/druid/client/ServerView.java @@ -91,6 +91,8 @@ public interface ServerView * should remain registered. */ public CallbackAction segmentRemoved(DruidServerMetadata server, DataSegment segment); + + public CallbackAction segmentViewInitialized(); } public static abstract class BaseSegmentCallback implements SegmentCallback @@ -106,5 +108,11 @@ public interface ServerView { return CallbackAction.CONTINUE; } + + @Override + public CallbackAction segmentViewInitialized() + { + return CallbackAction.CONTINUE; + } } } diff --git a/server/src/main/java/io/druid/client/SingleServerInventoryView.java b/server/src/main/java/io/druid/client/SingleServerInventoryView.java index 4acdbabf786..4f1d91bbf61 100644 --- a/server/src/main/java/io/druid/client/SingleServerInventoryView.java +++ b/server/src/main/java/io/druid/client/SingleServerInventoryView.java @@ -138,6 +138,12 @@ public class SingleServerInventoryView extends ServerInventoryView return action; } } + + @Override + public CallbackAction segmentViewInitialized() + { + return callback.segmentViewInitialized(); + } } ); } diff --git a/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java index d32bdd147f3..d61265b501c 100644 --- a/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java +++ b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java @@ -23,6 +23,7 @@ import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.MapMaker; +import com.google.common.collect.Sets; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; @@ -37,6 +38,7 @@ import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; import org.apache.curator.utils.ZKPaths; import java.io.IOException; +import java.util.Set; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicReference; @@ -62,6 +64,7 @@ public class CuratorInventoryManager private final CuratorInventoryManagerStrategy strategy; private final ConcurrentMap containers; + private final Set uninitializedInventory; private final PathChildrenCacheFactory cacheFactory; private volatile PathChildrenCache childrenCache; @@ -78,6 +81,7 @@ public class CuratorInventoryManager this.strategy = strategy; this.containers = new MapMaker().makeMap(); + this.uninitializedInventory = Sets.newConcurrentHashSet(); this.cacheFactory = new SimplePathChildrenCacheFactory(true, true, new ShutdownNowIgnoringExecutorService(exec)); } @@ -96,7 +100,7 @@ public class CuratorInventoryManager childrenCache.getListenable().addListener(new ContainerCacheListener()); try { - childrenCache.start(); + childrenCache.start(PathChildrenCache.StartMode.POST_INITIALIZED_EVENT); } catch (Exception e) { synchronized (lock) { @@ -165,6 +169,7 @@ public class CuratorInventoryManager { private final AtomicReference container; private final PathChildrenCache cache; + private boolean initialized = false; ContainerHolder( ContainerClass container, @@ -193,21 +198,19 @@ public class CuratorInventoryManager private class ContainerCacheListener implements PathChildrenCacheListener { + private volatile boolean containersInitialized = false; + private volatile boolean doneInitializing = false; + @Override public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception { - final ChildData child = event.getData(); - if (child == null) { - return; - } - - final String containerKey = ZKPaths.getNodeFromPath(child.getPath()); - final ContainerClass container; - switch (event.getType()) { case CHILD_ADDED: synchronized (lock) { - container = strategy.deserializeContainer(child.getData()); + final ChildData child = event.getData(); + final String containerKey = ZKPaths.getNodeFromPath(child.getPath()); + + final ContainerClass container = strategy.deserializeContainer(child.getData()); // This would normally be a race condition, but the only thing that should be mutating the containers // map is this listener, which should never run concurrently. If the same container is going to disappear @@ -222,7 +225,7 @@ public class CuratorInventoryManager containers.put(containerKey, new ContainerHolder(container, inventoryCache)); log.info("Starting inventory cache for %s, inventoryPath %s", containerKey, inventoryPath); - inventoryCache.start(); + inventoryCache.start(PathChildrenCache.StartMode.POST_INITIALIZED_EVENT); strategy.newContainer(container); } @@ -230,6 +233,9 @@ public class CuratorInventoryManager } case CHILD_REMOVED: synchronized (lock) { + final ChildData child = event.getData(); + final String containerKey = ZKPaths.getNodeFromPath(child.getPath()); + final ContainerHolder removed = containers.remove(containerKey); if (removed == null) { log.warn("Container[%s] removed that wasn't a container!?", child.getPath()); @@ -243,11 +249,19 @@ public class CuratorInventoryManager removed.getCache().close(); strategy.deadContainer(removed.getContainer()); + // also remove node from uninitilized, in case a nodes gets removed while we are starting up + synchronized (removed) { + markInventoryInitialized(removed); + } + break; } case CHILD_UPDATED: synchronized (lock) { - container = strategy.deserializeContainer(child.getData()); + final ChildData child = event.getData(); + final String containerKey = ZKPaths.getNodeFromPath(child.getPath()); + + final ContainerClass container = strategy.deserializeContainer(child.getData()); ContainerHolder oldContainer = containers.get(containerKey); if (oldContainer == null) { @@ -260,6 +274,41 @@ public class CuratorInventoryManager break; } + case INITIALIZED: + synchronized (lock) { + // must await initialized of all containerholders + for(ContainerHolder holder : containers.values()) { + synchronized (holder) { + if(!holder.initialized) { + uninitializedInventory.add(holder); + } + } + } + containersInitialized = true; + maybeDoneInitializing(); + break; + } + } + } + + // must be run in synchronized(lock) { synchronized(holder) { ... } } block + private void markInventoryInitialized(final ContainerHolder holder) + { + holder.initialized = true; + uninitializedInventory.remove(holder); + maybeDoneInitializing(); + } + + private void maybeDoneInitializing() + { + if(doneInitializing) { + return; + } + + // only fire if we are done initializing the parent PathChildrenCache + if(containersInitialized && uninitializedInventory.isEmpty()) { + doneInitializing = true; + strategy.inventoryInitialized(); } } @@ -279,30 +328,28 @@ public class CuratorInventoryManager @Override public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception { - final ChildData child = event.getData(); - - if (child == null) { - return; - } - final ContainerHolder holder = containers.get(containerKey); - if (holder == null) { return; } - final String inventoryKey = ZKPaths.getNodeFromPath(child.getPath()); - switch (event.getType()) { - case CHILD_ADDED: + case CHILD_ADDED: { + final ChildData child = event.getData(); + final String inventoryKey = ZKPaths.getNodeFromPath(child.getPath()); + final InventoryClass addedInventory = strategy.deserializeInventory(child.getData()); synchronized (holder) { holder.setContainer(strategy.addInventory(holder.getContainer(), inventoryKey, addedInventory)); } - break; - case CHILD_UPDATED: + } + + case CHILD_UPDATED: { + final ChildData child = event.getData(); + final String inventoryKey = ZKPaths.getNodeFromPath(child.getPath()); + final InventoryClass updatedInventory = strategy.deserializeInventory(child.getData()); synchronized (holder) { @@ -310,11 +357,26 @@ public class CuratorInventoryManager } break; - case CHILD_REMOVED: + } + + case CHILD_REMOVED: { + final ChildData child = event.getData(); + final String inventoryKey = ZKPaths.getNodeFromPath(child.getPath()); + synchronized (holder) { holder.setContainer(strategy.removeInventory(holder.getContainer(), inventoryKey)); } + break; + } + case INITIALIZED: + // make sure to acquire locks in (lock -> holder) order + synchronized (lock) { + synchronized (holder) { + markInventoryInitialized(holder); + } + } + break; } } diff --git a/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManagerStrategy.java b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManagerStrategy.java index 785e35534df..1e53ff6e8c3 100644 --- a/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManagerStrategy.java +++ b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManagerStrategy.java @@ -35,4 +35,5 @@ public interface CuratorInventoryManagerStrategy public ContainerClass addInventory(ContainerClass container, String inventoryKey, InventoryClass inventory); public ContainerClass updateInventory(ContainerClass container, String inventoryKey, InventoryClass inventory); public ContainerClass removeInventory(ContainerClass container, String inventoryKey); + public void inventoryInitialized(); } diff --git a/server/src/main/java/io/druid/server/coordination/broker/DruidBroker.java b/server/src/main/java/io/druid/server/coordination/broker/DruidBroker.java new file mode 100644 index 00000000000..ab44dbe8f72 --- /dev/null +++ b/server/src/main/java/io/druid/server/coordination/broker/DruidBroker.java @@ -0,0 +1,67 @@ +package io.druid.server.coordination.broker; + +import com.google.common.util.concurrent.MoreExecutors; +import com.google.inject.Inject; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import io.druid.client.ServerInventoryView; +import io.druid.client.ServerView; +import io.druid.curator.discovery.ServiceAnnouncer; +import io.druid.guice.ManageLifecycle; +import io.druid.guice.annotations.Self; +import io.druid.server.DruidNode; + +@ManageLifecycle +public class DruidBroker +{ + private final DruidNode self; + private final ServiceAnnouncer serviceAnnouncer; + private volatile boolean started = false; + + @Inject + public DruidBroker( + final ServerInventoryView serverInventoryView, + final @Self DruidNode self, + final ServiceAnnouncer serviceAnnouncer + ) + { + this.self = self; + this.serviceAnnouncer = serviceAnnouncer; + + serverInventoryView.registerSegmentCallback( + MoreExecutors.sameThreadExecutor(), + new ServerView.BaseSegmentCallback() + { + @Override + public ServerView.CallbackAction segmentViewInitialized() + { + serviceAnnouncer.announce(self); + return ServerView.CallbackAction.UNREGISTER; + } + } + ); + } + + @LifecycleStart + public void start() + { + synchronized (self) { + if(started) { + return; + } + started = true; + } + } + + @LifecycleStop + public void stop() + { + synchronized (self) { + if (!started) { + return; + } + serviceAnnouncer.unannounce(self); + started = false; + } + } +} diff --git a/server/src/test/java/io/druid/curator/inventory/CuratorInventoryManagerTest.java b/server/src/test/java/io/druid/curator/inventory/CuratorInventoryManagerTest.java index e325b2c59df..18b921fc827 100644 --- a/server/src/test/java/io/druid/curator/inventory/CuratorInventoryManagerTest.java +++ b/server/src/test/java/io/druid/curator/inventory/CuratorInventoryManagerTest.java @@ -176,6 +176,7 @@ public class CuratorInventoryManagerTest extends io.druid.curator.CuratorTestBas private volatile CountDownLatch deadContainerLatch = null; private volatile CountDownLatch newInventoryLatch = null; private volatile CountDownLatch deadInventoryLatch = null; + private volatile boolean initialized = false; @Override public Map deserializeContainer(byte[] bytes) @@ -271,5 +272,11 @@ public class CuratorInventoryManagerTest extends io.druid.curator.CuratorTestBas { this.deadInventoryLatch = deadInventoryLatch; } + + @Override + public void inventoryInitialized() + { + initialized = true; + } } } diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index f23102bccdd..f70ea8837ad 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -34,19 +34,18 @@ import io.druid.client.cache.CacheProvider; import io.druid.client.selector.CustomTierSelectorStrategyConfig; import io.druid.client.selector.ServerSelectorStrategy; import io.druid.client.selector.TierSelectorStrategy; -import io.druid.curator.discovery.DiscoveryModule; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.guice.LifecycleModule; import io.druid.guice.ManageLifecycle; -import io.druid.guice.annotations.Self; import io.druid.query.MapQueryToolChestWarehouse; import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChestWarehouse; import io.druid.server.ClientInfoResource; import io.druid.server.ClientQuerySegmentWalker; import io.druid.server.QueryResource; +import io.druid.server.coordination.broker.DruidBroker; import io.druid.server.initialization.JettyServerInitializer; import io.druid.server.metrics.MetricsModule; import org.eclipse.jetty.server.Server; @@ -95,8 +94,8 @@ public class CliBroker extends ServerRunnable Jerseys.addResource(binder, QueryResource.class); Jerseys.addResource(binder, ClientInfoResource.class); LifecycleModule.register(binder, QueryResource.class); + LifecycleModule.register(binder, DruidBroker.class); - DiscoveryModule.register(binder, Self.class); MetricsModule.register(binder, CacheMonitor.class); LifecycleModule.register(binder, Server.class); From f006de8639371e844451db9024bcf4c8823b45b6 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Tue, 16 Sep 2014 22:12:36 +0530 Subject: [PATCH 19/46] fix #732 fix metric list discovery --- .../druid/indexing/firehose/IngestSegmentFirehoseFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index ca496d346ba..94adbc0f4f1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -177,7 +177,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory metricsSet = new HashSet<>(); for (TimelineObjectHolder timelineObjectHolder : timeLineSegments) { - metricsSet.addAll(timelineObjectHolder.getObject().getChunk(0).getObject().getDimensions()); + metricsSet.addAll(timelineObjectHolder.getObject().getChunk(0).getObject().getMetrics()); } metricsList = Lists.newArrayList(metricsSet); } From 762485a0c9655f7599720e67d3759d3fc9650637 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 16 Sep 2014 09:59:25 -0700 Subject: [PATCH 20/46] remove unnecessary transient --- .../src/main/java/io/druid/segment/realtime/plumber/Sink.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java index 8b4afa7904b..cb95217a23a 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java @@ -50,7 +50,7 @@ public class Sink implements Iterable private static final Logger log = new Logger(Sink.class); private volatile FireHydrant currHydrant; - private final transient Object hydrantLock = new Object(); + private final Object hydrantLock = new Object(); private final Interval interval; private final DataSchema schema; From 9e0a2fd487d3b2386d78c7379e35cdf1994da3ce Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Sep 2014 14:01:21 -0700 Subject: [PATCH 21/46] fix select for non existant dimensions and metrics --- .../druid/query/select/SelectQueryEngine.java | 28 ++++-- .../query/select/SelectQueryRunnerTest.java | 99 +++++++++++++++++++ 2 files changed, 118 insertions(+), 9 deletions(-) diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java b/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java index 432d311b453..9ef4f3c1258 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java @@ -115,24 +115,34 @@ public class SelectQueryEngine for (Map.Entry dimSelector : dimSelectors.entrySet()) { final String dim = dimSelector.getKey(); final DimensionSelector selector = dimSelector.getValue(); - final IndexedInts vals = selector.getRow(); - if (vals.size() == 1) { - final String dimVal = selector.lookupName(vals.get(0)); - theEvent.put(dim, dimVal); + if (selector == null) { + theEvent.put(dim, null); } else { - List dimVals = Lists.newArrayList(); - for (int i = 0; i < vals.size(); ++i) { - dimVals.add(selector.lookupName(vals.get(i))); + final IndexedInts vals = selector.getRow(); + + if (vals.size() == 1) { + final String dimVal = selector.lookupName(vals.get(0)); + theEvent.put(dim, dimVal); + } else { + List dimVals = Lists.newArrayList(); + for (int i = 0; i < vals.size(); ++i) { + dimVals.add(selector.lookupName(vals.get(i))); + } + theEvent.put(dim, dimVals); } - theEvent.put(dim, dimVals); } } for (Map.Entry metSelector : metSelectors.entrySet()) { final String metric = metSelector.getKey(); final ObjectColumnSelector selector = metSelector.getValue(); - theEvent.put(metric, selector.get()); + + if (selector == null) { + theEvent.put(metric, null); + } else { + theEvent.put(metric, selector.get()); + } } builder.addEntry( diff --git a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java index 8831c095cc5..0f8df1962dc 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java @@ -30,6 +30,8 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; import io.druid.query.TableDataSource; +import io.druid.query.filter.AndDimFilter; +import io.druid.query.filter.DimFilter; import io.druid.query.filter.SelectorDimFilter; import io.druid.query.spec.LegacySegmentSpec; import org.joda.time.DateTime; @@ -358,6 +360,103 @@ public class SelectQueryRunnerTest verify(expectedResults, results); } + @Test + public void testFullSelectNoResults() + { + SelectQuery query = new SelectQuery( + new TableDataSource(QueryRunnerTestHelper.dataSource), + new LegacySegmentSpec(new Interval("2011-01-12/2011-01-14")), + new AndDimFilter( + Arrays.asList( + new SelectorDimFilter(QueryRunnerTestHelper.providerDimension, "spot"), + new SelectorDimFilter(QueryRunnerTestHelper.providerDimension, "foo") + ) + ), + QueryRunnerTestHelper.allGran, + Lists.newArrayList(), + Lists.newArrayList(), + new PagingSpec(null, 3), + null + ); + + Iterable> results = Sequences.toList( + runner.run(query), + Lists.>newArrayList() + ); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SelectResultValue( + ImmutableMap.of(), + Lists.newArrayList() + ) + ) + ); + + verify(expectedResults, results); + } + + + @Test + public void testFullSelectNoDimensionAndMetric() + { + SelectQuery query = new SelectQuery( + new TableDataSource(QueryRunnerTestHelper.dataSource), + new LegacySegmentSpec(new Interval("2011-01-12/2011-01-14")), + null, + QueryRunnerTestHelper.allGran, + Lists.newArrayList("foo"), + Lists.newArrayList("foo2"), + new PagingSpec(null, 3), + null + ); + + Iterable> results = Sequences.toList( + runner.run(query), + Lists.>newArrayList() + ); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SelectResultValue( + ImmutableMap.of(QueryRunnerTestHelper.segmentId, 2), + Arrays.asList( + new EventHolder( + QueryRunnerTestHelper.segmentId, + 0, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z")) + .put("foo", null) + .put("foo2", null) + .build() + ), + new EventHolder( + QueryRunnerTestHelper.segmentId, + 1, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z")) + .put(providerLowercase, "spot") + .put(QueryRunnerTestHelper.indexMetric, 100.000000F) + .build() + ), + new EventHolder( + QueryRunnerTestHelper.segmentId, + 2, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z")) + .put(providerLowercase, "spot") + .put(QueryRunnerTestHelper.indexMetric, 100.000000F) + .build() + ) + ) + ) + ); + + verify(expectedResults, results); + } + private static void verify( Iterable> expectedResults, Iterable> actualResults From 8b5c2fcd82411d6ca5b1fb502303adc1a6be9d94 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Sep 2014 14:10:02 -0700 Subject: [PATCH 22/46] fix compile error --- .../test/java/io/druid/query/select/SelectQueryRunnerTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java index 0f8df1962dc..27bfb256aa2 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java @@ -452,7 +452,8 @@ public class SelectQueryRunnerTest ) ) ) - ); + ) + ); verify(expectedResults, results); } From cada14586a70d48f78bbad88f70e2eeaae95c2a2 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Sep 2014 14:16:49 -0700 Subject: [PATCH 23/46] fix tests --- .../query/select/SelectQueryRunnerTest.java | 23 +++++++------------ 1 file changed, 8 insertions(+), 15 deletions(-) diff --git a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java index 27bfb256aa2..b60683c6d38 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java @@ -417,6 +417,11 @@ public class SelectQueryRunnerTest Lists.>newArrayList() ); + Map res = Maps.newHashMap(); + res.put("timestamp", new DateTime("2011-01-12T00:00:00.000Z")); + res.put("foo", null); + res.put("foo2", null); + List> expectedResults = Arrays.asList( new Result( new DateTime("2011-01-12T00:00:00.000Z"), @@ -426,29 +431,17 @@ public class SelectQueryRunnerTest new EventHolder( QueryRunnerTestHelper.segmentId, 0, - new ImmutableMap.Builder() - .put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z")) - .put("foo", null) - .put("foo2", null) - .build() + res ), new EventHolder( QueryRunnerTestHelper.segmentId, 1, - new ImmutableMap.Builder() - .put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z")) - .put(providerLowercase, "spot") - .put(QueryRunnerTestHelper.indexMetric, 100.000000F) - .build() + res ), new EventHolder( QueryRunnerTestHelper.segmentId, 2, - new ImmutableMap.Builder() - .put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z")) - .put(providerLowercase, "spot") - .put(QueryRunnerTestHelper.indexMetric, 100.000000F) - .build() + res ) ) ) From fbeb57d2612c5233b0635f21ec56fe1a7ef8650b Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Sep 2014 15:18:27 -0700 Subject: [PATCH 24/46] 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/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 +- 11 files changed, 20 insertions(+), 20 deletions(-) diff --git a/docs/content/Examples.md b/docs/content/Examples.md index dd7edee11a5..4e52e6f352d 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.152 +git checkout druid-0.6.154 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.152-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.154-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 8f1cf6a9874..df2bc695d44 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.152",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.154",...]` - becomes - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.152",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.154",...]` - 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 000fcb659ec..1644b8a5e1c 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.152"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.154"] 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.152","io.druid.extensions:druid-kafka-seven:0.6.152"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.154","io.druid.extensions:druid-kafka-seven:0.6.154"] 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.152"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.154"] 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 aa817e162e6..a9e680feba2 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.152"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.154"] 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.152","io.druid.extensions:druid-kafka-seven:0.6.152"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.154","io.druid.extensions:druid-kafka-seven:0.6.154"] 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 d8c25fe2a87..9116fe2f46e 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.152"] +-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.154"] -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 1343f9c7373..c2bdc630e6b 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.152-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.154-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.152 +cd druid-services-0.6.154 ``` 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 faf0bf534f0..22c0b98850b 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.152"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.154"] 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 126d9657e45..e90bce0a66f 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.152-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.154-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.152"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.154"] # 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.152","io.druid.extensions:druid-kafka-seven:0.6.152"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.154","io.druid.extensions:druid-kafka-seven:0.6.154"] # 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 e67c308d56a..6e73de9cfbe 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.152-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.154-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.152 +cd druid-services-0.6.154 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.md b/docs/content/Twitter-Tutorial.md index b299ad41ab5..2a9d6795f53 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.152-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.154-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 50cb60eab37..156461272bb 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.6 2.6.0 - 0.2.8 + 0.2.10 From f9c91ace3f242a9e794eab341cf9414e8bf44807 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Sep 2014 15:22:49 -0700 Subject: [PATCH 25/46] [maven-release-plugin] prepare release druid-0.6.154 --- 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 61316d59e1e..d1ab1763411 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154-SNAPSHOT + 0.6.154 diff --git a/common/pom.xml b/common/pom.xml index dd15871fcc3..cb5a8315b92 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154-SNAPSHOT + 0.6.154 diff --git a/examples/pom.xml b/examples/pom.xml index ab28920807d..9d5533a0a92 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154-SNAPSHOT + 0.6.154 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index f2e344d0e04..c02de193ed0 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154-SNAPSHOT + 0.6.154 diff --git a/histogram/pom.xml b/histogram/pom.xml index c0f8a20975d..883b2736d5b 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.154-SNAPSHOT + 0.6.154 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index cfa2c6627e8..993aef028c5 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154-SNAPSHOT + 0.6.154 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index a9f341f337f..00c375ef1de 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154-SNAPSHOT + 0.6.154 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 32759de1909..099ea63edb8 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154-SNAPSHOT + 0.6.154 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 6cb01258675..1b627244f8f 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154-SNAPSHOT + 0.6.154 diff --git a/pom.xml b/pom.xml index 156461272bb..64d130f4018 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.154-SNAPSHOT + 0.6.154 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.154 diff --git a/processing/pom.xml b/processing/pom.xml index f2d6dbc5707..5ae179c4f9a 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154-SNAPSHOT + 0.6.154 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index ad910d19c2d..686e32841bb 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.154-SNAPSHOT + 0.6.154 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index b49d04d2ea4..7af18fdd243 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154-SNAPSHOT + 0.6.154 diff --git a/server/pom.xml b/server/pom.xml index 8c0df2e1545..ba1b38f41ed 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154-SNAPSHOT + 0.6.154 diff --git a/services/pom.xml b/services/pom.xml index 23b3413b5ef..77df6377c22 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.154-SNAPSHOT + 0.6.154 From a17794a51668cb83d07a4c7220960bfc29ad309e Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Sep 2014 15:22:55 -0700 Subject: [PATCH 26/46] [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 d1ab1763411..4da364948f0 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154 + 0.6.155-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index cb5a8315b92..e998129bc3b 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154 + 0.6.155-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 9d5533a0a92..a33683bf415 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154 + 0.6.155-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index c02de193ed0..1de63dfff57 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154 + 0.6.155-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index 883b2736d5b..8f6eff8e056 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.154 + 0.6.155-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 993aef028c5..16845b52b84 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154 + 0.6.155-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 00c375ef1de..e3c55539e3b 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154 + 0.6.155-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 099ea63edb8..b1a63520214 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154 + 0.6.155-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 1b627244f8f..6438145ddd9 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154 + 0.6.155-SNAPSHOT diff --git a/pom.xml b/pom.xml index 64d130f4018..834b9493d6b 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.154 + 0.6.155-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.154 + druid-0.6.131-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index 5ae179c4f9a..1621a91d33e 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154 + 0.6.155-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 686e32841bb..d36a9d24bde 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.154 + 0.6.155-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 7af18fdd243..90c71008310 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154 + 0.6.155-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index ba1b38f41ed..f5eae7553a0 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.154 + 0.6.155-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 77df6377c22..457e9f36dd0 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.154 + 0.6.155-SNAPSHOT From d53ccf7ead5d04b700ce39ef7af73306a2e08d46 Mon Sep 17 00:00:00 2001 From: pdeva Date: Tue, 16 Sep 2014 15:56:15 -0700 Subject: [PATCH 27/46] segment metadata result explained better --- docs/content/SegmentMetadataQuery.md | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docs/content/SegmentMetadataQuery.md b/docs/content/SegmentMetadataQuery.md index ddc5910b618..c69cd33677c 100644 --- a/docs/content/SegmentMetadataQuery.md +++ b/docs/content/SegmentMetadataQuery.md @@ -46,6 +46,12 @@ The format of the result is: } ] ``` +Dimension columns will have type `STRING`. +Metric columns will have type `FLOAT`. +Timestamp column will have type `LONG`. + +Only columns which are dimensions (ie, have type `STRING`) will have any cardinality. Rest of the columns (timestamp and metric columns) will show cardinality as `null`. + ### toInclude There are 3 types of toInclude objects. @@ -72,4 +78,4 @@ The grammar is as follows: ``` json "toInclude": { "type": "list", "columns": []} -``` \ No newline at end of file +``` From e6d93a307019406c9e4ec26d9d865fd3717feeb7 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 17 Sep 2014 15:57:58 +0530 Subject: [PATCH 28/46] fix NPE fix NPE when the dimension of metric is not present one of the segments to be reIndexed. --- .../indexing/firehose/IngestSegmentFirehoseFactory.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index 94adbc0f4f1..5744fa7f006 100644 --- a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -255,13 +255,18 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory dimSelectors = Maps.newHashMap(); for (String dim : dims) { final DimensionSelector dimSelector = cursor.makeDimensionSelector(dim); - dimSelectors.put(dim, dimSelector); + // dimSelector is null if the dimension is not present + if (dimSelector != null) { + dimSelectors.put(dim, dimSelector); + } } final Map metSelectors = Maps.newHashMap(); for (String metric : metrics) { final ObjectColumnSelector metricSelector = cursor.makeObjectColumnSelector(metric); - metSelectors.put(metric, metricSelector); + if (metricSelector != null) { + metSelectors.put(metric, metricSelector); + } } return Sequences.simple( From e5a13544a0fe3222a0bd2c05b496ffe7dad92db6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 17 Sep 2014 10:35:58 -0700 Subject: [PATCH 29/46] fix column selector on varying incr index schema --- .../IncrementalIndexStorageAdapter.java | 8 +- .../IncrementalIndexStorageAdapterTest.java | 193 +++++++++++------- 2 files changed, 128 insertions(+), 73 deletions(-) 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 6d742509ca2..9c7d36aa0d3 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -420,7 +420,11 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter @Override public Object get() { - final String[] dimVals = currEntry.getKey().getDims()[dimensionIndex]; + final String[][] dims = currEntry.getKey().getDims(); + if(dimensionIndex >= dims.length) { + return null; + } + final String[] dimVals = dims[dimensionIndex]; if (dimVals.length == 1) { return dimVals[0]; } @@ -600,4 +604,4 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter }; } } -} \ No newline at end of file +} diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index 1ce05305b5d..7f17d7fc2e9 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -34,6 +34,7 @@ import io.druid.granularity.QueryGranularity; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.JavaScriptAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.filter.DimFilters; import io.druid.query.groupby.GroupByQuery; @@ -52,6 +53,7 @@ import org.junit.Test; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Arrays; /** */ @@ -79,26 +81,7 @@ public class IncrementalIndexStorageAdapterTest ) ); - GroupByQueryEngine engine = new GroupByQueryEngine( - Suppliers.ofInstance(new GroupByQueryConfig() - { - @Override - public int getMaxIntermediateRows() - { - return 5; - } - }), - new StupidPool( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(50000); - } - } - ) - ); + GroupByQueryEngine engine = makeGroupByQueryEngine(); final Sequence rows = engine.process( GroupByQuery.builder() @@ -123,6 +106,93 @@ public class IncrementalIndexStorageAdapterTest Assert.assertEquals(ImmutableMap.of("sally", "bo", "cnt", 1l), row.getEvent()); } + @Test + public void testObjectColumnSelectorOnVaryingColumnSchema() throws Exception + { + IncrementalIndex index = new IncrementalIndex( + 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")} + ); + + index.add( + new MapBasedInputRow( + new DateTime("2014-09-01T00:00:00"), + Lists.newArrayList("billy"), + ImmutableMap.of("billy", "hi") + ) + ); + index.add( + new MapBasedInputRow( + new DateTime("2014-09-01T01:00:00"), + Lists.newArrayList("billy", "sally"), + ImmutableMap.of( + "billy", "hip", + "sally", "hop" + ) + ) + ); + + GroupByQueryEngine engine = makeGroupByQueryEngine(); + + final Sequence rows = engine.process( + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(QueryGranularity.ALL) + .setInterval(new Interval(0, new DateTime().getMillis())) + .addDimension("billy") + .addDimension("sally") + .addAggregator( + new LongSumAggregatorFactory("cnt", "cnt") + ) + .addAggregator( + new JavaScriptAggregatorFactory( + "fieldLength", + Arrays.asList("sally", "billy"), + "function(current, s, b) { return current + (s == null ? 0 : s.length) + (b == null ? 0 : b.length); }", + "function() { return 0; }", + "function(a,b) { return a + b; }" + ) + ) + .build(), + new IncrementalIndexStorageAdapter(index) + ); + + final ArrayList results = Sequences.toList(rows, Lists.newArrayList()); + + Assert.assertEquals(2, results.size()); + + MapBasedRow row = (MapBasedRow) results.get(0); + Assert.assertEquals(ImmutableMap.of("billy", "hi", "cnt", 1l, "fieldLength", 2.0), row.getEvent()); + + row = (MapBasedRow) results.get(1); + Assert.assertEquals(ImmutableMap.of("billy", "hip", "sally", "hop", "cnt", 1l, "fieldLength", 6.0), row.getEvent()); + } + + private static GroupByQueryEngine makeGroupByQueryEngine() + { + return new GroupByQueryEngine( + Suppliers.ofInstance( + new GroupByQueryConfig() + { + @Override + public int getMaxIntermediateRows() + { + return 5; + } + } + ), + new StupidPool( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(50000); + } + } + ) + ); + } + @Test public void testResetSanity() { IncrementalIndex index = new IncrementalIndex( @@ -236,60 +306,41 @@ public class IncrementalIndexStorageAdapterTest 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")} ); - index.add( - new MapBasedInputRow( - new DateTime().minus(1).getMillis(), - Lists.newArrayList("billy"), - ImmutableMap.of("billy", "hi") - ) - ); - index.add( - new MapBasedInputRow( - new DateTime().minus(1).getMillis(), - Lists.newArrayList("sally"), - ImmutableMap.of("sally", "bo") - ) - ); + index.add( + new MapBasedInputRow( + new DateTime().minus(1).getMillis(), + Lists.newArrayList("billy"), + ImmutableMap.of("billy", "hi") + ) + ); + index.add( + new MapBasedInputRow( + new DateTime().minus(1).getMillis(), + Lists.newArrayList("sally"), + ImmutableMap.of("sally", "bo") + ) + ); - GroupByQueryEngine engine = new GroupByQueryEngine( - Suppliers.ofInstance(new GroupByQueryConfig() - { - @Override - public int getMaxIntermediateRows() - { - return 5; - } - }), - new StupidPool( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(50000); - } - } - ) - ); + GroupByQueryEngine engine = makeGroupByQueryEngine(); - final Sequence rows = engine.process( - GroupByQuery.builder() - .setDataSource("test") - .setGranularity(QueryGranularity.ALL) - .setInterval(new Interval(0, new DateTime().getMillis())) - .addDimension("billy") - .addDimension("sally") - .addAggregator(new LongSumAggregatorFactory("cnt", "cnt")) - .setDimFilter(DimFilters.dimEquals("sally", (String) null)) - .build(), - new IncrementalIndexStorageAdapter(index) - ); + final Sequence rows = engine.process( + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(QueryGranularity.ALL) + .setInterval(new Interval(0, new DateTime().getMillis())) + .addDimension("billy") + .addDimension("sally") + .addAggregator(new LongSumAggregatorFactory("cnt", "cnt")) + .setDimFilter(DimFilters.dimEquals("sally", (String) null)) + .build(), + new IncrementalIndexStorageAdapter(index) + ); - final ArrayList results = Sequences.toList(rows, Lists.newArrayList()); + final ArrayList results = Sequences.toList(rows, Lists.newArrayList()); - Assert.assertEquals(1, results.size()); + Assert.assertEquals(1, results.size()); - MapBasedRow row = (MapBasedRow) results.get(0); - Assert.assertEquals(ImmutableMap.of("billy", "hi", "cnt", 1l), row.getEvent()); - } + MapBasedRow row = (MapBasedRow) results.get(0); + Assert.assertEquals(ImmutableMap.of("billy", "hi", "cnt", 1l), row.getEvent()); + } } From 32b6135f3d7379910f7452417afbddb2558f4ce8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 18 Sep 2014 12:01:28 -0700 Subject: [PATCH 30/46] background announce segment cache --- .../segment/loading/SegmentLoaderConfig.java | 8 + .../server/coordination/ZkCoordinator.java | 144 ++++++++++++++++-- 2 files changed, 140 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java b/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java index 2cba40194a0..bbed8d27f47 100644 --- a/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java +++ b/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java @@ -40,6 +40,9 @@ public class SegmentLoaderConfig @JsonProperty("dropSegmentDelayMillis") private int dropSegmentDelayMillis = 30 * 1000; // 30 seconds + @JsonProperty("announceIntervalMillis") + private int announceIntervalMillis = 5 * 1000; // 5 seconds + @JsonProperty private File infoDir = null; @@ -58,6 +61,11 @@ public class SegmentLoaderConfig return dropSegmentDelayMillis; } + public int getAnnounceIntervalMillis() + { + return announceIntervalMillis; + } + public File getInfoDir() { if (infoDir == null) { 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 b0611077aa3..ec2510df1fb 100644 --- a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java @@ -21,7 +21,10 @@ package io.druid.server.coordination; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; +import com.google.common.collect.Queues; +import com.google.common.util.concurrent.SettableFuture; import com.google.inject.Inject; +import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.emitter.EmittingLogger; import io.druid.segment.loading.SegmentLoaderConfig; @@ -33,7 +36,10 @@ import org.apache.curator.framework.CuratorFramework; import java.io.File; import java.io.IOException; import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; /** @@ -170,10 +176,11 @@ public class ZkCoordinator extends BaseZkCoordinator public void addSegments(Iterable segments, DataSegmentChangeCallback callback) { - try { - final List segmentFailures = Lists.newArrayList(); - final List validSegments = Lists.newArrayList(); + try(final BackgroundSegmentAnnouncer backgroundSegmentAnnouncer = + new BackgroundSegmentAnnouncer(announcer, exec, config.getAnnounceIntervalMillis())) { + backgroundSegmentAnnouncer.startAnnouncing(); + final List segmentFailures = Lists.newArrayList(); for (DataSegment segment : segments) { log.info("Loading segment %s", segment.getIdentifier()); @@ -202,26 +209,25 @@ public class ZkCoordinator extends BaseZkCoordinator } } - validSegments.add(segment); + try { + backgroundSegmentAnnouncer.announceSegment(segment); + } catch(InterruptedException e) { + throw new SegmentLoadingException(e, "Loading Interrupted"); + } } } - try { - announcer.announceSegments(validSegments); - } - catch (IOException e) { - throw new SegmentLoadingException(e, "Failed to announce segments[%s]", segments); - } - if (!segmentFailures.isEmpty()) { for (String segmentFailure : segmentFailures) { log.error("%s failed to load", segmentFailure); } throw new SegmentLoadingException("%,d errors seen while loading segments", segmentFailures.size()); } + + backgroundSegmentAnnouncer.finishAnnouncing(); } catch (SegmentLoadingException e) { - log.makeAlert(e, "Failed to load segments for dataSource") + log.makeAlert(e, "Failed to load segments") .addData("segments", segments) .emit(); } @@ -272,4 +278,118 @@ public class ZkCoordinator extends BaseZkCoordinator callback.execute(); } } + + private static class BackgroundSegmentAnnouncer implements AutoCloseable { + private static final EmittingLogger log = new EmittingLogger(BackgroundSegmentAnnouncer.class); + + private final int intervalMillis; + private final DataSegmentAnnouncer announcer; + private final ScheduledExecutorService exec; + private final LinkedBlockingQueue queue; + private final SettableFuture doneAnnouncing; + + private volatile boolean finished = false; + private volatile ScheduledFuture startedAnnouncing = null; + private volatile ScheduledFuture nextAnnoucement = null; + + private BackgroundSegmentAnnouncer( + DataSegmentAnnouncer announcer, + ScheduledExecutorService exec, + int intervalMillis + ) + { + this.announcer = announcer; + this.exec = exec; + this.intervalMillis = intervalMillis; + this.queue = Queues.newLinkedBlockingQueue(); + this.doneAnnouncing = SettableFuture.create(); + } + + public void announceSegment(final DataSegment segment) throws InterruptedException + { + if (finished) { + throw new ISE("Announce segment called after finishAnnouncing"); + } + queue.put(segment); + } + + public void startAnnouncing() + { + if (intervalMillis <= 0) { + return; + } + + log.info("Starting background segment announcing task"); + + // schedule background announcing task + nextAnnoucement = startedAnnouncing = exec.schedule( + new Runnable() + { + @Override + public void run() + { + try { + if (!(finished && queue.isEmpty())) { + List segments = Lists.newLinkedList(); + queue.drainTo(segments); + try { + announcer.announceSegments(segments); + nextAnnoucement = exec.schedule(this, intervalMillis, TimeUnit.MILLISECONDS); + } + catch (IOException e) { + doneAnnouncing.setException( + new SegmentLoadingException(e, "Failed to announce segments[%s]", segments) + ); + } + } else { + doneAnnouncing.set(true); + } + } + catch (Exception e) { + doneAnnouncing.setException(e); + } + } + }, + intervalMillis, + TimeUnit.MILLISECONDS + ); + } + + public void finishAnnouncing() throws SegmentLoadingException + { + finished = true; + + if (startedAnnouncing != null && (startedAnnouncing.isDone() || !startedAnnouncing.cancel(false))) { + log.info("Waiting for background segment announcing task to complete"); + // background announcing already started, wait for it to complete + try { + doneAnnouncing.get(); + } + catch (InterruptedException e) { + throw new SegmentLoadingException(e, "Loading Interrupted"); + } + catch (ExecutionException e) { + throw new SegmentLoadingException(e.getCause(), "Background Announcing Task Failed"); + } + } else { + // background task has not started yet, announcing immediately + try { + announcer.announceSegments(queue); + } + catch (Exception e) { + throw new SegmentLoadingException(e, "Failed to announce segments[%s]", queue); + } + } + log.info("Completed background segment announcing"); + } + + @Override + public void close() + { + finished = true; + if (nextAnnoucement != null) { + nextAnnoucement.cancel(false); + } + } + } } From 815ebeee256ef6d1221d979e3e866d2c3450110b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 18 Sep 2014 12:06:50 -0700 Subject: [PATCH 31/46] add docs for background segments loading --- docs/content/Configuration.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/content/Configuration.md b/docs/content/Configuration.md index 267d43c7025..4681085b2b2 100644 --- a/docs/content/Configuration.md +++ b/docs/content/Configuration.md @@ -155,8 +155,9 @@ Druid storage nodes maintain information about segments they have already downlo |--------|-----------|-------| |`druid.segmentCache.locations`|Segments assigned to a Historical node are first stored on the local file system (in a disk cache) and then served by the Historical node. These locations define where that local cache resides. | none (no caching) | |`druid.segmentCache.deleteOnRemove`|Delete segment files from cache once a node is no longer serving a segment.|true| -|`druid.segmentCache.dropSegmentDelayMillis`|How long a node delays before completely dropping segment.|5 minutes| +|`druid.segmentCache.dropSegmentDelayMillis`|How long a node delays before completely dropping segment.|30000 (30 seconds)| |`druid.segmentCache.infoDir`|Historical nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir| +|`druid.segmentCache.announceIntervalMillis`|How frequently to announce segments while segments are loading from cache. Set this value to zero to wait for all segments to be loaded before announcing|5000 (5 seconds)| ### Jetty Server Module From 12449481e391c67eac8143d073a167d68f768f7f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 22 Sep 2014 16:33:25 -0700 Subject: [PATCH 32/46] parallelize segment loading --- .../segment/loading/SegmentLoaderConfig.java | 8 + .../server/bridge/BridgeZkCoordinator.java | 4 +- .../coordination/BaseZkCoordinator.java | 22 ++- .../server/coordination/ZkCoordinator.java | 144 ++++++++++-------- .../server/bridge/DruidClusterBridgeTest.java | 2 + 5 files changed, 112 insertions(+), 68 deletions(-) diff --git a/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java b/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java index bbed8d27f47..9abe5fc4497 100644 --- a/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java +++ b/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java @@ -43,6 +43,9 @@ public class SegmentLoaderConfig @JsonProperty("announceIntervalMillis") private int announceIntervalMillis = 5 * 1000; // 5 seconds + @JsonProperty("numLoadingThreads") + private int numLoadingThreads = 1; + @JsonProperty private File infoDir = null; @@ -66,6 +69,11 @@ public class SegmentLoaderConfig return announceIntervalMillis; } + public int getNumLoadingThreads() + { + return numLoadingThreads; + } + public File getInfoDir() { if (infoDir == null) { diff --git a/server/src/main/java/io/druid/server/bridge/BridgeZkCoordinator.java b/server/src/main/java/io/druid/server/bridge/BridgeZkCoordinator.java index 9cbafaaa4dd..633ce904059 100644 --- a/server/src/main/java/io/druid/server/bridge/BridgeZkCoordinator.java +++ b/server/src/main/java/io/druid/server/bridge/BridgeZkCoordinator.java @@ -26,6 +26,7 @@ import com.metamx.common.logger.Logger; import io.druid.client.ServerView; import io.druid.concurrent.Execs; import io.druid.db.DatabaseSegmentManager; +import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.segment.realtime.DbSegmentPublisher; import io.druid.server.coordination.BaseZkCoordinator; import io.druid.server.coordination.DataSegmentChangeCallback; @@ -53,6 +54,7 @@ public class BridgeZkCoordinator extends BaseZkCoordinator public BridgeZkCoordinator( ObjectMapper jsonMapper, ZkPathsConfig zkPaths, + SegmentLoaderConfig config, DruidServerMetadata me, @Bridge CuratorFramework curator, DbSegmentPublisher dbSegmentPublisher, @@ -60,7 +62,7 @@ public class BridgeZkCoordinator extends BaseZkCoordinator ServerView serverView ) { - super(jsonMapper, zkPaths, me, curator); + super(jsonMapper, zkPaths, config, me, curator); this.dbSegmentPublisher = dbSegmentPublisher; this.databaseSegmentManager = databaseSegmentManager; 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 e751041747b..f31cfb6311c 100644 --- a/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java @@ -21,10 +21,13 @@ package io.druid.server.coordination; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.emitter.EmittingLogger; +import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.ChildData; @@ -34,6 +37,8 @@ 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; /** */ @@ -45,23 +50,33 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler private final ObjectMapper jsonMapper; private final ZkPathsConfig zkPaths; + private final SegmentLoaderConfig config; private final DruidServerMetadata me; private final CuratorFramework curator; private volatile PathChildrenCache loadQueueCache; private volatile boolean started; + private final ListeningExecutorService loadingExec; public BaseZkCoordinator( ObjectMapper jsonMapper, ZkPathsConfig zkPaths, + SegmentLoaderConfig config, DruidServerMetadata me, CuratorFramework curator ) { this.jsonMapper = jsonMapper; this.zkPaths = zkPaths; + this.config = config; this.me = me; this.curator = curator; + this.loadingExec = MoreExecutors.listeningDecorator( + Executors.newFixedThreadPool( + config.getNumLoadingThreads(), + new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ZkCoordinator-%s").build() + ) + ); } @LifecycleStart @@ -83,7 +98,7 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler loadQueueLocation, true, true, - new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ZkCoordinator-%s").build() + loadingExec ); try { @@ -200,4 +215,9 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler public abstract void loadLocalCache(); public abstract DataSegmentChangeHandler getDataSegmentChangeHandler(); + + public ListeningExecutorService getLoadingExecutor() + { + return loadingExec; + } } 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 ec2510df1fb..70affc17854 100644 --- a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java @@ -22,6 +22,7 @@ package io.druid.server.coordination; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; import com.google.common.collect.Queues; +import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.google.inject.Inject; import com.metamx.common.ISE; @@ -36,6 +37,7 @@ import org.apache.curator.framework.CuratorFramework; import java.io.File; import java.io.IOException; import java.util.List; +import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; @@ -66,7 +68,7 @@ public class ZkCoordinator extends BaseZkCoordinator ScheduledExecutorFactory factory ) { - super(jsonMapper, zkPaths, me, curator); + super(jsonMapper, zkPaths, config, me, curator); this.jsonMapper = jsonMapper; this.config = config; @@ -127,42 +129,47 @@ public class ZkCoordinator extends BaseZkCoordinator return ZkCoordinator.this; } + private boolean loadSegment(DataSegment segment, DataSegmentChangeCallback callback) throws SegmentLoadingException + { + final boolean loaded; + try { + loaded = serverManager.loadSegment(segment); + } + catch (Exception e) { + removeSegment(segment, callback); + throw new SegmentLoadingException(e, "Exception loading segment[%s]", segment.getIdentifier()); + } + + if (loaded) { + File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); + if (!segmentInfoCacheFile.exists()) { + try { + jsonMapper.writeValue(segmentInfoCacheFile, segment); + } + catch (IOException e) { + removeSegment(segment, callback); + throw new SegmentLoadingException( + e, "Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile + ); + } + } + } + return loaded; + } + @Override public void addSegment(DataSegment segment, DataSegmentChangeCallback callback) { try { log.info("Loading segment %s", segment.getIdentifier()); - - final boolean loaded; - try { - loaded = serverManager.loadSegment(segment); - } - catch (Exception e) { - removeSegment(segment, callback); - throw new SegmentLoadingException(e, "Exception loading segment[%s]", segment.getIdentifier()); - } - - if (loaded) { - File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); - if (!segmentInfoCacheFile.exists()) { - try { - jsonMapper.writeValue(segmentInfoCacheFile, segment); - } - catch (IOException e) { - removeSegment(segment, callback); - throw new SegmentLoadingException( - e, "Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile - ); - } - } - + if(loadSegment(segment, callback)) { try { announcer.announceSegment(segment); } 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") @@ -174,54 +181,58 @@ public class ZkCoordinator extends BaseZkCoordinator } } - public void addSegments(Iterable segments, DataSegmentChangeCallback callback) + public void addSegments(Iterable segments, final DataSegmentChangeCallback callback) { try(final BackgroundSegmentAnnouncer backgroundSegmentAnnouncer = new BackgroundSegmentAnnouncer(announcer, exec, config.getAnnounceIntervalMillis())) { backgroundSegmentAnnouncer.startAnnouncing(); - final List segmentFailures = Lists.newArrayList(); - for (DataSegment segment : segments) { - log.info("Loading segment %s", segment.getIdentifier()); + final List> segmentLoading = Lists.newArrayList(); - final boolean loaded; - try { - loaded = serverManager.loadSegment(segment); - } - catch (Exception e) { - log.error(e, "Exception loading segment[%s]", segment.getIdentifier()); - removeSegment(segment, callback); - segmentFailures.add(segment.getIdentifier()); - continue; - } - - if (loaded) { - File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); - if (!segmentInfoCacheFile.exists()) { - try { - jsonMapper.writeValue(segmentInfoCacheFile, segment); - } - catch (IOException e) { - log.error(e, "Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile); - removeSegment(segment, callback); - segmentFailures.add(segment.getIdentifier()); - continue; - } - } - - try { - backgroundSegmentAnnouncer.announceSegment(segment); - } catch(InterruptedException e) { - throw new SegmentLoadingException(e, "Loading Interrupted"); - } - } + for (final DataSegment segment : segments) { + segmentLoading.add( + getLoadingExecutor().submit( + new Callable() + { + @Override + public Boolean call() throws SegmentLoadingException + { + try { + log.info("Loading segment %s", segment.getIdentifier()); + final boolean loaded = loadSegment(segment, callback); + if (loaded) { + try { + backgroundSegmentAnnouncer.announceSegment(segment); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new SegmentLoadingException(e, "Loading Interrupted"); + } + } + return loaded; + } catch(SegmentLoadingException e) { + log.error(e, "[%s] failed to load", segment.getIdentifier()); + throw e; + } + } + } + ) + ); } - if (!segmentFailures.isEmpty()) { - for (String segmentFailure : segmentFailures) { - log.error("%s failed to load", segmentFailure); + int failed = 0; + for(ListenableFuture future : segmentLoading) { + try { + future.get(); + } catch(InterruptedException e) { + Thread.currentThread().interrupt(); + throw new SegmentLoadingException(e, "Loading Interrupted"); + } catch(ExecutionException e) { + failed++; } - throw new SegmentLoadingException("%,d errors seen while loading segments", segmentFailures.size()); + } + if(failed > 0) { + throw new SegmentLoadingException("%,d errors seen while loading segments", failed); } backgroundSegmentAnnouncer.finishAnnouncing(); @@ -292,7 +303,7 @@ public class ZkCoordinator extends BaseZkCoordinator private volatile ScheduledFuture startedAnnouncing = null; private volatile ScheduledFuture nextAnnoucement = null; - private BackgroundSegmentAnnouncer( + public BackgroundSegmentAnnouncer( DataSegmentAnnouncer announcer, ScheduledExecutorService exec, int intervalMillis @@ -366,6 +377,7 @@ public class ZkCoordinator extends BaseZkCoordinator doneAnnouncing.get(); } catch (InterruptedException e) { + Thread.currentThread().interrupt(); throw new SegmentLoadingException(e, "Loading Interrupted"); } catch (ExecutionException e) { diff --git a/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java b/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java index 0136a99482e..670fd892c24 100644 --- a/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java +++ b/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java @@ -31,6 +31,7 @@ import io.druid.curator.PotentiallyGzippedCompressionProvider; import io.druid.curator.announcement.Announcer; import io.druid.db.DatabaseSegmentManager; import io.druid.jackson.DefaultObjectMapper; +import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.segment.realtime.DbSegmentPublisher; import io.druid.server.DruidNode; import io.druid.server.coordination.BatchDataSegmentAnnouncer; @@ -156,6 +157,7 @@ public class DruidClusterBridgeTest BridgeZkCoordinator bridgeZkCoordinator = new BridgeZkCoordinator( jsonMapper, zkPathsConfig, + new SegmentLoaderConfig(), metadata, remoteCf, dbSegmentPublisher, From d7b39fa7ea5db2abe11b5950c6fb3747caa14c6a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 23 Sep 2014 09:30:36 -0700 Subject: [PATCH 33/46] add documentation --- docs/content/Configuration.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/content/Configuration.md b/docs/content/Configuration.md index 4681085b2b2..5d09851a924 100644 --- a/docs/content/Configuration.md +++ b/docs/content/Configuration.md @@ -157,7 +157,8 @@ Druid storage nodes maintain information about segments they have already downlo |`druid.segmentCache.deleteOnRemove`|Delete segment files from cache once a node is no longer serving a segment.|true| |`druid.segmentCache.dropSegmentDelayMillis`|How long a node delays before completely dropping segment.|30000 (30 seconds)| |`druid.segmentCache.infoDir`|Historical nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir| -|`druid.segmentCache.announceIntervalMillis`|How frequently to announce segments while segments are loading from cache. Set this value to zero to wait for all segments to be loaded before announcing|5000 (5 seconds)| +|`druid.segmentCache.announceIntervalMillis`|How frequently to announce segments while segments are loading from cache. Set this value to zero to wait for all segments to be loaded before announcing.|5000 (5 seconds)| +|`druid.segmentCache.numLoadingThreads`|How many segments to load concurrently from from deep storage.|1| ### Jetty Server Module From e6b3e37aeb4a88357414715cd0e4bb4704bc3f80 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 23 Sep 2014 09:36:21 -0700 Subject: [PATCH 34/46] update java-util --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 834b9493d6b..b32c8c09670 100644 --- a/pom.xml +++ b/pom.xml @@ -39,7 +39,7 @@ UTF-8 - 0.26.6 + 0.26.7 2.6.0 0.2.10 From 05d4f71ddc98bc2e88def12f78854fc0806e645e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 23 Sep 2014 14:55:22 -0700 Subject: [PATCH 35/46] fix background annoucing race condition --- .../server/coordination/ZkCoordinator.java | 91 +++++++++++-------- 1 file changed, 53 insertions(+), 38 deletions(-) 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 70affc17854..8024edf3daa 100644 --- a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java @@ -187,15 +187,15 @@ public class ZkCoordinator extends BaseZkCoordinator new BackgroundSegmentAnnouncer(announcer, exec, config.getAnnounceIntervalMillis())) { backgroundSegmentAnnouncer.startAnnouncing(); - final List> segmentLoading = Lists.newArrayList(); + final List segmentLoading = Lists.newArrayList(); for (final DataSegment segment : segments) { segmentLoading.add( getLoadingExecutor().submit( - new Callable() + new Callable() { @Override - public Boolean call() throws SegmentLoadingException + public Void call() throws SegmentLoadingException { try { log.info("Loading segment %s", segment.getIdentifier()); @@ -209,7 +209,7 @@ public class ZkCoordinator extends BaseZkCoordinator throw new SegmentLoadingException(e, "Loading Interrupted"); } } - return loaded; + return null; } catch(SegmentLoadingException e) { log.error(e, "[%s] failed to load", segment.getIdentifier()); throw e; @@ -299,6 +299,8 @@ public class ZkCoordinator extends BaseZkCoordinator private final LinkedBlockingQueue queue; private final SettableFuture doneAnnouncing; + private final Object lock = new Object(); + private volatile boolean finished = false; private volatile ScheduledFuture startedAnnouncing = null; private volatile ScheduledFuture nextAnnoucement = null; @@ -339,25 +341,27 @@ public class ZkCoordinator extends BaseZkCoordinator @Override public void run() { - try { - if (!(finished && queue.isEmpty())) { - List segments = Lists.newLinkedList(); - queue.drainTo(segments); - try { - announcer.announceSegments(segments); - nextAnnoucement = exec.schedule(this, intervalMillis, TimeUnit.MILLISECONDS); + synchronized (lock) { + try { + if (!(finished && queue.isEmpty())) { + final List segments = Lists.newLinkedList(); + queue.drainTo(segments); + try { + announcer.announceSegments(segments); + nextAnnoucement = exec.schedule(this, intervalMillis, TimeUnit.MILLISECONDS); + } + catch (IOException e) { + doneAnnouncing.setException( + new SegmentLoadingException(e, "Failed to announce segments[%s]", segments) + ); + } + } else { + doneAnnouncing.set(true); } - catch (IOException e) { - doneAnnouncing.setException( - new SegmentLoadingException(e, "Failed to announce segments[%s]", segments) - ); - } - } else { - doneAnnouncing.set(true); } - } - catch (Exception e) { - doneAnnouncing.setException(e); + catch (Exception e) { + doneAnnouncing.setException(e); + } } } }, @@ -368,13 +372,29 @@ public class ZkCoordinator extends BaseZkCoordinator public void finishAnnouncing() throws SegmentLoadingException { - finished = true; - - if (startedAnnouncing != null && (startedAnnouncing.isDone() || !startedAnnouncing.cancel(false))) { - log.info("Waiting for background segment announcing task to complete"); - // background announcing already started, wait for it to complete + synchronized (lock) { + finished = true; + // announce any remaining segments try { - doneAnnouncing.get(); + final List segments = Lists.newLinkedList(); + queue.drainTo(segments); + announcer.announceSegments(segments); + } + catch (Exception e) { + throw new SegmentLoadingException(e, "Failed to announce segments[%s]", queue); + } + + // get any exception that may have been thrown in background annoucing + try { + // check in case we did not call startAnnouncing + if (startedAnnouncing != null) { + startedAnnouncing.cancel(false); + } + // - if the task is waiting on the lock, then the queue will be empty by the time it runs + // - if the task just released it, then the lock ensures any exception is set in doneAnnouncing + if (doneAnnouncing.isDone()) { + doneAnnouncing.get(); + } } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -383,14 +403,6 @@ public class ZkCoordinator extends BaseZkCoordinator catch (ExecutionException e) { throw new SegmentLoadingException(e.getCause(), "Background Announcing Task Failed"); } - } else { - // background task has not started yet, announcing immediately - try { - announcer.announceSegments(queue); - } - catch (Exception e) { - throw new SegmentLoadingException(e, "Failed to announce segments[%s]", queue); - } } log.info("Completed background segment announcing"); } @@ -398,9 +410,12 @@ public class ZkCoordinator extends BaseZkCoordinator @Override public void close() { - finished = true; - if (nextAnnoucement != null) { - nextAnnoucement.cancel(false); + // stop background scheduling + synchronized (lock) { + finished = true; + if (nextAnnoucement != null) { + nextAnnoucement.cancel(false); + } } } } From bdea4f45eac6327a85811483bea553d97cebc3ee Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 23 Sep 2014 15:41:56 -0700 Subject: [PATCH 36/46] fixes #733 --- docs/content/SelectQuery.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/content/SelectQuery.md b/docs/content/SelectQuery.md index 22eb0ca276f..9149358d605 100644 --- a/docs/content/SelectQuery.md +++ b/docs/content/SelectQuery.md @@ -25,6 +25,7 @@ There are several main parts to a select query: |queryType|This String should always be "select"; this is the first thing Druid looks at to figure out how to interpret the query|yes| |dataSource|A String defining the data source to query, very similar to a table in a relational database|yes| |intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| +|filter|See [Filters](Filters.html)|no| |dimensions|The list of dimensions to select. If left empty, all dimensions are returned.|no| |metrics|The list of metrics to select. If left empty, all metrics are returned.|no| |pagingSpec|A JSON object indicating offsets into different scanned segments. Select query results will return a pagingSpec that can be reused for pagination.|yes| From 35fb210cfa42fdc7521ead9fd33995666005c8bc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 23 Sep 2014 16:05:32 -0700 Subject: [PATCH 37/46] add test for parallel loading --- .../coordination/ZkCoordinatorTest.java | 91 +++++++++++++++---- 1 file changed, 73 insertions(+), 18 deletions(-) diff --git a/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java b/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java index f50ae3e16fc..666c72c0f5c 100644 --- a/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java @@ -21,6 +21,7 @@ package io.druid.server.coordination; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.concurrent.ScheduledExecutors; @@ -50,18 +51,22 @@ import java.io.File; import java.io.IOException; import java.util.Arrays; import java.util.Collections; +import java.util.Iterator; import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; /** */ public class ZkCoordinatorTest extends CuratorTestBase { private static final Logger log = new Logger(ZkCoordinatorTest.class); + public static final int COUNT = 50; private final ObjectMapper jsonMapper = new DefaultObjectMapper(); private ZkCoordinator zkCoordinator; private ServerManager serverManager; private DataSegmentAnnouncer announcer; private File infoDir; + private AtomicInteger announceCount; @Before public void setUp() throws Exception @@ -101,9 +106,41 @@ public class ZkCoordinatorTest extends CuratorTestBase } }; - announcer = new SingleDataSegmentAnnouncer( - me, zkPaths, new Announcer(curator, Execs.singleThreaded("blah")), jsonMapper - ); + announceCount = new AtomicInteger(0); + announcer = new DataSegmentAnnouncer() + { + private final DataSegmentAnnouncer delegate = new SingleDataSegmentAnnouncer( + me, zkPaths, new Announcer(curator, Execs.singleThreaded("blah")), jsonMapper + ); + + @Override + public void announceSegment(DataSegment segment) throws IOException + { + announceCount.incrementAndGet(); + delegate.announceSegment(segment); + } + + @Override + public void unannounceSegment(DataSegment segment) throws IOException + { + announceCount.decrementAndGet(); + delegate.unannounceSegment(segment); + } + + @Override + public void announceSegments(Iterable segments) throws IOException + { + announceCount.addAndGet(Iterables.size(segments)); + delegate.announceSegments(segments); + } + + @Override + public void unannounceSegments(Iterable segments) throws IOException + { + announceCount.addAndGet(-Iterables.size(segments)); + delegate.unannounceSegments(segments); + } + }; zkCoordinator = new ZkCoordinator( jsonMapper, @@ -114,6 +151,18 @@ public class ZkCoordinatorTest extends CuratorTestBase { return infoDir; } + + @Override + public int getNumLoadingThreads() + { + return 5; + } + + @Override + public int getAnnounceIntervalMillis() + { + return 50; + } }, zkPaths, me, @@ -133,21 +182,22 @@ public class ZkCoordinatorTest extends CuratorTestBase @Test public void testLoadCache() throws Exception { - List segments = Lists.newArrayList( - makeSegment("test", "1", new Interval("P1d/2011-04-01")), - makeSegment("test", "1", new Interval("P1d/2011-04-02")), - makeSegment("test", "2", new Interval("P1d/2011-04-02")), - makeSegment("test", "1", new Interval("P1d/2011-04-03")), - makeSegment("test", "1", new Interval("P1d/2011-04-04")), - makeSegment("test", "1", new Interval("P1d/2011-04-05")), - makeSegment("test", "2", new Interval("PT1h/2011-04-04T01")), - makeSegment("test", "2", new Interval("PT1h/2011-04-04T02")), - makeSegment("test", "2", new Interval("PT1h/2011-04-04T03")), - makeSegment("test", "2", new Interval("PT1h/2011-04-04T05")), - makeSegment("test", "2", new Interval("PT1h/2011-04-04T06")), - makeSegment("test2", "1", new Interval("P1d/2011-04-01")), - makeSegment("test2", "1", new Interval("P1d/2011-04-02")) - ); + List segments = Lists.newLinkedList(); + for(int i = 0; i < COUNT; ++i) { + segments.add(makeSegment("test" + i, "1", new Interval("P1d/2011-04-01"))); + segments.add(makeSegment("test" + i, "1", new Interval("P1d/2011-04-02"))); + segments.add(makeSegment("test" + i, "2", new Interval("P1d/2011-04-02"))); + segments.add(makeSegment("test" + i, "1", new Interval("P1d/2011-04-03"))); + segments.add(makeSegment("test" + i, "1", new Interval("P1d/2011-04-04"))); + segments.add(makeSegment("test" + i, "1", new Interval("P1d/2011-04-05"))); + segments.add(makeSegment("test" + i, "2", new Interval("PT1h/2011-04-04T01"))); + segments.add(makeSegment("test" + i, "2", new Interval("PT1h/2011-04-04T02"))); + segments.add(makeSegment("test" + i, "2", new Interval("PT1h/2011-04-04T03"))); + segments.add(makeSegment("test" + i, "2", new Interval("PT1h/2011-04-04T05"))); + segments.add(makeSegment("test" + i, "2", new Interval("PT1h/2011-04-04T06"))); + segments.add(makeSegment("test_two" + i, "1", new Interval("P1d/2011-04-01"))); + segments.add(makeSegment("test_two" + i, "1", new Interval("P1d/2011-04-02"))); + } Collections.sort(segments); for (DataSegment segment : segments) { @@ -158,6 +208,11 @@ public class ZkCoordinatorTest extends CuratorTestBase Assert.assertTrue(serverManager.getDataSourceCounts().isEmpty()); zkCoordinator.start(); Assert.assertTrue(!serverManager.getDataSourceCounts().isEmpty()); + for(int i = 0; i < COUNT; ++i) { + Assert.assertEquals(11L, serverManager.getDataSourceCounts().get("test" + i).longValue()); + Assert.assertEquals(2L, serverManager.getDataSourceCounts().get("test_two" + i).longValue()); + } + Assert.assertEquals(13 * COUNT, announceCount.get()); zkCoordinator.stop(); for (DataSegment segment : segments) { From c042ce5b4b6441a27e119028a8e2841ea3b6c48a Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 23 Sep 2014 16:10:12 -0700 Subject: [PATCH 38/46] prep next rel --- 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/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 +- 10 files changed, 19 insertions(+), 19 deletions(-) diff --git a/docs/content/Examples.md b/docs/content/Examples.md index 4e52e6f352d..f05212937b4 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.154 +git checkout druid-0.6.155 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.154-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.155-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 df2bc695d44..8df267877d5 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.154",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.155",...]` - becomes - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.154",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.155",...]` - 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 1644b8a5e1c..bd667ccddb7 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.154"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.155"] 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.154","io.druid.extensions:druid-kafka-seven:0.6.154"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.155","io.druid.extensions:druid-kafka-seven:0.6.155"] 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.154"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.155"] 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 a9e680feba2..441a0bef778 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.154"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.155"] 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.154","io.druid.extensions:druid-kafka-seven:0.6.154"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.155","io.druid.extensions:druid-kafka-seven:0.6.155"] 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 9116fe2f46e..d92ec08aeaf 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.154"] +-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.155"] -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 c2bdc630e6b..c6c6b63a850 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.154-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.155-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.154 +cd druid-services-0.6.155 ``` 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 22c0b98850b..454422139d3 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.154"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.155"] 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 e90bce0a66f..ec2ed8b7286 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.154-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.155-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.154"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.155"] # 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.154","io.druid.extensions:druid-kafka-seven:0.6.154"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.155","io.druid.extensions:druid-kafka-seven:0.6.155"] # 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 6e73de9cfbe..514f364c304 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.154-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.155-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.154 +cd druid-services-0.6.155 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.md b/docs/content/Twitter-Tutorial.md index 2a9d6795f53..af284b41806 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.154-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.155-bin.tar.gz). Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: From 576e9b8a845cad3da27569a78719fb1a1bbf5783 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 23 Sep 2014 16:12:39 -0700 Subject: [PATCH 39/46] [maven-release-plugin] prepare release druid-0.6.155 --- 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 4da364948f0..69dc9e91799 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155-SNAPSHOT + 0.6.155 diff --git a/common/pom.xml b/common/pom.xml index e998129bc3b..d6444eb72de 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155-SNAPSHOT + 0.6.155 diff --git a/examples/pom.xml b/examples/pom.xml index a33683bf415..5fd74a69d4e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155-SNAPSHOT + 0.6.155 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 1de63dfff57..2b7902d1936 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155-SNAPSHOT + 0.6.155 diff --git a/histogram/pom.xml b/histogram/pom.xml index 8f6eff8e056..d3e07a59591 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.155-SNAPSHOT + 0.6.155 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 16845b52b84..8a536a34bf9 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155-SNAPSHOT + 0.6.155 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index e3c55539e3b..8d46e05c269 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155-SNAPSHOT + 0.6.155 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index b1a63520214..15d974225b8 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155-SNAPSHOT + 0.6.155 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 6438145ddd9..b34e35e2bca 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155-SNAPSHOT + 0.6.155 diff --git a/pom.xml b/pom.xml index b32c8c09670..247f3af5e25 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.155-SNAPSHOT + 0.6.155 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.155 diff --git a/processing/pom.xml b/processing/pom.xml index 1621a91d33e..2fd4bd8ac03 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155-SNAPSHOT + 0.6.155 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index d36a9d24bde..322e07673dc 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.155-SNAPSHOT + 0.6.155 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 90c71008310..870b88c232e 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155-SNAPSHOT + 0.6.155 diff --git a/server/pom.xml b/server/pom.xml index f5eae7553a0..bfc34456fbe 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155-SNAPSHOT + 0.6.155 diff --git a/services/pom.xml b/services/pom.xml index 457e9f36dd0..6b0cda78e9f 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.155-SNAPSHOT + 0.6.155 From 4235e14fc6557d8d2b6458097e5e74d1b0bc6337 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 23 Sep 2014 16:12:43 -0700 Subject: [PATCH 40/46] [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 69dc9e91799..4652432914d 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155 + 0.6.156-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index d6444eb72de..38c9a308c67 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155 + 0.6.156-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 5fd74a69d4e..7af616bf2e6 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155 + 0.6.156-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 2b7902d1936..67489f5d095 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155 + 0.6.156-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index d3e07a59591..33ecbf063fe 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.155 + 0.6.156-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 8a536a34bf9..da5b77d7da0 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155 + 0.6.156-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 8d46e05c269..38949df1548 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155 + 0.6.156-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 15d974225b8..98a0282f672 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155 + 0.6.156-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index b34e35e2bca..d60cbbd1246 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155 + 0.6.156-SNAPSHOT diff --git a/pom.xml b/pom.xml index 247f3af5e25..2212b2d7d3d 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.155 + 0.6.156-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.155 + druid-0.6.131-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index 2fd4bd8ac03..9e55e5c3f8a 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155 + 0.6.156-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 322e07673dc..a1de49283ec 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.155 + 0.6.156-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 870b88c232e..12c762720dc 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155 + 0.6.156-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index bfc34456fbe..aaa5c3b2417 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.155 + 0.6.156-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 6b0cda78e9f..f2fa23c1f39 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.155 + 0.6.156-SNAPSHOT From 8603d2fd3e47d3c90de78f4a8066789f42968dab Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 23 Sep 2014 16:23:02 -0700 Subject: [PATCH 41/46] keep default behavior --- .../main/java/io/druid/segment/loading/SegmentLoaderConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java b/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java index 9abe5fc4497..171e5d44a96 100644 --- a/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java +++ b/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java @@ -41,7 +41,7 @@ public class SegmentLoaderConfig private int dropSegmentDelayMillis = 30 * 1000; // 30 seconds @JsonProperty("announceIntervalMillis") - private int announceIntervalMillis = 5 * 1000; // 5 seconds + private int announceIntervalMillis = 0; // do not background announce @JsonProperty("numLoadingThreads") private int numLoadingThreads = 1; From d02fd97ea3e9dc9441d1774ce103cf1659f41225 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 23 Sep 2014 16:25:46 -0700 Subject: [PATCH 42/46] fix comment --- .../main/java/io/druid/server/coordination/ZkCoordinator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 8024edf3daa..34a0ade4574 100644 --- a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java @@ -386,7 +386,7 @@ public class ZkCoordinator extends BaseZkCoordinator // get any exception that may have been thrown in background annoucing try { - // check in case we did not call startAnnouncing + // check in case intervalMillis is <= 0 if (startedAnnouncing != null) { startedAnnouncing.cancel(false); } From 13ac7de882c69b057ff38183f5e2ec20bfb85869 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 23 Sep 2014 16:28:18 -0700 Subject: [PATCH 43/46] update doc versions --- 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/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 +- 10 files changed, 19 insertions(+), 19 deletions(-) diff --git a/docs/content/Examples.md b/docs/content/Examples.md index f05212937b4..468bdb8f41a 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.155 +git checkout druid-0.6.156 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.155-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.156-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 8df267877d5..d8b3c80232e 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.155",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.156",...]` - becomes - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.155",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.156",...]` - 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 bd667ccddb7..d8ed57ab048 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.155"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.156"] 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.155","io.druid.extensions:druid-kafka-seven:0.6.155"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.156","io.druid.extensions:druid-kafka-seven:0.6.156"] 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.155"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.156"] 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 441a0bef778..6cb566fac16 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.155"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.156"] 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.155","io.druid.extensions:druid-kafka-seven:0.6.155"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.156","io.druid.extensions:druid-kafka-seven:0.6.156"] 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 d92ec08aeaf..788f2d65af8 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.155"] +-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.156"] -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 c6c6b63a850..a0c89c0fafb 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.155-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.156-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.155 +cd druid-services-0.6.156 ``` 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 454422139d3..399af3cc786 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.155"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.156"] 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 ec2ed8b7286..13483109143 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.155-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.156-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.155"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.156"] # 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.155","io.druid.extensions:druid-kafka-seven:0.6.155"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.156","io.druid.extensions:druid-kafka-seven:0.6.156"] # 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 514f364c304..d28dc27923c 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.155-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.156-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.155 +cd druid-services-0.6.156 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.md b/docs/content/Twitter-Tutorial.md index af284b41806..db0eab430c6 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.155-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.156-bin.tar.gz). Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: From 80166314d9726ecb8cdb54f69a4d352e9713891a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 23 Sep 2014 16:31:21 -0700 Subject: [PATCH 44/46] [maven-release-plugin] prepare release druid-0.6.156 --- 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 4652432914d..ce5dacd5eda 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156-SNAPSHOT + 0.6.156 diff --git a/common/pom.xml b/common/pom.xml index 38c9a308c67..5b87b1beff5 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156-SNAPSHOT + 0.6.156 diff --git a/examples/pom.xml b/examples/pom.xml index 7af616bf2e6..17429e70c2d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156-SNAPSHOT + 0.6.156 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 67489f5d095..13fd07377c3 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156-SNAPSHOT + 0.6.156 diff --git a/histogram/pom.xml b/histogram/pom.xml index 33ecbf063fe..56b61a3c25c 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.156-SNAPSHOT + 0.6.156 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index da5b77d7da0..96dc0d8e0d0 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156-SNAPSHOT + 0.6.156 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 38949df1548..3bde42f4c88 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156-SNAPSHOT + 0.6.156 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 98a0282f672..e9754e376c3 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156-SNAPSHOT + 0.6.156 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index d60cbbd1246..578d810a73f 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156-SNAPSHOT + 0.6.156 diff --git a/pom.xml b/pom.xml index 2212b2d7d3d..4a4269c51e6 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.156-SNAPSHOT + 0.6.156 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.156 diff --git a/processing/pom.xml b/processing/pom.xml index 9e55e5c3f8a..7fdc9efcb5b 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156-SNAPSHOT + 0.6.156 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index a1de49283ec..206944bfc24 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.156-SNAPSHOT + 0.6.156 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 12c762720dc..dd1a71da2be 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156-SNAPSHOT + 0.6.156 diff --git a/server/pom.xml b/server/pom.xml index aaa5c3b2417..65c10b64b11 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156-SNAPSHOT + 0.6.156 diff --git a/services/pom.xml b/services/pom.xml index f2fa23c1f39..ef517dfdcaf 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.156-SNAPSHOT + 0.6.156 From 7cfe86b7bc0d7136418972ddacae4e2a38cc6380 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 23 Sep 2014 16:31:25 -0700 Subject: [PATCH 45/46] [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 ce5dacd5eda..e07536e0705 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156 + 0.6.157-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 5b87b1beff5..687d67afe92 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156 + 0.6.157-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 17429e70c2d..ed6bfaadae4 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156 + 0.6.157-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 13fd07377c3..2ab577dd4ef 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156 + 0.6.157-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index 56b61a3c25c..e2420e915b3 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.156 + 0.6.157-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 96dc0d8e0d0..32336247e92 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156 + 0.6.157-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 3bde42f4c88..a398851837c 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156 + 0.6.157-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index e9754e376c3..2251c71c8cd 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156 + 0.6.157-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 578d810a73f..9f4274211d0 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156 + 0.6.157-SNAPSHOT diff --git a/pom.xml b/pom.xml index 4a4269c51e6..20c89c4c3ba 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.156 + 0.6.157-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.156 + druid-0.6.131-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index 7fdc9efcb5b..c00837b2d91 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156 + 0.6.157-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 206944bfc24..2c0590533f9 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.156 + 0.6.157-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index dd1a71da2be..a512b649ecd 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156 + 0.6.157-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 65c10b64b11..7382aefb237 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.156 + 0.6.157-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index ef517dfdcaf..cfccaaed8de 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.156 + 0.6.157-SNAPSHOT From 63fcd1186462d94a68058d8d93011e14e3065912 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Tue, 30 Sep 2014 22:12:06 +0530 Subject: [PATCH 46/46] fix compilation --- .../segment/incremental/IncrementalIndexStorageAdapterTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index 572130c906e..bc15e5b930e 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -112,7 +112,7 @@ public class IncrementalIndexStorageAdapterTest public void testObjectColumnSelectorOnVaryingColumnSchema() throws Exception { IncrementalIndex index = new IncrementalIndex( - 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")} + 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, TestQueryRunners.pool ); index.add(