From 3bfbcbe95c2ec64b9a5a2bdbe3f04fc63a8d907d Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Fri, 28 Dec 2012 10:50:40 -0600 Subject: [PATCH 01/92] 1) Create DirectClientQuerySegmentWalker to make it a bit easier to embed and use DirectDruidClient as a client. --- .../http/DirectClientQuerySegmentWalker.java | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) create mode 100644 client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java diff --git a/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java b/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java new file mode 100644 index 00000000000..e1cac8a90c3 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java @@ -0,0 +1,63 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.http; + +import com.metamx.druid.Query; +import com.metamx.druid.client.DirectDruidClient; +import com.metamx.druid.query.FinalizeResultsQueryRunner; +import com.metamx.druid.query.QueryRunner; +import com.metamx.druid.query.QueryToolChestWarehouse; +import com.metamx.druid.query.segment.QuerySegmentWalker; +import com.metamx.druid.query.segment.SegmentDescriptor; +import org.joda.time.Interval; + +/** + */ +public class DirectClientQuerySegmentWalker implements QuerySegmentWalker +{ + private final QueryToolChestWarehouse warehouse; + private final DirectDruidClient baseClient; + + public DirectClientQuerySegmentWalker( + QueryToolChestWarehouse warehouse, + DirectDruidClient baseClient + ) + { + this.warehouse = warehouse; + this.baseClient = baseClient; + } + + @Override + public QueryRunner getQueryRunnerForIntervals(Query query, Iterable intervals) + { + return makeRunner(query); + } + + @Override + public QueryRunner getQueryRunnerForSegments(Query query, Iterable specs) + { + return makeRunner(query); + } + + private FinalizeResultsQueryRunner makeRunner(final Query query) + { + return new FinalizeResultsQueryRunner(baseClient, warehouse.getToolChest(query)); + } +} From 19c3fd336350eac29e8c7e6114aab1d6c4dd8cbc Mon Sep 17 00:00:00 2001 From: xvrl Date: Wed, 2 Jan 2013 14:27:34 -0800 Subject: [PATCH 02/92] make reference timestamp explicit for time based rules --- .../druid/master/DruidMasterRuleRunner.java | 4 +++- .../druid/master/rules/IntervalDropRule.java | 3 ++- .../druid/master/rules/IntervalLoadRule.java | 3 ++- .../druid/master/rules/PeriodDropRule.java | 4 ++-- .../druid/master/rules/PeriodLoadRule.java | 4 ++-- .../com/metamx/druid/master/rules/Rule.java | 3 ++- .../druid/master/rules/PeriodDropRuleTest.java | 15 ++++++++++----- .../druid/master/rules/PeriodLoadRuleTest.java | 17 ++++++++++------- 8 files changed, 33 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java b/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java index 56d27f47544..5c33a257f26 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java @@ -23,6 +23,7 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.master.rules.Rule; import com.metamx.emitter.EmittingLogger; +import org.joda.time.DateTime; import java.util.List; @@ -51,13 +52,14 @@ public class DruidMasterRuleRunner implements DruidMasterHelper } // Run through all matched rules for available segments + DateTime now = new DateTime(); DatabaseRuleManager databaseRuleManager = params.getDatabaseRuleManager(); for (DataSegment segment : params.getAvailableSegments()) { List rules = databaseRuleManager.getRulesWithDefault(segment.getDataSource()); boolean foundMatchingRule = false; for (Rule rule : rules) { - if (rule.appliesTo(segment)) { + if (rule.appliesTo(segment, now)) { stats.accumulate(rule.run(master, params, segment)); foundMatchingRule = true; break; diff --git a/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java b/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java index 6546fce40b3..0acdd8bc2f8 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java @@ -22,6 +22,7 @@ package com.metamx.druid.master.rules; import com.metamx.druid.client.DataSegment; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; +import org.joda.time.DateTime; import org.joda.time.Interval; /** @@ -52,7 +53,7 @@ public class IntervalDropRule extends DropRule } @Override - public boolean appliesTo(DataSegment segment) + public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp) { return interval.contains(segment.getInterval()); } diff --git a/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java index 8c77594a177..5aa984ccba8 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java @@ -23,6 +23,7 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; +import org.joda.time.DateTime; import org.joda.time.Interval; /** @@ -81,7 +82,7 @@ public class IntervalLoadRule extends LoadRule } @Override - public boolean appliesTo(DataSegment segment) + public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp) { return interval.contains(segment.getInterval()); } diff --git a/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java b/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java index ce3c472a28f..83194dcc9de 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java @@ -54,9 +54,9 @@ public class PeriodDropRule extends DropRule } @Override - public boolean appliesTo(DataSegment segment) + public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp) { - final Interval currInterval = new Interval(new DateTime().minus(period), period); + final Interval currInterval = new Interval(referenceTimestamp.minus(period), period); return currInterval.contains(segment.getInterval()); } } diff --git a/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java index 60d27eb01cd..051967e65ab 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java @@ -81,9 +81,9 @@ public class PeriodLoadRule extends LoadRule } @Override - public boolean appliesTo(DataSegment segment) + public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp) { - final Interval currInterval = new Interval(period, new DateTime()); + final Interval currInterval = new Interval(period, referenceTimestamp); return currInterval.overlaps(segment.getInterval()); } } diff --git a/server/src/main/java/com/metamx/druid/master/rules/Rule.java b/server/src/main/java/com/metamx/druid/master/rules/Rule.java index a6fbfa358cc..a6eced93c68 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/Rule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/Rule.java @@ -25,6 +25,7 @@ import com.metamx.druid.master.DruidMasterRuntimeParams; import com.metamx.druid.master.MasterStats; import org.codehaus.jackson.annotate.JsonSubTypes; import org.codehaus.jackson.annotate.JsonTypeInfo; +import org.joda.time.DateTime; /** */ @@ -40,7 +41,7 @@ public interface Rule { public String getType(); - public boolean appliesTo(DataSegment segment); + public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp); public MasterStats run(DruidMaster master, DruidMasterRuntimeParams params, DataSegment segment); } diff --git a/server/src/test/java/com/metamx/druid/master/rules/PeriodDropRuleTest.java b/server/src/test/java/com/metamx/druid/master/rules/PeriodDropRuleTest.java index ae0c7cedd2e..2f00c85ed6c 100644 --- a/server/src/test/java/com/metamx/druid/master/rules/PeriodDropRuleTest.java +++ b/server/src/test/java/com/metamx/druid/master/rules/PeriodDropRuleTest.java @@ -51,13 +51,15 @@ public class PeriodDropRuleTest now.minusDays(2), now.minusDays(1) ) - ).build() + ).build(), + now ) ); Assert.assertTrue( rule.appliesTo( builder.interval(new Interval(now.minusYears(100), now.minusDays(1))) - .build() + .build(), + now ) ); } @@ -73,19 +75,22 @@ public class PeriodDropRuleTest Assert.assertTrue( rule.appliesTo( builder.interval(new Interval(now.minusWeeks(1), now.minusDays(1))) - .build() + .build(), + now ) ); Assert.assertFalse( rule.appliesTo( builder.interval(new Interval(now.minusYears(1), now.minusDays(1))) - .build() + .build(), + now ) ); Assert.assertFalse( rule.appliesTo( builder.interval(new Interval(now.minusMonths(2), now.minusDays(1))) - .build() + .build(), + now ) ); } diff --git a/server/src/test/java/com/metamx/druid/master/rules/PeriodLoadRuleTest.java b/server/src/test/java/com/metamx/druid/master/rules/PeriodLoadRuleTest.java index 283d684cb07..3944d96ecb9 100644 --- a/server/src/test/java/com/metamx/druid/master/rules/PeriodLoadRuleTest.java +++ b/server/src/test/java/com/metamx/druid/master/rules/PeriodLoadRuleTest.java @@ -39,38 +39,41 @@ public class PeriodLoadRuleTest @Test public void testAppliesToAll() { + DateTime now = new DateTime("2013-01-01"); PeriodLoadRule rule = new PeriodLoadRule( new Period("P5000Y"), 0, "" ); - Assert.assertTrue(rule.appliesTo(builder.interval(new Interval("2012-01-01/2012-12-31")).build())); - Assert.assertTrue(rule.appliesTo(builder.interval(new Interval("1000-01-01/2012-12-31")).build())); - Assert.assertTrue(rule.appliesTo(builder.interval(new Interval("0500-01-01/2100-12-31")).build())); + Assert.assertTrue(rule.appliesTo(builder.interval(new Interval("2012-01-01/2012-12-31")).build(), now)); + Assert.assertTrue(rule.appliesTo(builder.interval(new Interval("1000-01-01/2012-12-31")).build(), now)); + Assert.assertTrue(rule.appliesTo(builder.interval(new Interval("0500-01-01/2100-12-31")).build(), now)); } @Test public void testAppliesToPeriod() { - DateTime now = new DateTime(); + DateTime now = new DateTime("2012-12-31T01:00:00"); PeriodLoadRule rule = new PeriodLoadRule( new Period("P1M"), 0, "" ); - Assert.assertTrue(rule.appliesTo(builder.interval(new Interval(now.minusWeeks(1), now)).build())); + Assert.assertTrue(rule.appliesTo(builder.interval(new Interval(now.minusWeeks(1), now)).build(), now)); Assert.assertTrue( rule.appliesTo( builder.interval(new Interval(now.minusDays(1), now.plusDays(1))) - .build() + .build(), + now ) ); Assert.assertFalse( rule.appliesTo( builder.interval(new Interval(now.plusDays(1), now.plusDays(2))) - .build() + .build(), + now ) ); } From 0543322c5694712d5b649a0d83e8a8b5089e53cd Mon Sep 17 00:00:00 2001 From: xvrl Date: Wed, 2 Jan 2013 14:31:51 -0800 Subject: [PATCH 03/92] tested timestamps should be deterministic --- .../com/metamx/druid/master/rules/PeriodDropRuleTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/com/metamx/druid/master/rules/PeriodDropRuleTest.java b/server/src/test/java/com/metamx/druid/master/rules/PeriodDropRuleTest.java index 2f00c85ed6c..cd0fbc85505 100644 --- a/server/src/test/java/com/metamx/druid/master/rules/PeriodDropRuleTest.java +++ b/server/src/test/java/com/metamx/druid/master/rules/PeriodDropRuleTest.java @@ -33,13 +33,13 @@ public class PeriodDropRuleTest { private final static DataSegment.Builder builder = DataSegment.builder() .dataSource("test") - .version(new DateTime().toString()) + .version(new DateTime("2012-12-31T01:00:00").toString()) .shardSpec(new NoneShardSpec()); @Test public void testAppliesToAll() { - DateTime now = new DateTime(); + DateTime now = new DateTime("2012-12-31T01:00:00"); PeriodDropRule rule = new PeriodDropRule( new Period("P5000Y") ); @@ -67,7 +67,7 @@ public class PeriodDropRuleTest @Test public void testAppliesToPeriod() { - DateTime now = new DateTime(); + DateTime now = new DateTime("2012-12-31T01:00:00"); PeriodDropRule rule = new PeriodDropRule( new Period("P1M") ); From a9b8d8f1cef2f7bb47734514006fbbd3311da920 Mon Sep 17 00:00:00 2001 From: xvrl Date: Wed, 2 Jan 2013 15:44:14 -0800 Subject: [PATCH 04/92] fix end of month bug in PeriodDropRule as well --- .../java/com/metamx/druid/master/rules/PeriodDropRule.java | 2 +- .../com/metamx/druid/master/rules/PeriodDropRuleTest.java | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java b/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java index 83194dcc9de..152f074dc3a 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java @@ -56,7 +56,7 @@ public class PeriodDropRule extends DropRule @Override public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp) { - final Interval currInterval = new Interval(referenceTimestamp.minus(period), period); + final Interval currInterval = new Interval(period, referenceTimestamp); return currInterval.contains(segment.getInterval()); } } diff --git a/server/src/test/java/com/metamx/druid/master/rules/PeriodDropRuleTest.java b/server/src/test/java/com/metamx/druid/master/rules/PeriodDropRuleTest.java index cd0fbc85505..c6bceb08e5e 100644 --- a/server/src/test/java/com/metamx/druid/master/rules/PeriodDropRuleTest.java +++ b/server/src/test/java/com/metamx/druid/master/rules/PeriodDropRuleTest.java @@ -79,6 +79,13 @@ public class PeriodDropRuleTest now ) ); + Assert.assertTrue( + rule.appliesTo( + builder.interval(new Interval(now.minusDays(1), now)) + .build(), + now + ) + ); Assert.assertFalse( rule.appliesTo( builder.interval(new Interval(now.minusYears(1), now.minusDays(1))) From 6525d818adf8d5ddb1aa433a81f36cd71203eb89 Mon Sep 17 00:00:00 2001 From: xvrl Date: Wed, 2 Jan 2013 18:14:00 -0800 Subject: [PATCH 05/92] postAggregators now return field dependencies --- .../src/main/java/com/metamx/druid/query/Queries.java | 9 +++++++-- .../aggregation/post/ArithmeticPostAggregator.java | 10 +++++----- .../druid/aggregation/post/ConstantPostAggregator.java | 5 +++-- .../aggregation/post/FieldAccessPostAggregator.java | 5 +++-- .../metamx/druid/aggregation/post/PostAggregator.java | 2 +- 5 files changed, 19 insertions(+), 12 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/Queries.java b/client/src/main/java/com/metamx/druid/query/Queries.java index 9d8e052c0c0..d55a5da0ea8 100644 --- a/client/src/main/java/com/metamx/druid/query/Queries.java +++ b/client/src/main/java/com/metamx/druid/query/Queries.java @@ -20,6 +20,7 @@ package com.metamx.druid.query; import com.google.common.base.Function; +import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -27,6 +28,7 @@ import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.post.PostAggregator; import javax.annotation.Nullable; +import java.util.HashSet; import java.util.List; import java.util.Set; @@ -58,9 +60,12 @@ public class Queries ); for (PostAggregator postAgg : postAggs) { + Set dependencies = postAgg.getDependentFields(); + Set missing = Sets.difference(dependencies, combinedAggNames); + Preconditions.checkArgument( - postAgg.verifyFields(combinedAggNames), - String.format("Missing field[%s]", postAgg.getName()) + missing.isEmpty(), + String.format("Missing fields [%s] for postAggregator [%s]", Joiner.on(",").join(missing), postAgg.getName()) ); combinedAggNames.add(postAgg.getName()); } diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java index 035d0fa6652..cec19d80d78 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java @@ -20,6 +20,7 @@ package com.metamx.druid.aggregation.post; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import com.metamx.common.IAE; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; @@ -69,14 +70,13 @@ public class ArithmeticPostAggregator implements PostAggregator } @Override - public boolean verifyFields(Set fieldNames) + public Set getDependentFields() { + Set dependentFields = Sets.newHashSet(); for (PostAggregator field : fields) { - if (!field.verifyFields(fieldNames)) { - return false; - } + dependentFields.addAll(field.getDependentFields()); } - return true; + return dependentFields; } @Override diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java index 645de6c9b80..f1bbb0d8392 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java @@ -19,6 +19,7 @@ package com.metamx.druid.aggregation.post; +import com.google.common.collect.Sets; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; @@ -44,9 +45,9 @@ public class ConstantPostAggregator implements PostAggregator } @Override - public boolean verifyFields(Set fields) + public Set getDependentFields() { - return true; + return Sets.newHashSet(); } @Override diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java index 0a1c866d044..780c720103d 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java @@ -19,6 +19,7 @@ package com.metamx.druid.aggregation.post; +import com.google.common.collect.Sets; import com.metamx.common.ISE; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; @@ -45,9 +46,9 @@ public class FieldAccessPostAggregator implements PostAggregator } @Override - public boolean verifyFields(Set fieldNames) + public Set getDependentFields() { - return fieldNames.contains(fieldName); + return Sets.newHashSet(fieldName); } @Override diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java index 487ac30efb3..5b1ebc60528 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java @@ -37,7 +37,7 @@ import java.util.Set; }) public interface PostAggregator { - public boolean verifyFields(Set fieldNames); + public Set getDependentFields(); public Comparator getComparator(); From 41edec62ce893482d759f04b89247f72b69b01ec Mon Sep 17 00:00:00 2001 From: xvrl Date: Wed, 2 Jan 2013 18:46:07 -0800 Subject: [PATCH 06/92] remove unused imports --- client/src/main/java/com/metamx/druid/query/Queries.java | 1 - 1 file changed, 1 deletion(-) diff --git a/client/src/main/java/com/metamx/druid/query/Queries.java b/client/src/main/java/com/metamx/druid/query/Queries.java index d55a5da0ea8..fc33a0c2b74 100644 --- a/client/src/main/java/com/metamx/druid/query/Queries.java +++ b/client/src/main/java/com/metamx/druid/query/Queries.java @@ -28,7 +28,6 @@ import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.post.PostAggregator; import javax.annotation.Nullable; -import java.util.HashSet; import java.util.List; import java.util.Set; From 824997eb43e99d0dcb76da7dc997731ea348bb5f Mon Sep 17 00:00:00 2001 From: xvrl Date: Thu, 3 Jan 2013 10:01:42 -0800 Subject: [PATCH 07/92] simplify code --- client/src/main/java/com/metamx/druid/query/Queries.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/src/main/java/com/metamx/druid/query/Queries.java b/client/src/main/java/com/metamx/druid/query/Queries.java index fc33a0c2b74..ae5f958b563 100644 --- a/client/src/main/java/com/metamx/druid/query/Queries.java +++ b/client/src/main/java/com/metamx/druid/query/Queries.java @@ -64,7 +64,7 @@ public class Queries Preconditions.checkArgument( missing.isEmpty(), - String.format("Missing fields [%s] for postAggregator [%s]", Joiner.on(",").join(missing), postAgg.getName()) + "Missing fields [%s] for postAggregator [%s]", missing, postAgg.getName() ); combinedAggNames.add(postAgg.getName()); } From f2c2d3f4cfe97100251a46273986d5a3879c7d6f Mon Sep 17 00:00:00 2001 From: xvrl Date: Thu, 3 Jan 2013 10:09:43 -0800 Subject: [PATCH 08/92] [maven-release-plugin] prepare release druid-0.1.25 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 55358d59bab..d43158bee42 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.25-SNAPSHOT + 0.1.25 diff --git a/common/pom.xml b/common/pom.xml index cfdbf4782d8..1a121c5292a 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.25-SNAPSHOT + 0.1.25 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index cba7eec5f3b..56b8d990bf8 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.1.25-SNAPSHOT + 0.1.25 com.metamx druid - 0.1.25-SNAPSHOT + 0.1.25 diff --git a/examples/pom.xml b/examples/pom.xml index 489d4481371..c4971269cea 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.25-SNAPSHOT + 0.1.25 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 995d649ff0c..7a27749277e 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.1.25-SNAPSHOT + 0.1.25 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index a5e7d242f1a..1886e27d49c 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.1.25-SNAPSHOT + 0.1.25 diff --git a/index-common/pom.xml b/index-common/pom.xml index 4cb30db8b05..cc141a27425 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.25-SNAPSHOT + 0.1.25 diff --git a/indexer/pom.xml b/indexer/pom.xml index ff29c24a67c..0f698f85757 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.25-SNAPSHOT + 0.1.25 diff --git a/merger/pom.xml b/merger/pom.xml index eb191b90701..019e17e4618 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.25-SNAPSHOT + 0.1.25 diff --git a/pom.xml b/pom.xml index 83a963f1c38..6a74b13d617 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.1.25-SNAPSHOT + 0.1.25 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 5da5645f35c..650842a4baf 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.25-SNAPSHOT + 0.1.25 diff --git a/server/pom.xml b/server/pom.xml index 4a1f2065059..8c9c05012e1 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.25-SNAPSHOT + 0.1.25 From 9627fdcd1cf13c25554c89c407743cb67b16e554 Mon Sep 17 00:00:00 2001 From: xvrl Date: Thu, 3 Jan 2013 10:09:52 -0800 Subject: [PATCH 09/92] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index d43158bee42..e737c55bc20 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.25 + 0.1.26-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 1a121c5292a..ec21eb37008 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.25 + 0.1.26-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 56b8d990bf8..81af25085e9 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.1.25 + 0.1.26-SNAPSHOT com.metamx druid - 0.1.25 + 0.1.26-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index c4971269cea..d1aae11058e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.25 + 0.1.26-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 7a27749277e..ae6b1282605 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.1.25 + 0.1.26-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 1886e27d49c..40420b8b2f3 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.1.25 + 0.1.26-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index cc141a27425..c5ca45c1002 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.25 + 0.1.26-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 0f698f85757..3ab32ed0b90 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.25 + 0.1.26-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 019e17e4618..1ea3478f7b9 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.25 + 0.1.26-SNAPSHOT diff --git a/pom.xml b/pom.xml index 6a74b13d617..b94153ac887 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.1.25 + 0.1.26-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 650842a4baf..6a0e30f2c8e 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.25 + 0.1.26-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 8c9c05012e1..88bfc7e2982 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.25 + 0.1.26-SNAPSHOT From b184c46ae06d1c6c0fdb25ce40968ad45ca8dc29 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Fri, 4 Jan 2013 18:12:46 -0600 Subject: [PATCH 10/92] 1) Introduce idea of value types and generic columns to the serialization format 2) Create SegmentLoader interface for the loading of segments! 3) Setup serialization of new format and conversion function inside IndexIO 4) Make some adjustments to the twitter demo to make it make a bit more sense based on some email feedback from someone kicking the tires. --- .../druid/initialization/Initialization.java | 2 +- .../druid/jackson/DefaultObjectMapper.java | 20 + .../processing/MetricSelectorFactory.java | 2 - .../examples/RealtimeStandaloneMain.java | 4 +- .../src/main/resources/runtime.properties | 8 - .../examples/RealtimeStandaloneMain.java | 4 +- .../TwitterSpritzerFirehoseFactory.java | 13 - .../src/main/resources/runtime.properties | 8 - .../metamx/druid/index/QueryableIndex.java | 33 + .../druid/index/SimpleQueryableIndex.java | 82 ++ .../druid/index/column/AbstractColumn.java | 61 ++ .../druid/index/column/BitmapIndex.java | 29 + .../com/metamx/druid/index/column/Column.java | 33 + .../druid/index/column/ColumnBuilder.java | 99 ++ .../index/column/ColumnCapabilities.java | 32 + .../index/column/ColumnCapabilitiesImpl.java | 98 ++ .../druid/index/column/ColumnDescriptor.java | 147 +++ .../druid/index/column/ColumnSelector.java | 28 + .../druid/index/column/ComplexColumn.java | 30 + .../druid/index/column/ComplexColumnImpl.java | 49 + .../index/column/DictionaryEncodedColumn.java | 34 + .../druid/index/column/FloatColumn.java | 50 + .../druid/index/column/GenericColumn.java | 40 + .../index/column/IndexedComplexColumn.java | 54 ++ .../column/IndexedFloatsGenericColumn.java | 87 ++ .../column/IndexedLongsGenericColumn.java | 87 ++ .../metamx/druid/index/column/LongColumn.java | 50 + .../druid/index/column/RunLengthColumn.java | 27 + .../druid/index/column/SimpleColumn.java | 87 ++ .../column/SimpleDictionaryEncodedColumn.java | 81 ++ .../index/column/StringMultiValueColumn.java | 106 +++ .../metamx/druid/index/column/ValueType.java | 11 + .../serde/BitmapIndexColumnPartSupplier.java | 58 ++ .../druid/index/serde/ColumnPartSerde.java | 44 + .../index/serde/ComplexColumnPartSerde.java | 77 ++ .../serde/ComplexColumnPartSupplier.java | 47 + .../DictionaryEncodedColumnPartSerde.java | 147 +++ .../DictionaryEncodedColumnSupplier.java | 53 ++ .../serde/FloatGenericColumnPartSerde.java | 83 ++ .../serde/FloatGenericColumnSupplier.java | 49 + .../serde/LongGenericColumnPartSerde.java | 83 ++ .../serde/LongGenericColumnSupplier.java | 46 + .../v1/CompressedFloatsIndexedSupplier.java | 10 +- .../v1/CompressedLongsIndexedSupplier.java | 10 +- .../com/metamx/druid/index/v1/IndexIO.java | 438 ++++++++- .../metamx/druid/index/v1/MetricHolder.java | 9 +- .../index/v1/serde/ComplexMetricSerde.java | 28 + .../com/metamx/druid/kv/GenericIndexed.java | 13 +- .../com/metamx/druid/kv/VSizeIndexed.java | 15 +- .../com/metamx/druid/kv/VSizeIndexedInts.java | 38 + .../CompressedFloatsIndexedSupplierTest.java | 54 ++ .../CompressedLongsIndexedSupplierTest.java | 53 ++ .../metamx/druid/kv/GenericIndexedTest.java | 3 +- .../metamx/druid/kv/VSizeIndexedIntsTest.java | 25 + .../com/metamx/druid/kv/VSizeIndexedTest.java | 7 +- .../druid/merger/common/TaskToolbox.java | 18 +- .../common/index/YeOldePlumberSchool.java | 4 +- .../common/task/IndexGeneratorTask.java | 3 +- .../druid/merger/common/task/MergeTask.java | 8 +- .../merger/coordinator/exec/TaskConsumer.java | 2 +- .../http/IndexerCoordinatorNode.java | 6 +- .../druid/merger/worker/http/WorkerNode.java | 7 +- pom.xml | 10 +- realtime/pom.xml | 7 - .../metamx/druid/realtime/FireHydrant.java | 14 +- .../metamx/druid/realtime/RealtimeNode.java | 3 + .../druid/realtime/RealtimePlumberSchool.java | 26 +- .../druid/realtime/S3SegmentPusher.java | 128 +-- server/pom.xml | 7 - .../druid/coordination/ServerManager.java | 66 +- .../com/metamx/druid/http/ComputeNode.java | 20 +- .../IncrementalIndexSegment.java} | 42 +- .../druid/index/QueryableIndexSegment.java | 62 ++ .../Segment.java} | 13 +- .../metamx/druid/index/v1/IndexMerger.java | 3 +- .../index/v1/MMappedIndexQueryableIndex.java | 97 ++ .../index/v1/MMappedIndexStorageAdapter.java | 2 +- .../v1/QueryableIndexStorageAdapter.java | 864 ++++++++++++++++++ .../v1/processing/DimensionSelector.java | 18 +- .../druid/initialization/ServerInit.java | 39 +- ...ader.java => DelegatingSegmentLoader.java} | 23 +- ...java => MMappedQueryableIndexFactory.java} | 12 +- ...actory.java => QueryableIndexFactory.java} | 6 +- ...egmentGetter.java => S3SegmentPuller.java} | 13 +- .../metamx/druid/loading/S3SegmentPusher.java | 149 +++ .../druid/loading}/S3SegmentPusherConfig.java | 2 +- ...Getter.java => S3ZippedSegmentPuller.java} | 13 +- ...AdapterFactory.java => SegmentLoader.java} | 17 +- ...{SegmentGetter.java => SegmentPuller.java} | 8 +- .../metamx/druid/loading}/SegmentPusher.java | 4 +- ...ntGetter.java => SingleSegmentLoader.java} | 41 +- .../druid/query/QueryRunnerFactory.java | 3 +- .../{ => query/group}/GroupByQueryEngine.java | 4 +- .../group}/GroupByQueryEngineConfig.java | 2 +- .../group/GroupByQueryRunnerFactory.java | 41 +- .../SegmentMetadataQueryRunnerFactory.java | 5 +- .../search/SearchQueryRunnerFactory.java | 89 +- .../TimeBoundaryQueryRunnerFactory.java | 80 +- .../timeseries/TimeseriesQueryEngine.java | 100 ++ .../TimeseriesQueryRunnerFactory.java | 100 +- .../java/com/metamx/druid/TestHelper.java | 2 +- .../metamx/druid/client/RangeIterable.java | 0 .../druid/coordination/ServerManagerTest.java | 68 +- .../druid/coordination/ZkCoordinatorTest.java | 4 +- .../com/metamx/druid/index/v1/TestIndex.java | 221 +---- .../druid/loading/NoopSegmentLoader.java | 67 ++ .../loading/NoopStorageAdapterLoader.java | 98 -- .../druid/query/QueryRunnerTestHelper.java | 25 +- .../GroupByTimeseriesQueryRunnerTest.java | 12 +- .../timeseries/TimeseriesQueryRunnerTest.java | 42 + 110 files changed, 4624 insertions(+), 982 deletions(-) create mode 100644 index-common/src/main/java/com/metamx/druid/index/QueryableIndex.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/AbstractColumn.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/BitmapIndex.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/Column.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/ColumnBuilder.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilities.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/ColumnSelector.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/ComplexColumn.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/LongColumn.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/RunLengthColumn.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/StringMultiValueColumn.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/column/ValueType.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnSupplier.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnSupplier.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java create mode 100644 index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnSupplier.java rename server/src/main/java/com/metamx/druid/{loading/SingleStorageAdapterLoader.java => index/IncrementalIndexSegment.java} (54%) create mode 100644 server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java rename server/src/main/java/com/metamx/druid/{loading/StorageAdapterLoader.java => index/Segment.java} (74%) create mode 100644 server/src/main/java/com/metamx/druid/index/v1/MMappedIndexQueryableIndex.java create mode 100644 server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java rename server/src/main/java/com/metamx/druid/loading/{DelegatingStorageAdapterLoader.java => DelegatingSegmentLoader.java} (63%) rename server/src/main/java/com/metamx/druid/loading/{ConvertingBaseQueryableFactory.java => MMappedQueryableIndexFactory.java} (85%) rename server/src/main/java/com/metamx/druid/loading/{StorageAdapterFactory.java => QueryableIndexFactory.java} (85%) rename server/src/main/java/com/metamx/druid/loading/{S3SegmentGetter.java => S3SegmentPuller.java} (92%) create mode 100644 server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java rename {realtime/src/main/java/com/metamx/druid/realtime => server/src/main/java/com/metamx/druid/loading}/S3SegmentPusherConfig.java (96%) rename server/src/main/java/com/metamx/druid/loading/{S3ZippedSegmentGetter.java => S3ZippedSegmentPuller.java} (91%) rename server/src/main/java/com/metamx/druid/loading/{MMappedStorageAdapterFactory.java => SegmentLoader.java} (65%) rename server/src/main/java/com/metamx/druid/loading/{SegmentGetter.java => SegmentPuller.java} (77%) rename {realtime/src/main/java/com/metamx/druid/realtime => server/src/main/java/com/metamx/druid/loading}/SegmentPusher.java (95%) rename server/src/main/java/com/metamx/druid/loading/{RealtimeSegmentGetter.java => SingleSegmentLoader.java} (50%) rename server/src/main/java/com/metamx/druid/{ => query/group}/GroupByQueryEngine.java (99%) rename server/src/main/java/com/metamx/druid/{ => query/group}/GroupByQueryEngineConfig.java (96%) create mode 100644 server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryEngine.java rename {client => server}/src/test/java/com/metamx/druid/TestHelper.java (99%) rename {client => server}/src/test/java/com/metamx/druid/client/RangeIterable.java (100%) create mode 100644 server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java delete mode 100644 server/src/test/java/com/metamx/druid/loading/NoopStorageAdapterLoader.java diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index 7745260609f..cfb2d9302a2 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -203,7 +203,7 @@ public class Initialization log.info("Loaded(properties stored in zk) Property[%s] as [%s]", prop, zkProps.getProperty(prop)); } } // get props from zk - } else { // ToDo: should this be an error? + } else { log.warn("property druid.zk.service.host is not set, so no way to contact zookeeper for coordination."); } // validate properties now that all levels of precedence are loaded diff --git a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java index 7c96a8a3b2b..c80958eff88 100644 --- a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java +++ b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java @@ -35,10 +35,13 @@ import org.codehaus.jackson.map.JsonSerializer; import org.codehaus.jackson.map.ObjectMapper; import org.codehaus.jackson.map.SerializationConfig; import org.codehaus.jackson.map.SerializerProvider; +import org.codehaus.jackson.map.Serializers; import org.codehaus.jackson.map.module.SimpleModule; +import org.codehaus.jackson.map.ser.std.ToStringSerializer; import org.joda.time.DateTimeZone; import java.io.IOException; +import java.nio.ByteOrder; import java.util.TimeZone; /** @@ -131,6 +134,23 @@ public class DefaultObjectMapper extends ObjectMapper } } ); + serializerModule.addSerializer(ByteOrder.class, ToStringSerializer.instance); + serializerModule.addDeserializer( + ByteOrder.class, + new JsonDeserializer() + { + @Override + public ByteOrder deserialize( + JsonParser jp, DeserializationContext ctxt + ) throws IOException, JsonProcessingException + { + if (ByteOrder.BIG_ENDIAN.toString().equals(jp.getText())) { + return ByteOrder.BIG_ENDIAN; + } + return ByteOrder.LITTLE_ENDIAN; + } + } + ); registerModule(serializerModule); configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false); diff --git a/common/src/main/java/com/metamx/druid/processing/MetricSelectorFactory.java b/common/src/main/java/com/metamx/druid/processing/MetricSelectorFactory.java index 8c023c322f4..5864766787f 100644 --- a/common/src/main/java/com/metamx/druid/processing/MetricSelectorFactory.java +++ b/common/src/main/java/com/metamx/druid/processing/MetricSelectorFactory.java @@ -19,8 +19,6 @@ package com.metamx.druid.processing; -import java.io.Closeable; - /** * Factory class for MetricSelectors */ diff --git a/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java index 29263f1d6e5..4ffd7119431 100644 --- a/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -11,7 +11,7 @@ import com.metamx.druid.log.LogLevelAdjuster; import com.metamx.druid.realtime.MetadataUpdater; import com.metamx.druid.realtime.MetadataUpdaterConfig; import com.metamx.druid.realtime.RealtimeNode; -import com.metamx.druid.realtime.SegmentPusher; +import com.metamx.druid.loading.SegmentPusher; import com.metamx.phonebook.PhoneBook; import org.codehaus.jackson.map.jsontype.NamedType; @@ -21,8 +21,6 @@ import java.io.IOException; /** * Standalone Demo Realtime process. * Created: 20121009T2050 - * - * @author pbaclace */ public class RealtimeStandaloneMain { diff --git a/examples/rand/src/main/resources/runtime.properties b/examples/rand/src/main/resources/runtime.properties index 35ed5646a3f..c9483846106 100644 --- a/examples/rand/src/main/resources/runtime.properties +++ b/examples/rand/src/main/resources/runtime.properties @@ -41,14 +41,6 @@ druid.paths.segmentInfoCache=/tmp/rand_realtime/segmentInfoCache # Path to schema definition file druid.request.logging.dir=/tmp/rand_realtime/log -# TODO: have these moved to spec file? -# unknown # druid.realtime.dataSources= -# unknown # druid.realtime.index.maxSize=500000 -# unknown # druid.realtime.persistPeriod=PT600S -# unknown # druid.realtime.scheduledExec.threads=1 -# unknown # druid.realtime.uploadPeriod=PT3600S -# unknown # druid.realtime.windowPeriod=PT600S - #druid.server.maxSize=0 druid.server.maxSize=300000000000 # =realtime or =historical (default) diff --git a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java index 32f355c756b..fd072a427f3 100644 --- a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -11,7 +11,7 @@ import com.metamx.druid.log.LogLevelAdjuster; import com.metamx.druid.realtime.MetadataUpdater; import com.metamx.druid.realtime.MetadataUpdaterConfig; import com.metamx.druid.realtime.RealtimeNode; -import com.metamx.druid.realtime.SegmentPusher; +import com.metamx.druid.loading.SegmentPusher; import com.metamx.phonebook.PhoneBook; import druid.examples.twitter.TwitterSpritzerFirehoseFactory; import org.codehaus.jackson.map.jsontype.NamedType; @@ -21,8 +21,6 @@ import java.io.IOException; /** Standalone Demo Realtime process. * Created: 20121009T2050 - * - * @author pbaclace */ public class RealtimeStandaloneMain { diff --git a/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java b/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java index 249ea12e3b1..826218afbd8 100644 --- a/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java +++ b/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java @@ -117,12 +117,8 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory { final long startMsec = System.currentTimeMillis(); dimensions.add("htags"); - dimensions.add("retweet_count"); - dimensions.add("follower_count"); - dimensions.add("friends_count"); dimensions.add("lang"); dimensions.add("utc_offset"); - dimensions.add("statuses_count"); // // set up Twitter Spritzer @@ -254,7 +250,6 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory { if (hts != null && hts.length > 0) { // ToDo: get all the hash tags instead of just the first one theMap.put("htags", hts[0].getText()); - log.info("htags=" + hts[0].getText()); // about 16% } else { theMap.put("htags", null); } @@ -271,14 +266,6 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory { } else { log.error("status.getUser() is null"); } - if (rowCount % 10 == 0) { - log.info("" + status.getCreatedAt() + - " follower_count=" + u.getFollowersCount() + - " friends_count=" + u.getFriendsCount() + - " statuses_count=" + u.getStatusesCount() + - " retweet_count=" + retweetCount - ); - } return new MapBasedInputRow(status.getCreatedAt().getTime(), dimensions, theMap); } diff --git a/examples/twitter/src/main/resources/runtime.properties b/examples/twitter/src/main/resources/runtime.properties index dfa12a8f0ec..8299b6d3c36 100644 --- a/examples/twitter/src/main/resources/runtime.properties +++ b/examples/twitter/src/main/resources/runtime.properties @@ -41,14 +41,6 @@ druid.paths.segmentInfoCache=/tmp/twitter_realtime/segmentInfoCache # Path to schema definition file druid.request.logging.dir=/tmp/twitter_realtime/log -# TODO: have these moved to spec file? -# unknown # druid.realtime.dataSources= -# unknown # druid.realtime.index.maxSize=500000 -# unknown # druid.realtime.persistPeriod=PT600S -# unknown # druid.realtime.scheduledExec.threads=1 -# unknown # druid.realtime.uploadPeriod=PT3600S -# unknown # druid.realtime.windowPeriod=PT600S - #druid.server.maxSize=0 druid.server.maxSize=300000000000 # =realtime or =historical (default) diff --git a/index-common/src/main/java/com/metamx/druid/index/QueryableIndex.java b/index-common/src/main/java/com/metamx/druid/index/QueryableIndex.java new file mode 100644 index 00000000000..0c4f57cf9b9 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/QueryableIndex.java @@ -0,0 +1,33 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index; + +import com.metamx.druid.index.column.ColumnSelector; +import com.metamx.druid.kv.Indexed; +import org.joda.time.Interval; + +/** + */ +public interface QueryableIndex extends ColumnSelector +{ + public Interval getDataInterval(); + public Indexed getColumnNames(); + public Indexed getAvailableDimensions(); +} diff --git a/index-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java b/index-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java new file mode 100644 index 00000000000..7e8a6ba7635 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java @@ -0,0 +1,82 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index; + +import com.metamx.druid.index.column.Column; +import com.metamx.druid.kv.Indexed; +import org.joda.time.Interval; + +import java.util.Map; + +/** + */ +public class SimpleQueryableIndex implements QueryableIndex +{ + private final Interval dataInterval; + private final Indexed columnNames; + private final Indexed availableDimensions; + private final Column timeColumn; + private final Map otherColumns; + + public SimpleQueryableIndex( + Interval dataInterval, + Indexed columnNames, + Indexed dimNames, + Column timeColumn, + Map otherColumns + ) + { + this.dataInterval = dataInterval; + this.columnNames = columnNames; + this.availableDimensions = dimNames; + this.timeColumn = timeColumn; + this.otherColumns = otherColumns; + } + + @Override + public Interval getDataInterval() + { + return dataInterval; + } + + @Override + public Indexed getColumnNames() + { + return columnNames; + } + + @Override + public Indexed getAvailableDimensions() + { + return availableDimensions; + } + + @Override + public Column getTimeColumn() + { + return timeColumn; + } + + @Override + public Column getColumn(String columnName) + { + return otherColumns.get(columnName); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/AbstractColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/AbstractColumn.java new file mode 100644 index 00000000000..978a3822071 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/AbstractColumn.java @@ -0,0 +1,61 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +/** + */ +public abstract class AbstractColumn implements Column +{ + @Override + public BitmapIndex getBitmapIndex() + { + throw new UnsupportedOperationException(); + } + + @Override + public ComplexColumn getComplexColumn() + { + throw new UnsupportedOperationException(); + } + + @Override + public GenericColumn getGenericColumn() + { + throw new UnsupportedOperationException(); + } + + @Override + public RunLengthColumn getRunLengthColumn() + { + throw new UnsupportedOperationException(); + } + + @Override + public DictionaryEncodedColumn getDictionaryEncoding() + { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnCapabilities getCapabilities() + { + throw new UnsupportedOperationException(); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/BitmapIndex.java b/index-common/src/main/java/com/metamx/druid/index/column/BitmapIndex.java new file mode 100644 index 00000000000..884b126d250 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/BitmapIndex.java @@ -0,0 +1,29 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; + +/** + */ +public interface BitmapIndex +{ + public ImmutableConciseSet getConciseSet(String value); +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/Column.java b/index-common/src/main/java/com/metamx/druid/index/column/Column.java new file mode 100644 index 00000000000..5702b42206e --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/Column.java @@ -0,0 +1,33 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +/** + */ +public interface Column +{ + public ColumnCapabilities getCapabilities(); + + public DictionaryEncodedColumn getDictionaryEncoding(); + public RunLengthColumn getRunLengthColumn(); + public GenericColumn getGenericColumn(); + public ComplexColumn getComplexColumn(); + public BitmapIndex getBitmapIndex(); +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ColumnBuilder.java b/index-common/src/main/java/com/metamx/druid/index/column/ColumnBuilder.java new file mode 100644 index 00000000000..7bf77e4fa10 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/ColumnBuilder.java @@ -0,0 +1,99 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; + +/** + */ +public class ColumnBuilder +{ + private ValueType type = null; + private boolean hasMultipleValues = false; + + private Supplier dictionaryEncodedColumn = null; + private Supplier runLengthColumn = null; + private Supplier genericColumn = null; + private Supplier complexColumn = null; + private Supplier bitmapIndex = null; + + public ColumnBuilder setType(ValueType type) + { + this.type = type; + return this; + } + + public ColumnBuilder setHasMultipleValues(boolean hasMultipleValues) + { + this.hasMultipleValues = hasMultipleValues; + return this; + } + + public ColumnBuilder setDictionaryEncodedColumn(Supplier dictionaryEncodedColumn) + { + this.dictionaryEncodedColumn = dictionaryEncodedColumn; + return this; + } + + public ColumnBuilder setRunLengthColumn(Supplier runLengthColumn) + { + this.runLengthColumn = runLengthColumn; + return this; + } + + public ColumnBuilder setGenericColumn(Supplier genericColumn) + { + this.genericColumn = genericColumn; + return this; + } + + public ColumnBuilder setComplexColumn(Supplier complexColumn) + { + this.complexColumn = complexColumn; + return this; + } + + public ColumnBuilder setBitmapIndex(Supplier bitmapIndex) + { + this.bitmapIndex = bitmapIndex; + return this; + } + + public Column build() + { + Preconditions.checkState(type != null, "Type must be set."); + + return new SimpleColumn( + new ColumnCapabilitiesImpl() + .setType(type) + .setDictionaryEncoded(dictionaryEncodedColumn != null) + .setHasBitmapIndexes(bitmapIndex != null) + .setRunLengthEncoded(runLengthColumn != null) + .setHasMultipleValues(hasMultipleValues) + , + dictionaryEncodedColumn, + runLengthColumn, + genericColumn, + complexColumn, + bitmapIndex + ); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilities.java b/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilities.java new file mode 100644 index 00000000000..8e2ee8d1af4 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilities.java @@ -0,0 +1,32 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +/** + */ +public interface ColumnCapabilities +{ + public ValueType getType(); + + public boolean isDictionaryEncoded(); + public boolean isRunLengthEncoded(); + public boolean hasBitmapIndexes(); + public boolean hasMultipleValues(); +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java b/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java new file mode 100644 index 00000000000..2be3a50a595 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java @@ -0,0 +1,98 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +import org.codehaus.jackson.annotate.JsonProperty; + +/** + */ +public class ColumnCapabilitiesImpl implements ColumnCapabilities +{ + private ValueType type = null; + private boolean dictionaryEncoded = false; + private boolean runLengthEncoded = false; + private boolean hasInvertedIndexes = false; + private boolean hasMultipleValues = false; + + @Override + @JsonProperty + public ValueType getType() + { + return type; + } + + public ColumnCapabilitiesImpl setType(ValueType type) + { + this.type = type; + return this; + } + + @Override + @JsonProperty + public boolean isDictionaryEncoded() + { + return dictionaryEncoded; + } + + public ColumnCapabilitiesImpl setDictionaryEncoded(boolean dictionaryEncoded) + { + this.dictionaryEncoded = dictionaryEncoded; + return this; + } + + @Override + @JsonProperty + public boolean isRunLengthEncoded() + { + return runLengthEncoded; + } + + public ColumnCapabilitiesImpl setRunLengthEncoded(boolean runLengthEncoded) + { + this.runLengthEncoded = runLengthEncoded; + return this; + } + + @Override + @JsonProperty("hasBitmapIndexes") + public boolean hasBitmapIndexes() + { + return hasInvertedIndexes; + } + + public ColumnCapabilitiesImpl setHasBitmapIndexes(boolean hasInvertedIndexes) + { + this.hasInvertedIndexes = hasInvertedIndexes; + return this; + } + + @Override + @JsonProperty("hasMultipleValues") + public boolean hasMultipleValues() + { + return hasMultipleValues; + } + + public ColumnCapabilitiesImpl setHasMultipleValues(boolean hasMultipleValues) + { + this.hasMultipleValues = hasMultipleValues; + return this; + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java b/index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java new file mode 100644 index 00000000000..7f9b7ac7ee2 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java @@ -0,0 +1,147 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.metamx.common.IAE; +import com.metamx.druid.index.serde.ColumnPartSerde; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; +import java.util.List; + +/** + */ +public class ColumnDescriptor +{ + public static Builder builder() + { + return new Builder(); + } + + private final ValueType valueType; + private final boolean hasMultipleValues; + private final List parts; + + @JsonCreator + public ColumnDescriptor( + @JsonProperty("valueType") ValueType valueType, + @JsonProperty("hasMultipleValues") boolean hasMultipleValues, + @JsonProperty("parts") List parts + ) + { + this.valueType = valueType; + this.hasMultipleValues = hasMultipleValues; + this.parts = parts; + } + + @JsonProperty + public ValueType getValueType() + { + return valueType; + } + + @JsonProperty + public boolean isHasMultipleValues() + { + return hasMultipleValues; + } + + @JsonProperty + public List getParts() + { + return parts; + } + + public int numBytes() + { + int retVal = 0; + + for (ColumnPartSerde part : parts) { + retVal += part.numBytes(); + } + + return retVal; + } + + public void write(WritableByteChannel channel) throws IOException + { + for (ColumnPartSerde part : parts) { + part.write(channel); + } + } + + public Column read(ByteBuffer buffer) + { + final ColumnBuilder builder = new ColumnBuilder() + .setType(valueType) + .setHasMultipleValues(hasMultipleValues); + + for (ColumnPartSerde part : parts) { + part.read(buffer, builder); + } + + return builder.build(); + } + + public static class Builder + { + private ValueType valueType = null; + private Boolean hasMultipleValues = null; + + private final List parts = Lists.newArrayList(); + + public Builder setValueType(ValueType valueType) + { + if (this.valueType != null && this.valueType != valueType) { + throw new IAE("valueType[%s] is already set, cannot change to[%s]", this.valueType, valueType); + } + this.valueType = valueType; + return this; + } + + public Builder setHasMultipleValues(boolean hasMultipleValues) + { + if (this.hasMultipleValues != null && this.hasMultipleValues != hasMultipleValues) { + throw new IAE( + "hasMultipleValues[%s] is already set, cannot change to[%s]", this.hasMultipleValues, hasMultipleValues + ); + } + this.hasMultipleValues = hasMultipleValues; + return this; + } + + public Builder addSerde(ColumnPartSerde serde) + { + parts.add(serde); + return this; + } + + public ColumnDescriptor build() + { + Preconditions.checkNotNull(valueType, "must specify a valueType"); + return new ColumnDescriptor(valueType, hasMultipleValues == null ? false : hasMultipleValues, parts); + } + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ColumnSelector.java b/index-common/src/main/java/com/metamx/druid/index/column/ColumnSelector.java new file mode 100644 index 00000000000..0ed66850dcb --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/ColumnSelector.java @@ -0,0 +1,28 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +/** + */ +public interface ColumnSelector +{ + public Column getTimeColumn(); + public Column getColumn(String columnName); +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ComplexColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/ComplexColumn.java new file mode 100644 index 00000000000..7d21987f6f9 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/ComplexColumn.java @@ -0,0 +1,30 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +import java.io.Closeable; + +/** + */ +public interface ComplexColumn extends Closeable +{ + public Class getClazz(); + public Object getRowValue(int rowNum); +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java b/index-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java new file mode 100644 index 00000000000..8f9a302adc5 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java @@ -0,0 +1,49 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +import com.metamx.druid.kv.Indexed; + +/** + */ +public class ComplexColumnImpl extends AbstractColumn +{ + private static final ColumnCapabilitiesImpl CAPABILITIES = new ColumnCapabilitiesImpl() + .setType(ValueType.COMPLEX); + + private final Indexed column; + + public ComplexColumnImpl(Indexed column) + { + this.column = column; + } + + @Override + public ColumnCapabilities getCapabilities() + { + return CAPABILITIES; + } + + @Override + public ComplexColumn getComplexColumn() + { + return new IndexedComplexColumn(column); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java new file mode 100644 index 00000000000..e2bd177a25c --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java @@ -0,0 +1,34 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +import com.metamx.druid.kv.IndexedInts; + +/** + */ +public interface DictionaryEncodedColumn +{ + public int size(); + public int getSingleValueRow(int rowNum); + public IndexedInts getMultiValueRow(int rowNum); + public String lookupName(int id); + public int lookupId(String name); + public int getCardinality(); +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java new file mode 100644 index 00000000000..491baa3c41d --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java @@ -0,0 +1,50 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +import com.google.common.base.Supplier; +import com.metamx.druid.kv.IndexedFloats; + +/** + */ +public class FloatColumn extends AbstractColumn +{ + private static final ColumnCapabilitiesImpl CAPABILITIES = new ColumnCapabilitiesImpl() + .setType(ValueType.FLOAT); + + private final Supplier column; + + public FloatColumn(Supplier column) + { + this.column = column; + } + + @Override + public ColumnCapabilities getCapabilities() + { + return CAPABILITIES; + } + + @Override + public GenericColumn getGenericColumn() + { + return new IndexedFloatsGenericColumn(column.get()); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java new file mode 100644 index 00000000000..58ea4b86d98 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java @@ -0,0 +1,40 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +import com.metamx.druid.kv.Indexed; +import com.metamx.druid.kv.IndexedFloats; +import com.metamx.druid.kv.IndexedLongs; + +import java.io.Closeable; + +/** + */ +public interface GenericColumn extends Closeable +{ + public int size(); + + public String getStringSingleValueRow(int rowNum); + public Indexed getStringMultiValueRow(int rowNum); + public float getFloatSingleValueRow(int rowNum); + public IndexedFloats getFloatMultiValueRow(int rowNum); + public long getLongSingleValueRow(int rowNum); + public IndexedLongs getLongMultiValueRow(int rowNum); +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java new file mode 100644 index 00000000000..192734aeec2 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java @@ -0,0 +1,54 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +import com.metamx.druid.kv.Indexed; + +import java.io.IOException; + +/** +*/ +public class IndexedComplexColumn implements ComplexColumn +{ + private final Indexed column; + + public IndexedComplexColumn( + Indexed column + ) + { + this.column = column; + } + @Override + public Class getClazz() + { + return column.getClazz(); + } + + @Override + public Object getRowValue(int rowNum) + { + return column.get(rowNum); + } + + @Override + public void close() throws IOException + { + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java new file mode 100644 index 00000000000..815ff84c583 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java @@ -0,0 +1,87 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +import com.metamx.druid.kv.Indexed; +import com.metamx.druid.kv.IndexedFloats; +import com.metamx.druid.kv.IndexedLongs; + +import java.io.IOException; + +/** +*/ +public class IndexedFloatsGenericColumn implements GenericColumn +{ + private final IndexedFloats column; + + public IndexedFloatsGenericColumn( + final IndexedFloats column + ) { + this.column = column; + } + + @Override + public int size() + { + return column.size(); + } + + @Override + public String getStringSingleValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public Indexed getStringMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public float getFloatSingleValueRow(int rowNum) + { + return column.get(rowNum); + } + + @Override + public IndexedFloats getFloatMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public long getLongSingleValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public IndexedLongs getLongMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws IOException + { + column.close(); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java new file mode 100644 index 00000000000..01ff6063870 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java @@ -0,0 +1,87 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +import com.metamx.druid.kv.Indexed; +import com.metamx.druid.kv.IndexedFloats; +import com.metamx.druid.kv.IndexedLongs; + +import java.io.IOException; + +/** +*/ +public class IndexedLongsGenericColumn implements GenericColumn +{ + private final IndexedLongs column; + + public IndexedLongsGenericColumn( + final IndexedLongs column + ) { + this.column = column; + } + + @Override + public int size() + { + return column.size(); + } + + @Override + public String getStringSingleValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public Indexed getStringMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public float getFloatSingleValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public IndexedFloats getFloatMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public long getLongSingleValueRow(int rowNum) + { + return column.get(rowNum); + } + + @Override + public IndexedLongs getLongMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws IOException + { + column.close(); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/LongColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/LongColumn.java new file mode 100644 index 00000000000..76d74ba377e --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/LongColumn.java @@ -0,0 +1,50 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +import com.google.common.base.Supplier; +import com.metamx.druid.kv.IndexedLongs; + +/** + */ +public class LongColumn extends AbstractColumn +{ + private static final ColumnCapabilitiesImpl CAPABILITIES = new ColumnCapabilitiesImpl() + .setType(ValueType.LONG); + + private final Supplier column; + + public LongColumn(Supplier column) + { + this.column = column; + } + + @Override + public ColumnCapabilities getCapabilities() + { + return CAPABILITIES; + } + + @Override + public GenericColumn getGenericColumn() + { + return new IndexedLongsGenericColumn(column.get()); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/RunLengthColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/RunLengthColumn.java new file mode 100644 index 00000000000..5242e994008 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/RunLengthColumn.java @@ -0,0 +1,27 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +/** + */ +public interface RunLengthColumn +{ + public void thisIsAFictionalInterfaceThatWillHopefullyMeanSomethingSometime(); +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java new file mode 100644 index 00000000000..2d3fae51abf --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java @@ -0,0 +1,87 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +import com.google.common.base.Supplier; + +/** + */ +class SimpleColumn implements Column +{ + private final ColumnCapabilities capabilities; + private final Supplier dictionaryEncodedColumn; + private final Supplier runLengthColumn; + private final Supplier genericColumn; + private final Supplier complexColumn; + private final Supplier bitmapIndex; + + SimpleColumn( + ColumnCapabilities capabilities, + Supplier dictionaryEncodedColumn, + Supplier runLengthColumn, + Supplier genericColumn, + Supplier complexColumn, + Supplier bitmapIndex + ) + { + this.capabilities = capabilities; + this.dictionaryEncodedColumn = dictionaryEncodedColumn; + this.runLengthColumn = runLengthColumn; + this.genericColumn = genericColumn; + this.complexColumn = complexColumn; + this.bitmapIndex = bitmapIndex; + } + + @Override + public ColumnCapabilities getCapabilities() + { + return capabilities; + } + + @Override + public DictionaryEncodedColumn getDictionaryEncoding() + { + return dictionaryEncodedColumn.get(); + } + + @Override + public RunLengthColumn getRunLengthColumn() + { + return runLengthColumn.get(); + } + + @Override + public GenericColumn getGenericColumn() + { + return genericColumn.get(); + } + + @Override + public ComplexColumn getComplexColumn() + { + return complexColumn.get(); + } + + @Override + public BitmapIndex getBitmapIndex() + { + return bitmapIndex.get(); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java new file mode 100644 index 00000000000..87bc5bb6a2d --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java @@ -0,0 +1,81 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +import com.metamx.druid.kv.GenericIndexed; +import com.metamx.druid.kv.IndexedInts; +import com.metamx.druid.kv.VSizeIndexed; +import com.metamx.druid.kv.VSizeIndexedInts; + +/** +*/ +public class SimpleDictionaryEncodedColumn implements DictionaryEncodedColumn +{ + private final VSizeIndexedInts column; + private final VSizeIndexed multiValueColumn; + private final GenericIndexed lookups; + + public SimpleDictionaryEncodedColumn( + VSizeIndexedInts singleValueColumn, + VSizeIndexed multiValueColumn, + GenericIndexed lookups + ) + { + this.column = singleValueColumn; + this.multiValueColumn = multiValueColumn; + this.lookups = lookups; + } + + @Override + public int size() + { + return column == null ? multiValueColumn.size() : column.size(); + } + + @Override + public int getSingleValueRow(int rowNum) + { + return column.get(rowNum); + } + + @Override + public IndexedInts getMultiValueRow(int rowNum) + { + return multiValueColumn.get(rowNum); + } + + @Override + public String lookupName(int id) + { + return lookups.get(id); + } + + @Override + public int lookupId(String name) + { + return lookups.indexOf(name); + } + + @Override + public int getCardinality() + { + return lookups.size(); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/StringMultiValueColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/StringMultiValueColumn.java new file mode 100644 index 00000000000..7d52e42ba4f --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/StringMultiValueColumn.java @@ -0,0 +1,106 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.column; + +import com.metamx.druid.kv.Indexed; +import com.metamx.druid.kv.IndexedInts; +import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; + +/** + */ +public class StringMultiValueColumn extends AbstractColumn +{ + private static final ImmutableConciseSet emptySet = new ImmutableConciseSet(); + private static final ColumnCapabilitiesImpl CAPABILITIES = new ColumnCapabilitiesImpl() + .setType(ValueType.STRING) + .setDictionaryEncoded(true) + .setHasBitmapIndexes(true) + .setHasMultipleValues(true); + + private final Indexed lookups; + private final Indexed column; + private final Indexed bitmapIndexes; + + public StringMultiValueColumn( + Indexed lookups, + Indexed column, + Indexed bitmapIndexes + ) + { + this.lookups = lookups; + this.column = column; + this.bitmapIndexes = bitmapIndexes; + } + + @Override + public ColumnCapabilities getCapabilities() + { + return CAPABILITIES; + } + + @Override + public DictionaryEncodedColumn getDictionaryEncoding() + { + return new DictionaryEncodedColumn() + { + @Override + public int size() + { + return column.size(); + } + + @Override + public int getSingleValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public IndexedInts getMultiValueRow(int rowNum) + { + return column.get(rowNum); + } + + @Override + public String lookupName(int id) + { + return lookups.get(id); + } + + @Override + public int lookupId(String name) + { + return lookups.indexOf(name); + } + + @Override + public int getCardinality() + { + return lookups.size(); + } + }; + } + + @Override + public BitmapIndex getBitmapIndex() + { + throw new UnsupportedOperationException(); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ValueType.java b/index-common/src/main/java/com/metamx/druid/index/column/ValueType.java new file mode 100644 index 00000000000..245ff682bd7 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/column/ValueType.java @@ -0,0 +1,11 @@ +package com.metamx.druid.index.column; + +/** +*/ +public enum ValueType +{ + FLOAT, + LONG, + STRING, + COMPLEX +} diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java b/index-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java new file mode 100644 index 00000000000..b282ab56a5c --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java @@ -0,0 +1,58 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.serde; + +import com.google.common.base.Supplier; +import com.metamx.druid.index.column.BitmapIndex; +import com.metamx.druid.kv.GenericIndexed; +import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; + +/** +*/ +public class BitmapIndexColumnPartSupplier implements Supplier +{ + private static final ImmutableConciseSet EMPTY_SET = new ImmutableConciseSet(); + + private final GenericIndexed bitmaps; + private final GenericIndexed dictionary; + + public BitmapIndexColumnPartSupplier( + GenericIndexed bitmaps, + GenericIndexed dictionary + ) { + this.bitmaps = bitmaps; + this.dictionary = dictionary; + } + + @Override + public BitmapIndex get() + { + return new BitmapIndex() + { + @Override + public ImmutableConciseSet getConciseSet(String value) + { + final int index = dictionary.indexOf(value); + + return index >= 0 ? bitmaps.get(index) : EMPTY_SET; + } + }; + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java new file mode 100644 index 00000000000..d60428fbe3e --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java @@ -0,0 +1,44 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.serde; + +import com.metamx.druid.index.column.ColumnBuilder; +import org.codehaus.jackson.annotate.JsonSubTypes; +import org.codehaus.jackson.annotate.JsonTypeInfo; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +/** + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "complex", value = ComplexColumnPartSerde.class), + @JsonSubTypes.Type(name = "float", value = FloatGenericColumnPartSerde.class), + @JsonSubTypes.Type(name = "long", value = LongGenericColumnPartSerde.class), + @JsonSubTypes.Type(name = "stringDictionary", value = DictionaryEncodedColumnPartSerde.class) +}) +public interface ColumnPartSerde +{ + public int numBytes(); + public void write(WritableByteChannel channel) throws IOException; + public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder); +} diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java new file mode 100644 index 00000000000..7e425f9f525 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java @@ -0,0 +1,77 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.serde; + +import com.metamx.druid.index.column.ColumnBuilder; +import com.metamx.druid.index.v1.serde.ComplexMetricSerde; +import com.metamx.druid.index.v1.serde.ComplexMetrics; +import com.metamx.druid.kv.GenericIndexed; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +/** +*/ +public class ComplexColumnPartSerde implements ColumnPartSerde +{ + @JsonCreator + public static ComplexColumnPartSerde createDeserializer( + @JsonProperty("complexType") String complexType + ) + { + return new ComplexColumnPartSerde(null, complexType); + } + + private final GenericIndexed column; + private final ComplexMetricSerde serde; + + public ComplexColumnPartSerde(GenericIndexed column, String complexType) + { + this.column = column; + serde = ComplexMetrics.getSerdeForType(complexType); + } + + @JsonProperty + public GenericIndexed getColumn() + { + return column; + } + + @Override + public int numBytes() + { + return column.getSerializedSize(); + } + + @Override + public void write(WritableByteChannel channel) throws IOException + { + column.writeToChannel(channel); + } + + @Override + public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder) + { + return serde.deserializeColumn(buffer, builder); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java b/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java new file mode 100644 index 00000000000..cb1a4bea395 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java @@ -0,0 +1,47 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.serde; + +import com.google.common.base.Supplier; +import com.metamx.druid.index.column.ComplexColumn; +import com.metamx.druid.index.column.IndexedComplexColumn; +import com.metamx.druid.kv.GenericIndexed; + +/** +*/ +public class ComplexColumnPartSupplier implements Supplier +{ + private final GenericIndexed complexType; + private final String typeName; + + public ComplexColumnPartSupplier( + final GenericIndexed complexType, + final String typeName + ) { + this.complexType = complexType; + this.typeName = typeName; + } + + @Override + public ComplexColumn get() + { + return new IndexedComplexColumn(complexType); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java new file mode 100644 index 00000000000..d30c6c6bbc0 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java @@ -0,0 +1,147 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.serde; + +import com.metamx.common.IAE; +import com.metamx.druid.index.column.ColumnBuilder; +import com.metamx.druid.index.column.ValueType; +import com.metamx.druid.kv.ConciseCompressedIndexedInts; +import com.metamx.druid.kv.GenericIndexed; +import com.metamx.druid.kv.VSizeIndexed; +import com.metamx.druid.kv.VSizeIndexedInts; +import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +/** +*/ +public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde +{ + @JsonCreator + public static DictionaryEncodedColumnPartSerde createDeserializer(boolean singleValued) + { + return new DictionaryEncodedColumnPartSerde(); + } + + private final GenericIndexed dictionary; + private final VSizeIndexedInts singleValuedColumn; + private final VSizeIndexed multiValuedColumn; + private final GenericIndexed bitmaps; + + private final int size; + + public DictionaryEncodedColumnPartSerde( + GenericIndexed dictionary, + VSizeIndexedInts singleValCol, + VSizeIndexed multiValCol, + GenericIndexed bitmaps + ) + { + this.dictionary = dictionary; + this.singleValuedColumn = singleValCol; + this.multiValuedColumn = multiValCol; + this.bitmaps = bitmaps; + + int size = dictionary.getSerializedSize(); + if (singleValCol != null && multiValCol == null) { + size += singleValCol.getSerializedSize(); + } + else if (singleValCol == null && multiValCol != null) { + size += multiValCol.getSerializedSize(); + } + else { + throw new IAE("Either singleValCol[%s] or multiValCol[%s] must be set", singleValCol, multiValCol); + } + size += bitmaps.getSerializedSize(); + + this.size = size; + } + + private DictionaryEncodedColumnPartSerde() + { + dictionary = null; + singleValuedColumn = null; + multiValuedColumn = null; + bitmaps = null; + size = 0; + } + + @JsonProperty + private boolean isSingleValued() + { + return singleValuedColumn != null; + } + + @Override + public int numBytes() + { + return 1 + size; + } + + @Override + public void write(WritableByteChannel channel) throws IOException + { + channel.write(ByteBuffer.wrap(new byte[]{(byte) (isSingleValued() ? 0x0 : 0x1)})); + dictionary.writeToChannel(channel); + if (isSingleValued()) { + singleValuedColumn.writeToChannel(channel); + } + else { + multiValuedColumn.writeToChannel(channel); + } + bitmaps.writeToChannel(channel); + } + + @Override + public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder) + { + final boolean isSingleValued = buffer.get() == 0x0; + final GenericIndexed dictionary = GenericIndexed.read(buffer, GenericIndexed.stringStrategy); + final VSizeIndexedInts singleValuedColumn; + final VSizeIndexed multiValuedColumn; + + builder.setType(ValueType.STRING); + + if (isSingleValued) { + singleValuedColumn = VSizeIndexedInts.readFromByteBuffer(buffer); + multiValuedColumn = null; + builder.setHasMultipleValues(false) + .setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, singleValuedColumn, null)); + } + else { + singleValuedColumn = null; + multiValuedColumn = VSizeIndexed.readFromByteBuffer(buffer); + builder.setHasMultipleValues(true) + .setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, null, multiValuedColumn)); + } + + GenericIndexed bitmaps = GenericIndexed.read( + buffer, ConciseCompressedIndexedInts.objectStrategy + ); + + builder.setBitmapIndex(new BitmapIndexColumnPartSupplier(bitmaps, dictionary)); + + return new DictionaryEncodedColumnPartSerde(dictionary, singleValuedColumn, multiValuedColumn, bitmaps); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnSupplier.java b/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnSupplier.java new file mode 100644 index 00000000000..40285cf550d --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnSupplier.java @@ -0,0 +1,53 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.serde; + +import com.google.common.base.Supplier; +import com.metamx.druid.index.column.DictionaryEncodedColumn; +import com.metamx.druid.index.column.SimpleDictionaryEncodedColumn; +import com.metamx.druid.kv.GenericIndexed; +import com.metamx.druid.kv.VSizeIndexed; +import com.metamx.druid.kv.VSizeIndexedInts; + +/** +*/ +public class DictionaryEncodedColumnSupplier implements Supplier +{ + private final GenericIndexed dictionary; + private final VSizeIndexedInts singleValuedColumn; + private final VSizeIndexed multiValuedColumn; + + public DictionaryEncodedColumnSupplier( + GenericIndexed dictionary, + VSizeIndexedInts singleValuedColumn, + VSizeIndexed multiValuedColumn + ) + { + this.dictionary = dictionary; + this.singleValuedColumn = singleValuedColumn; + this.multiValuedColumn = multiValuedColumn; + } + + @Override + public DictionaryEncodedColumn get() + { + return new SimpleDictionaryEncodedColumn(singleValuedColumn, multiValuedColumn, dictionary); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java new file mode 100644 index 00000000000..87cb95758db --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java @@ -0,0 +1,83 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.serde; + +import com.metamx.druid.index.column.ColumnBuilder; +import com.metamx.druid.index.column.ValueType; +import com.metamx.druid.index.v1.CompressedFloatsIndexedSupplier; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +/** +*/ +public class FloatGenericColumnPartSerde implements ColumnPartSerde +{ + @JsonCreator + public static FloatGenericColumnPartSerde createDeserializer( + @JsonProperty("byteOrder") ByteOrder byteOrder + ) + { + return new FloatGenericColumnPartSerde(null, byteOrder); + } + + private final CompressedFloatsIndexedSupplier compressedFloats; + private final ByteOrder byteOrder; + + public FloatGenericColumnPartSerde(CompressedFloatsIndexedSupplier compressedFloats, ByteOrder byteOrder) + { + this.compressedFloats = compressedFloats; + this.byteOrder = byteOrder; + } + + @JsonProperty + public ByteOrder getByteOrder() + { + return byteOrder; + } + + @Override + public int numBytes() + { + return compressedFloats.getSerializedSize(); + } + + @Override + public void write(WritableByteChannel channel) throws IOException + { + compressedFloats.writeToChannel(channel); + } + + @Override + public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder) + { + final CompressedFloatsIndexedSupplier column = CompressedFloatsIndexedSupplier.fromByteBuffer(buffer, byteOrder); + + builder.setType(ValueType.FLOAT) + .setHasMultipleValues(false) + .setGenericColumn(new FloatGenericColumnSupplier(column, byteOrder)); + + return new FloatGenericColumnPartSerde(column, byteOrder); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnSupplier.java b/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnSupplier.java new file mode 100644 index 00000000000..c13a5c11942 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnSupplier.java @@ -0,0 +1,49 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.serde; + +import com.google.common.base.Supplier; +import com.metamx.druid.index.column.GenericColumn; +import com.metamx.druid.index.column.IndexedFloatsGenericColumn; +import com.metamx.druid.index.v1.CompressedFloatsIndexedSupplier; + +import java.nio.ByteOrder; + +/** +*/ +public class FloatGenericColumnSupplier implements Supplier +{ + private final CompressedFloatsIndexedSupplier column; + private final ByteOrder byteOrder; + + public FloatGenericColumnSupplier( + CompressedFloatsIndexedSupplier column, + ByteOrder byteOrder + ) { + this.column = column; + this.byteOrder = byteOrder; + } + + @Override + public GenericColumn get() + { + return new IndexedFloatsGenericColumn(column.get()); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java new file mode 100644 index 00000000000..0d140751e4c --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java @@ -0,0 +1,83 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.serde; + +import com.metamx.druid.index.column.ColumnBuilder; +import com.metamx.druid.index.column.ValueType; +import com.metamx.druid.index.v1.CompressedLongsIndexedSupplier; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +/** +*/ +public class LongGenericColumnPartSerde implements ColumnPartSerde +{ + @JsonCreator + public static LongGenericColumnPartSerde createDeserializer( + @JsonProperty("byteOrder") ByteOrder byteOrder + ) + { + return new LongGenericColumnPartSerde(null, byteOrder); + } + + private final CompressedLongsIndexedSupplier compressedLongs; + private final ByteOrder byteOrder; + + public LongGenericColumnPartSerde(CompressedLongsIndexedSupplier compressedLongs, ByteOrder byteOrder) + { + this.compressedLongs = compressedLongs; + this.byteOrder = byteOrder; + } + + @JsonProperty + public ByteOrder getByteOrder() + { + return byteOrder; + } + + @Override + public int numBytes() + { + return compressedLongs.getSerializedSize(); + } + + @Override + public void write(WritableByteChannel channel) throws IOException + { + compressedLongs.writeToChannel(channel); + } + + @Override + public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder) + { + final CompressedLongsIndexedSupplier column = CompressedLongsIndexedSupplier.fromByteBuffer(buffer, byteOrder); + + builder.setType(ValueType.LONG) + .setHasMultipleValues(false) + .setGenericColumn(new LongGenericColumnSupplier(column)); + + return new LongGenericColumnPartSerde(column, byteOrder); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnSupplier.java b/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnSupplier.java new file mode 100644 index 00000000000..babdf7949f6 --- /dev/null +++ b/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnSupplier.java @@ -0,0 +1,46 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.serde; + +import com.google.common.base.Supplier; +import com.metamx.druid.index.column.GenericColumn; +import com.metamx.druid.index.column.IndexedLongsGenericColumn; +import com.metamx.druid.index.v1.CompressedLongsIndexedSupplier; + +import java.nio.ByteOrder; + +/** +*/ +public class LongGenericColumnSupplier implements Supplier +{ + private final CompressedLongsIndexedSupplier column; + + public LongGenericColumnSupplier( + CompressedLongsIndexedSupplier column + ) { + this.column = column; + } + + @Override + public GenericColumn get() + { + return new IndexedLongsGenericColumn(column.get()); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java b/index-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java index 12751ebebe8..2ff8f747adc 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java @@ -29,7 +29,6 @@ import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.collect.StupidResourceHolder; import com.metamx.druid.kv.GenericIndexed; import com.metamx.druid.kv.IndexedFloats; -import com.metamx.druid.kv.IndexedIterable; import java.io.IOException; import java.nio.ByteBuffer; @@ -146,6 +145,11 @@ public class CompressedFloatsIndexedSupplier implements Supplier }; } + public int getSerializedSize() + { + return baseFloatBuffers.getSerializedSize() + 1 + 4 + 4; + } + public void writeToChannel(WritableByteChannel channel) throws IOException { channel.write(ByteBuffer.wrap(new byte[]{version})); @@ -178,7 +182,7 @@ public class CompressedFloatsIndexedSupplier implements Supplier return MAX_FLOATS_IN_BUFFER - (MAX_FLOATS_IN_BUFFER % numFloatsInChunk); } - public static CompressedFloatsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) throws IOException + public static CompressedFloatsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) { byte versionFromBuffer = buffer.get(); @@ -186,7 +190,7 @@ public class CompressedFloatsIndexedSupplier implements Supplier return new CompressedFloatsIndexedSupplier( buffer.getInt(), buffer.getInt(), - GenericIndexed.readFromByteBuffer(buffer, CompressedFloatBufferObjectStrategy.getBufferForOrder(order)) + GenericIndexed.read(buffer, CompressedFloatBufferObjectStrategy.getBufferForOrder(order)) ); } diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplier.java b/index-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplier.java index 2f37d393763..d66b68a0c3d 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplier.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplier.java @@ -28,7 +28,6 @@ import com.metamx.common.IAE; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.collect.StupidResourceHolder; import com.metamx.druid.kv.GenericIndexed; -import com.metamx.druid.kv.IndexedIterable; import com.metamx.druid.kv.IndexedLongs; import java.io.IOException; @@ -162,6 +161,11 @@ public class CompressedLongsIndexedSupplier implements Supplier }; } + public int getSerializedSize() + { + return baseLongBuffers.getSerializedSize() + 1 + 4 + 4; + } + public void writeToChannel(WritableByteChannel channel) throws IOException { channel.write(ByteBuffer.wrap(new byte[]{version})); @@ -188,7 +192,7 @@ public class CompressedLongsIndexedSupplier implements Supplier return baseLongBuffers; } - public static CompressedLongsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) throws IOException + public static CompressedLongsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) { byte versionFromBuffer = buffer.get(); @@ -196,7 +200,7 @@ public class CompressedLongsIndexedSupplier implements Supplier return new CompressedLongsIndexedSupplier( buffer.getInt(), buffer.getInt(), - GenericIndexed.readFromByteBuffer(buffer, CompressedLongBufferObjectStrategy.getBufferForOrder(order)) + GenericIndexed.read(buffer, CompressedLongBufferObjectStrategy.getBufferForOrder(order)) ); } diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index 4b5512e5ee7..2afcbed6446 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -20,33 +20,91 @@ package com.metamx.druid.index.v1; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; +import com.google.common.io.Files; +import com.google.common.primitives.Ints; +import com.metamx.common.IAE; import com.metamx.common.ISE; +import com.metamx.common.io.smoosh.FileSmoosher; import com.metamx.common.io.smoosh.Smoosh; import com.metamx.common.io.smoosh.SmooshedFileMapper; +import com.metamx.common.io.smoosh.SmooshedWriter; import com.metamx.common.logger.Logger; +import com.metamx.druid.index.QueryableIndex; +import com.metamx.druid.index.SimpleQueryableIndex; +import com.metamx.druid.index.column.Column; +import com.metamx.druid.index.column.ColumnBuilder; +import com.metamx.druid.index.column.ColumnDescriptor; +import com.metamx.druid.index.column.ValueType; +import com.metamx.druid.index.serde.BitmapIndexColumnPartSupplier; +import com.metamx.druid.index.serde.ComplexColumnPartSerde; +import com.metamx.druid.index.serde.ComplexColumnPartSupplier; +import com.metamx.druid.index.serde.DictionaryEncodedColumnPartSerde; +import com.metamx.druid.index.serde.DictionaryEncodedColumnSupplier; +import com.metamx.druid.index.serde.FloatGenericColumnPartSerde; +import com.metamx.druid.index.serde.FloatGenericColumnSupplier; +import com.metamx.druid.index.serde.LongGenericColumnPartSerde; +import com.metamx.druid.index.serde.LongGenericColumnSupplier; +import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.kv.ArrayIndexed; import com.metamx.druid.kv.ConciseCompressedIndexedInts; import com.metamx.druid.kv.GenericIndexed; import com.metamx.druid.kv.IndexedIterable; import com.metamx.druid.kv.VSizeIndexed; +import com.metamx.druid.kv.VSizeIndexedInts; import com.metamx.druid.utils.SerializerUtils; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; +import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.Interval; +import java.io.ByteArrayOutputStream; import java.io.File; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.util.AbstractList; +import java.util.LinkedHashSet; +import java.util.List; import java.util.Map; +import java.util.Set; /** * This should be changed to use DI instead of a static reference... */ public class IndexIO { + public static final byte V8_VERSION = 0x8; + public static final byte V9_VERSION = 0x9; + + private static final Map indexLoaders = + ImmutableMap.builder() + .put(0, new LegacyIndexLoader()) + .put(1, new LegacyIndexLoader()) + .put(2, new LegacyIndexLoader()) + .put(3, new LegacyIndexLoader()) + .put(4, new LegacyIndexLoader()) + .put(5, new LegacyIndexLoader()) + .put(6, new LegacyIndexLoader()) + .put(7, new LegacyIndexLoader()) + .put(8, new LegacyIndexLoader()) + .put(9, new V9IndexLoader()) + .build(); + + private static final Logger log = new Logger(IndexIO.class); + private static final SerializerUtils serializerUtils = new SerializerUtils(); + private static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder(); + + // This should really be provided by DI, should be changed once we switch around to using a DI framework + private static final ObjectMapper mapper = new DefaultObjectMapper(); + + private static volatile IndexIOHandler handler = null; public static final byte CURRENT_VERSION_ID = 0x8; @@ -68,6 +126,20 @@ public class IndexIO return handler.mapDir(inDir); } + public static QueryableIndex loadIndex(File inDir) throws IOException + { + init(); + final int version = getVersionFromDir(inDir); + + final IndexLoader loader = indexLoaders.get(version); + + if (loader != null) { + return loader.load(inDir); + } else { + throw new ISE("Unknown index version[%s]", version); + } + } + public static void storeLatest(Index index, File file) throws IOException { handler.storeLatest(index, file); @@ -82,8 +154,7 @@ public class IndexIO { if (IndexIO.handler == null) { IndexIO.handler = handler; - } - else { + } else { throw new ISE("Already have a handler[%s], cannot register another[%s]", IndexIO.handler, handler); } } @@ -95,6 +166,26 @@ public class IndexIO } } + private static int getVersionFromDir(File inDir) throws IOException + { + File versionFile = new File(inDir, "version.bin"); + if (versionFile.exists()) { + return Ints.fromByteArray(Files.toByteArray(versionFile)); + } + + final File indexFile = new File(inDir, "index.drd"); + InputStream in = null; + int version; + try { + in = new FileInputStream(indexFile); + version = in.read(); + } + finally { + Closeables.closeQuietly(in); + } + return version; + } + public static void checkFileSize(File indexFile) throws IOException { final long fileSize = indexFile.length(); @@ -128,7 +219,6 @@ public class IndexIO * This only exists for some legacy compatibility reasons, Metamarkets is working on getting rid of it in * future versions. Normal persisting of indexes is done via IndexMerger. * - * * @param file */ public void storeLatest(Index index, File file) throws IOException; @@ -137,9 +227,6 @@ public class IndexIO static class DefaultIndexIOHandler implements IndexIOHandler { private static final Logger log = new Logger(DefaultIndexIOHandler.class); - private static final SerializerUtils serializerUtils = new SerializerUtils(); - private static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder(); - @Override public Index readIndex(File inDir) { @@ -152,8 +239,6 @@ public class IndexIO return true; } - public static final byte VERSION_ID = 0x8; - @Override public MMappedIndex mapDir(File inDir) throws IOException { @@ -164,7 +249,7 @@ public class IndexIO try { indexIn = new FileInputStream(new File(inDir, "index.drd")); byte theVersion = (byte) indexIn.read(); - if (theVersion != VERSION_ID) { + if (theVersion != V8_VERSION) { throw new IllegalArgumentException(String.format("Unknown version[%s]", theVersion)); } } @@ -176,10 +261,10 @@ public class IndexIO ByteBuffer indexBuffer = smooshedFiles.mapFile("index.drd"); indexBuffer.get(); // Skip the version byte - final GenericIndexed availableDimensions = GenericIndexed.readFromByteBuffer( + final GenericIndexed availableDimensions = GenericIndexed.read( indexBuffer, GenericIndexed.stringStrategy ); - final GenericIndexed availableMetrics = GenericIndexed.readFromByteBuffer( + final GenericIndexed availableMetrics = GenericIndexed.read( indexBuffer, GenericIndexed.stringStrategy ); final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer)); @@ -213,7 +298,7 @@ public class IndexIO fileDimensionName ); - dimValueLookups.put(dimension, GenericIndexed.readFromByteBuffer(dimBuffer, GenericIndexed.stringStrategy)); + dimValueLookups.put(dimension, GenericIndexed.read(dimBuffer, GenericIndexed.stringStrategy)); dimColumns.put(dimension, VSizeIndexed.readFromByteBuffer(dimBuffer)); } @@ -221,7 +306,7 @@ public class IndexIO for (int i = 0; i < availableDimensions.size(); ++i) { invertedIndexed.put( serializerUtils.readString(invertedBuffer), - GenericIndexed.readFromByteBuffer(invertedBuffer, ConciseCompressedIndexedInts.objectStrategy) + GenericIndexed.read(invertedBuffer, ConciseCompressedIndexedInts.objectStrategy) ); } @@ -246,6 +331,333 @@ public class IndexIO { throw new UnsupportedOperationException("Shouldn't ever happen in a cluster that is not owned by MMX."); } + + public void convertV8toV9(File v8Dir, File v9Dir) throws IOException + { + log.info("Converting v8[%s] to v9[%s]", v8Dir, v9Dir); + + InputStream indexIn = null; + try { + indexIn = new FileInputStream(new File(v8Dir, "index.drd")); + byte theVersion = (byte) indexIn.read(); + if (theVersion != V8_VERSION) { + throw new IAE("Unknown version[%s]", theVersion); + } + } + finally { + Closeables.close(indexIn, false); + } + + SmooshedFileMapper v8SmooshedFiles = Smoosh.map(v8Dir); + + v9Dir.mkdirs(); + final FileSmoosher v9Smoosher = new FileSmoosher(v9Dir); + + ByteStreams.write(Ints.toByteArray(9), Files.newOutputStreamSupplier(new File(v9Dir, "version.bin"))); + Map> bitmapIndexes = Maps.newHashMap(); + + final ByteBuffer invertedBuffer = v8SmooshedFiles.mapFile("inverted.drd"); + while (invertedBuffer.hasRemaining()) { + bitmapIndexes.put( + serializerUtils.readString(invertedBuffer), + GenericIndexed.read(invertedBuffer, ConciseCompressedIndexedInts.objectStrategy) + ); + } + + LinkedHashSet skippedFiles = Sets.newLinkedHashSet(); + for (String filename : v8SmooshedFiles.getInternalFilenames()) { + log.info("Processing file[%s]", filename); + if (filename.startsWith("dim_")) { + final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); + builder.setValueType(ValueType.STRING); + + final List outParts = Lists.newArrayList(); + + ByteBuffer dimBuffer = v8SmooshedFiles.mapFile(filename); + String dimension = serializerUtils.readString(dimBuffer); + if (!filename.equals(String.format("dim_%s.drd", dimension))) { + throw new ISE("loaded dimension[%s] from file[%s]", dimension, filename); + } + + ByteArrayOutputStream nameBAOS = new ByteArrayOutputStream(); + serializerUtils.writeString(nameBAOS, dimension); + outParts.add(ByteBuffer.wrap(nameBAOS.toByteArray())); + + final GenericIndexed dictionary = GenericIndexed.read( + dimBuffer, GenericIndexed.stringStrategy + ); + VSizeIndexedInts singleValCol = null; + VSizeIndexed multiValCol = VSizeIndexed.readFromByteBuffer(dimBuffer.asReadOnlyBuffer()); + boolean onlyOneValue = true; + for (VSizeIndexedInts rowValue : multiValCol) { + if (!onlyOneValue) { + break; + } + if (rowValue.size() > 1) { + onlyOneValue = false; + } + } + + if (onlyOneValue) { + final VSizeIndexed finalMultiValCol = multiValCol; + singleValCol = VSizeIndexedInts.fromList( + new AbstractList() + { + @Override + public Integer get(int index) + { + return finalMultiValCol.get(index).get(0); + } + + @Override + public int size() + { + return finalMultiValCol.size(); + } + }, + dictionary.size() + ); + multiValCol = null; + } else { + builder.setHasMultipleValues(true); + } + + builder.addSerde( + new DictionaryEncodedColumnPartSerde(dictionary, singleValCol, multiValCol, bitmapIndexes.get(dimension)) + ); + + final ColumnDescriptor serdeficator = builder.build(); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + serializerUtils.writeString(baos, mapper.writeValueAsString(serdeficator)); + byte[] specBytes = baos.toByteArray(); + + final SmooshedWriter channel = v9Smoosher.addWithSmooshedWriter( + dimension, serdeficator.numBytes() + specBytes.length + ); + channel.write(ByteBuffer.wrap(specBytes)); + serdeficator.write(channel); + channel.close(); + } else if (filename.startsWith("met_")) { + if (!filename.endsWith(String.format("%s.drd", BYTE_ORDER))) { + skippedFiles.add(filename); + continue; + } + + MetricHolder holder = MetricHolder.fromByteBuffer(v8SmooshedFiles.mapFile(filename)); + final String metric = holder.getName(); + + final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); + + switch (holder.getType()) { + case FLOAT: + builder.setValueType(ValueType.FLOAT); + builder.addSerde(new FloatGenericColumnPartSerde(holder.floatType, BYTE_ORDER)); + break; + case COMPLEX: + if (!(holder.complexType instanceof GenericIndexed)) { + throw new ISE("Serialized complex types must be GenericIndexed objects."); + } + final GenericIndexed column = (GenericIndexed) holder.complexType; + final String complexType = holder.getTypeName(); + + builder.setValueType(ValueType.COMPLEX); + builder.addSerde(new ComplexColumnPartSerde(column, complexType)); + break; + default: + throw new ISE("Unknown type[%s]", holder.getType()); + } + + final ColumnDescriptor serdeficator = builder.build(); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + serializerUtils.writeString(baos, mapper.writeValueAsString(serdeficator)); + byte[] specBytes = baos.toByteArray(); + + final SmooshedWriter channel = v9Smoosher.addWithSmooshedWriter( + metric, serdeficator.numBytes() + specBytes.length + ); + channel.write(ByteBuffer.wrap(specBytes)); + serdeficator.write(channel); + channel.close(); + } else if (String.format("time_%s.drd", BYTE_ORDER).equals(filename)) { + CompressedLongsIndexedSupplier timestamps = CompressedLongsIndexedSupplier.fromByteBuffer( + v8SmooshedFiles.mapFile(filename), BYTE_ORDER + ); + + final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); + builder.setValueType(ValueType.LONG); + builder.addSerde(new LongGenericColumnPartSerde(timestamps, BYTE_ORDER)); + + final ColumnDescriptor serdeficator = builder.build(); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + serializerUtils.writeString(baos, mapper.writeValueAsString(serdeficator)); + byte[] specBytes = baos.toByteArray(); + + final SmooshedWriter channel = v9Smoosher.addWithSmooshedWriter( + "__time", serdeficator.numBytes() + specBytes.length + ); + channel.write(ByteBuffer.wrap(specBytes)); + serdeficator.write(channel); + channel.close(); + } else if ("index.drd".equals(filename)) { + final ByteBuffer indexBuffer = v8SmooshedFiles.mapFile(filename); + + indexBuffer.get(); // Skip the version byte + final GenericIndexed dims = GenericIndexed.read( + indexBuffer, GenericIndexed.stringStrategy + ); + final GenericIndexed availableMetrics = GenericIndexed.read( + indexBuffer, GenericIndexed.stringStrategy + ); + final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer)); + + Set columns = Sets.newTreeSet(); + columns.addAll(Lists.newArrayList(dims)); + columns.addAll(Lists.newArrayList(availableMetrics)); + + GenericIndexed cols = GenericIndexed.fromIterable(columns, GenericIndexed.stringStrategy); + + final int numBytes = cols.getSerializedSize() + dims.getSerializedSize() + 16; + final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes); + cols.writeToChannel(writer); + dims.writeToChannel(writer); + serializerUtils.writeLong(writer, dataInterval.getStartMillis()); + serializerUtils.writeLong(writer, dataInterval.getEndMillis()); + writer.close(); + } else { + skippedFiles.add(filename); + } + } + log.info("Skipped files[%s]", skippedFiles); + + v9Smoosher.close(); + } + } + + static interface IndexLoader + { + public QueryableIndex load(File inDir) throws IOException; + } + + static class LegacyIndexLoader implements IndexLoader + { + @Override + public QueryableIndex load(File inDir) throws IOException + { + MMappedIndex index = IndexIO.mapDir(inDir); + + Map columns = Maps.newHashMap(); + + for (String dimension : index.getAvailableDimensions()) { + columns.put( + dimension.toLowerCase(), + new ColumnBuilder() + .setType(ValueType.STRING) + .setHasMultipleValues(true) + .setDictionaryEncodedColumn( + new DictionaryEncodedColumnSupplier( + index.getDimValueLookup(dimension), null, (index.getDimColumn(dimension)) + ) + ) + .setBitmapIndex( + new BitmapIndexColumnPartSupplier( + index.getInvertedIndexes().get(dimension), index.getDimValueLookup(dimension) + ) + ) + .build() + ); + } + + for (String metric : index.getAvailableMetrics()) { + final MetricHolder metricHolder = index.getMetricHolder(metric); + if (metricHolder.getType() == MetricHolder.MetricType.FLOAT) { + columns.put( + metric.toLowerCase(), + new ColumnBuilder() + .setType(ValueType.FLOAT) + .setGenericColumn(new FloatGenericColumnSupplier(metricHolder.floatType, BYTE_ORDER)) + .build() + ); + } else if (metricHolder.getType() == MetricHolder.MetricType.COMPLEX) { + columns.put( + metric.toLowerCase(), + new ColumnBuilder() + .setType(ValueType.COMPLEX) + .setComplexColumn( + new ComplexColumnPartSupplier( + (GenericIndexed) metricHolder.complexType, metricHolder.getTypeName() + ) + ) + .build() + ); + } + } + + Set colSet = Sets.newTreeSet(); + colSet.addAll(Lists.newArrayList(index.getAvailableDimensions())); + colSet.addAll(Lists.newArrayList(index.getAvailableMetrics())); + + String[] cols = colSet.toArray(new String[colSet.size()]); + + return new SimpleQueryableIndex( + index.getDataInterval(), + new ArrayIndexed(cols, String.class), + index.getAvailableDimensions(), + new ColumnBuilder() + .setType(ValueType.LONG) + .setGenericColumn(new LongGenericColumnSupplier(index.timestamps)) + .build(), + columns + ); + } + } + + static class V9IndexLoader implements IndexLoader + { + @Override + public QueryableIndex load(File inDir) throws IOException + { + log.debug("Mapping v9 index[%s]", inDir); + long startTime = System.currentTimeMillis(); + + final int theVersion = Ints.fromByteArray(Files.toByteArray(new File(inDir, "version.bin"))); + if (theVersion != V9_VERSION) { + throw new IllegalArgumentException(String.format("Expected version[9], got[%s]", theVersion)); + } + + SmooshedFileMapper smooshedFiles = Smoosh.map(inDir); + + ByteBuffer indexBuffer = smooshedFiles.mapFile("index.drd"); + final GenericIndexed cols = GenericIndexed.read(indexBuffer, GenericIndexed.stringStrategy); + final GenericIndexed dims = GenericIndexed.read(indexBuffer, GenericIndexed.stringStrategy); + final Interval dataInterval = new Interval(indexBuffer.getLong(), indexBuffer.getLong()); + + Map columns = Maps.newHashMap(); + + ObjectMapper mapper = new DefaultObjectMapper(); + + for (String columnName : cols) { + columns.put(columnName, deserializeColumn(mapper, smooshedFiles.mapFile(columnName))); + } + + final QueryableIndex index = new SimpleQueryableIndex( + dataInterval, cols, dims, deserializeColumn(mapper, smooshedFiles.mapFile("__time")), columns + ); + + log.debug("Mapped v9 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime); + + return index; + } + + private Column deserializeColumn(ObjectMapper mapper, ByteBuffer byteBuffer) throws IOException + { + ColumnDescriptor serde = mapper.readValue( + serializerUtils.readString(byteBuffer), ColumnDescriptor.class + ); + return serde.read(byteBuffer); + } } public static File makeDimFile(File dir, String dimension) diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/MetricHolder.java b/index-common/src/main/java/com/metamx/druid/index/v1/MetricHolder.java index 91cbf4f6818..075b9dbfe52 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/MetricHolder.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/MetricHolder.java @@ -39,7 +39,6 @@ import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; /** @@ -140,7 +139,7 @@ public class MetricHolder break; case COMPLEX: if (strategy != null) { - holder.complexType = GenericIndexed.readFromByteBuffer(buf, strategy); + holder.complexType = GenericIndexed.read(buf, strategy); } else { final ComplexMetricSerde serdeForType = ComplexMetrics.getSerdeForType(holder.getTypeName()); @@ -148,7 +147,7 @@ public class MetricHolder throw new ISE("Unknown type[%s], cannot load.", holder.getTypeName()); } - holder.complexType = GenericIndexed.readFromByteBuffer(buf, serdeForType.getObjectStrategy()); + holder.complexType = GenericIndexed.read(buf, serdeForType.getObjectStrategy()); } break; } @@ -174,8 +173,8 @@ public class MetricHolder } } - private CompressedFloatsIndexedSupplier floatType = null; - private Indexed complexType = null; + CompressedFloatsIndexedSupplier floatType = null; + Indexed complexType = null; private MetricHolder( String name, diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricSerde.java b/index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricSerde.java index e2657dc60d1..7473aaec327 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricSerde.java @@ -19,13 +19,41 @@ package com.metamx.druid.index.v1.serde; +import com.metamx.druid.index.column.ColumnBuilder; +import com.metamx.druid.index.serde.ColumnPartSerde; import com.metamx.druid.kv.ObjectStrategy; +import java.nio.ByteBuffer; + /** */ public interface ComplexMetricSerde { public String getTypeName(); public ComplexMetricExtractor getExtractor(); + + /** + * Deserializes a ByteBuffer and adds it to the ColumnBuilder. This method allows for the ComplexMetricSerde + * to implement it's own versioning scheme to allow for changes of binary format in a forward-compatible manner. + * + * The method is also in charge of returning a ColumnPartSerde that knows how to serialize out the object it + * added to the builder. + * + * @param buffer the buffer to deserialize + * @return a ColumnPartSerde that can serialize out the object that was read from the buffer to the builder + */ + public ColumnPartSerde deserializeColumn(ByteBuffer buffer, ColumnBuilder builder); + + /** + * This is deprecated because its usage is going to be removed from the code. + * + * It was introduced before deserializeColumn() existed. This method creates the assumption that Druid knows + * how to interpret the actual column representation of the data, but I would much prefer that the ComplexMetricSerde + * objects be in charge of creating and interpreting the whole column, which is what deserializeColumn lets + * them do. + * + * @return an ObjectStrategy as used by GenericIndexed + */ + @Deprecated public ObjectStrategy getObjectStrategy(); } diff --git a/index-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java b/index-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java index d98abc1b670..5fed0c13b1f 100644 --- a/index-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java +++ b/index-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java @@ -150,8 +150,11 @@ public class GenericIndexed implements Indexed @Override public T get(int index) { + if (index < 0) { + throw new IAE("Index[%s] < 0", index); + } if (index >= size) { - throw new IllegalArgumentException(String.format("Index[%s] >= size[%s]", index, size)); + throw new IAE(String.format("Index[%s] >= size[%s]", index, size)); } ByteBuffer myBuffer = theBuffer.asReadOnlyBuffer(); @@ -204,6 +207,11 @@ public class GenericIndexed implements Indexed return -(minIndex + 1); } + public int getSerializedSize() + { + return theBuffer.remaining() + 2 + 4 + 4; + } + public void writeToChannel(WritableByteChannel channel) throws IOException { channel.write(ByteBuffer.wrap(new byte[]{version, allowReverseLookup ? (byte) 0x1 : (byte) 0x0})); @@ -212,8 +220,7 @@ public class GenericIndexed implements Indexed channel.write(theBuffer.asReadOnlyBuffer()); } - public static GenericIndexed readFromByteBuffer(ByteBuffer buffer, ObjectStrategy strategy) - throws IOException + public static GenericIndexed read(ByteBuffer buffer, ObjectStrategy strategy) { byte versionFromBuffer = buffer.get(); diff --git a/index-common/src/main/java/com/metamx/druid/kv/VSizeIndexed.java b/index-common/src/main/java/com/metamx/druid/kv/VSizeIndexed.java index b77ab3ba4db..9ef63f39ac0 100644 --- a/index-common/src/main/java/com/metamx/druid/kv/VSizeIndexed.java +++ b/index-common/src/main/java/com/metamx/druid/kv/VSizeIndexed.java @@ -153,6 +153,11 @@ public class VSizeIndexed implements Indexed throw new UnsupportedOperationException("Reverse lookup not allowed."); } + public int getSerializedSize() + { + return theBuffer.remaining() + 4 + 4 + 2; + } + public void writeToChannel(WritableByteChannel channel) throws IOException { channel.write(ByteBuffer.wrap(new byte[]{version, (byte) numBytes})); @@ -161,10 +166,7 @@ public class VSizeIndexed implements Indexed channel.write(theBuffer.asReadOnlyBuffer()); } - public static VSizeIndexed readFromByteBuffer( - ByteBuffer buffer - ) - throws IOException + public static VSizeIndexed readFromByteBuffer(ByteBuffer buffer) { byte versionFromBuffer = buffer.get(); @@ -175,10 +177,7 @@ public class VSizeIndexed implements Indexed bufferToUse.limit(bufferToUse.position() + size); buffer.position(bufferToUse.limit()); - return new VSizeIndexed( - bufferToUse, - numBytes - ); + return new VSizeIndexed(bufferToUse, numBytes); } throw new IAE("Unknown version[%s]", versionFromBuffer); diff --git a/index-common/src/main/java/com/metamx/druid/kv/VSizeIndexedInts.java b/index-common/src/main/java/com/metamx/druid/kv/VSizeIndexedInts.java index c4bd1adad2f..2b880f96f97 100644 --- a/index-common/src/main/java/com/metamx/druid/kv/VSizeIndexedInts.java +++ b/index-common/src/main/java/com/metamx/druid/kv/VSizeIndexedInts.java @@ -22,7 +22,9 @@ package com.metamx.druid.kv; import com.google.common.primitives.Ints; import com.metamx.common.IAE; +import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; import java.util.Iterator; import java.util.List; @@ -30,6 +32,8 @@ import java.util.List; */ public class VSizeIndexedInts implements IndexedInts, Comparable { + private static final byte version = 0x0; + public static VSizeIndexedInts fromArray(int[] array) { return fromArray(array, Ints.max(array)); @@ -143,9 +147,43 @@ public class VSizeIndexedInts implements IndexedInts, Comparable iterator() { return new IndexedIntsIterator(this); } + + public void writeToChannel(WritableByteChannel channel) throws IOException + { + channel.write(ByteBuffer.wrap(new byte[]{version, (byte) numBytes})); + channel.write(ByteBuffer.wrap(Ints.toByteArray(buffer.remaining()))); + channel.write(buffer.asReadOnlyBuffer()); + } + + public static VSizeIndexedInts readFromByteBuffer(ByteBuffer buffer) + { + byte versionFromBuffer = buffer.get(); + + if (version == versionFromBuffer) { + int numBytes = buffer.get(); + int size = buffer.getInt(); + ByteBuffer bufferToUse = buffer.asReadOnlyBuffer(); + bufferToUse.limit(bufferToUse.position() + size); + buffer.position(bufferToUse.limit()); + + return new VSizeIndexedInts( + bufferToUse, + numBytes + ); + } + + throw new IAE("Unknown version[%s]", versionFromBuffer); + } + } diff --git a/index-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplierTest.java b/index-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplierTest.java index 850217c1f0d..125aa7618f8 100644 --- a/index-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplierTest.java +++ b/index-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplierTest.java @@ -27,8 +27,12 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.FloatBuffer; +import java.nio.channels.Channels; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; @@ -71,6 +75,25 @@ public class CompressedFloatsIndexedSupplierTest indexed = supplier.get(); } + private void setupSimpleWithSerde() throws IOException + { + vals = new float[]{ + 0.0f, 0.1f, 0.2f, 0.3f, 0.4f, 0.5f, 0.6f, 0.7f, 0.8f, 0.9f, 0.10f, 0.11f, 0.12f, 0.13f, 0.14f, 0.15f + }; + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final CompressedFloatsIndexedSupplier theSupplier = CompressedFloatsIndexedSupplier.fromFloatBuffer( + FloatBuffer.wrap(vals), 5, ByteOrder.nativeOrder() + ); + theSupplier.writeToChannel(Channels.newChannel(baos)); + + final byte[] bytes = baos.toByteArray(); + Assert.assertEquals(theSupplier.getSerializedSize(), bytes.length); + + supplier = CompressedFloatsIndexedSupplier.fromByteBuffer(ByteBuffer.wrap(bytes), ByteOrder.nativeOrder()); + indexed = supplier.get(); + } + @Test public void testSanity() throws Exception { @@ -102,6 +125,37 @@ public class CompressedFloatsIndexedSupplierTest tryFill(7, 10); } + @Test + public void testSanityWithSerde() throws Exception + { + setupSimpleWithSerde(); + + Assert.assertEquals(4, supplier.getBaseFloatBuffers().size()); + + Assert.assertEquals(vals.length, indexed.size()); + for (int i = 0; i < indexed.size(); ++i) { + Assert.assertEquals(vals[i], indexed.get(i), 0.0); + } + } + + @Test + public void testBulkFillWithSerde() throws Exception + { + setupSimpleWithSerde(); + + tryFill(0, 15); + tryFill(3, 6); + tryFill(7, 7); + tryFill(7, 9); + } + + @Test(expected = IndexOutOfBoundsException.class) + public void testBulkFillTooMuchWithSerde() throws Exception + { + setupSimpleWithSerde(); + tryFill(7, 10); + } + // This test attempts to cause a race condition with the DirectByteBuffers, it's non-deterministic in causing it, // which sucks but I can't think of a way to deterministically cause it... @Test diff --git a/index-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplierTest.java b/index-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplierTest.java index f14fcd29f86..0f06cff2b64 100644 --- a/index-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplierTest.java +++ b/index-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplierTest.java @@ -27,8 +27,13 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.nio.FloatBuffer; import java.nio.LongBuffer; +import java.nio.channels.Channels; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; @@ -69,6 +74,23 @@ public class CompressedLongsIndexedSupplierTest indexed = supplier.get(); } + private void setupSimpleWithSerde() throws IOException + { + vals = new long[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 16}; + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final CompressedLongsIndexedSupplier theSupplier = CompressedLongsIndexedSupplier.fromLongBuffer( + LongBuffer.wrap(vals), 5, ByteOrder.nativeOrder() + ); + theSupplier.writeToChannel(Channels.newChannel(baos)); + + final byte[] bytes = baos.toByteArray(); + Assert.assertEquals(theSupplier.getSerializedSize(), bytes.length); + + supplier = CompressedLongsIndexedSupplier.fromByteBuffer(ByteBuffer.wrap(bytes), ByteOrder.nativeOrder()); + indexed = supplier.get(); + } + @Test public void testSanity() throws Exception { @@ -100,6 +122,37 @@ public class CompressedLongsIndexedSupplierTest tryFill(7, 10); } + @Test + public void testSanityWithSerde() throws Exception + { + setupSimpleWithSerde(); + + Assert.assertEquals(4, supplier.getBaseLongBuffers().size()); + + Assert.assertEquals(vals.length, indexed.size()); + for (int i = 0; i < indexed.size(); ++i) { + Assert.assertEquals(vals[i], indexed.get(i)); + } + } + + @Test + public void testBulkFillWithSerde() throws Exception + { + setupSimpleWithSerde(); + + tryFill(0, 15); + tryFill(3, 6); + tryFill(7, 7); + tryFill(7, 9); + } + + @Test(expected = IndexOutOfBoundsException.class) + public void testBulkFillTooMuchWithSerde() throws Exception + { + setupSimpleWithSerde(); + tryFill(7, 10); + } + // This test attempts to cause a race condition with the DirectByteBuffers, it's non-deterministic in causing it, // which sucks but I can't think of a way to deterministically cause it... @Test diff --git a/index-common/src/test/java/com/metamx/druid/kv/GenericIndexedTest.java b/index-common/src/test/java/com/metamx/druid/kv/GenericIndexedTest.java index 41e6a65ebde..fad903abc96 100644 --- a/index-common/src/test/java/com/metamx/druid/kv/GenericIndexedTest.java +++ b/index-common/src/test/java/com/metamx/druid/kv/GenericIndexedTest.java @@ -114,7 +114,8 @@ public class GenericIndexedTest channel.close(); final ByteBuffer byteBuffer = ByteBuffer.wrap(baos.toByteArray()); - GenericIndexed deserialized = GenericIndexed.readFromByteBuffer( + Assert.assertEquals(indexed.getSerializedSize(), byteBuffer.remaining()); + GenericIndexed deserialized = GenericIndexed.read( byteBuffer, GenericIndexed.stringStrategy ); Assert.assertEquals(0, byteBuffer.remaining()); diff --git a/index-common/src/test/java/com/metamx/druid/kv/VSizeIndexedIntsTest.java b/index-common/src/test/java/com/metamx/druid/kv/VSizeIndexedIntsTest.java index 2c080e91469..d953958116b 100644 --- a/index-common/src/test/java/com/metamx/druid/kv/VSizeIndexedIntsTest.java +++ b/index-common/src/test/java/com/metamx/druid/kv/VSizeIndexedIntsTest.java @@ -22,6 +22,10 @@ package com.metamx.druid.kv; import org.junit.Assert; import org.junit.Test; +import java.io.ByteArrayOutputStream; +import java.nio.ByteBuffer; +import java.nio.channels.Channels; + /** */ public class VSizeIndexedIntsTest @@ -38,4 +42,25 @@ public class VSizeIndexedIntsTest Assert.assertEquals(array[i], ints.get(i)); } } + + @Test + public void testSerialization() throws Exception + { + final int[] array = {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}; + VSizeIndexedInts ints = VSizeIndexedInts.fromArray(array); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ints.writeToChannel(Channels.newChannel(baos)); + + final byte[] bytes = baos.toByteArray(); + Assert.assertEquals(ints.getSerializedSize(), bytes.length); + VSizeIndexedInts deserialized = VSizeIndexedInts.readFromByteBuffer(ByteBuffer.wrap(bytes)); + + Assert.assertEquals(1, deserialized.getNumBytes()); + Assert.assertEquals(array.length, deserialized.size()); + for (int i = 0; i < array.length; i++) { + Assert.assertEquals(array[i], deserialized.get(i)); + } + } + } diff --git a/index-common/src/test/java/com/metamx/druid/kv/VSizeIndexedTest.java b/index-common/src/test/java/com/metamx/druid/kv/VSizeIndexedTest.java index 39ce52a5133..631ec03869f 100644 --- a/index-common/src/test/java/com/metamx/druid/kv/VSizeIndexedTest.java +++ b/index-common/src/test/java/com/metamx/druid/kv/VSizeIndexedTest.java @@ -61,10 +61,11 @@ public class VSizeIndexedTest assertSame(someInts, indexed); final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - WritableByteChannel byteChannel = Channels.newChannel(baos); - indexed.writeToChannel(byteChannel); + indexed.writeToChannel(Channels.newChannel(baos)); - VSizeIndexed deserializedIndexed = VSizeIndexed.readFromByteBuffer(ByteBuffer.wrap(baos.toByteArray())); + final byte[] bytes = baos.toByteArray(); + Assert.assertEquals(indexed.getSerializedSize(), bytes.length); + VSizeIndexed deserializedIndexed = VSizeIndexed.readFromByteBuffer(ByteBuffer.wrap(bytes)); assertSame(someInts, deserializedIndexed); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index e631db0eea2..74a546cf696 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -20,13 +20,13 @@ package com.metamx.druid.merger.common; import com.google.common.collect.ImmutableMap; -import com.metamx.druid.loading.S3SegmentGetter; +import com.metamx.druid.loading.S3SegmentPuller; import com.metamx.druid.loading.S3SegmentGetterConfig; -import com.metamx.druid.loading.S3ZippedSegmentGetter; -import com.metamx.druid.loading.SegmentGetter; +import com.metamx.druid.loading.S3ZippedSegmentPuller; +import com.metamx.druid.loading.SegmentPuller; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; -import com.metamx.druid.realtime.SegmentPusher; +import com.metamx.druid.loading.SegmentPusher; import com.metamx.emitter.service.ServiceEmitter; import org.codehaus.jackson.map.ObjectMapper; import org.jets3t.service.impl.rest.httpclient.RestS3Service; @@ -85,7 +85,7 @@ public class TaskToolbox return objectMapper; } - public Map getSegmentGetters(final Task task) + public Map getSegmentGetters(final Task task) { final S3SegmentGetterConfig getterConfig = new S3SegmentGetterConfig() { @@ -96,10 +96,10 @@ public class TaskToolbox } }; - return ImmutableMap.builder() - .put("s3", new S3SegmentGetter(s3Client, getterConfig)) - .put("s3_union", new S3SegmentGetter(s3Client, getterConfig)) - .put("s3_zip", new S3ZippedSegmentGetter(s3Client, getterConfig)) + return ImmutableMap.builder() + .put("s3", new S3SegmentPuller(s3Client, getterConfig)) + .put("s3_union", new S3SegmentPuller(s3Client, getterConfig)) + .put("s3_zip", new S3ZippedSegmentPuller(s3Client, getterConfig)) .build(); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java index c77dfbd9e80..a496f560970 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java @@ -36,7 +36,7 @@ import com.metamx.druid.realtime.FireHydrant; import com.metamx.druid.realtime.Plumber; import com.metamx.druid.realtime.PlumberSchool; import com.metamx.druid.realtime.Schema; -import com.metamx.druid.realtime.SegmentPusher; +import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.realtime.Sink; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; @@ -168,7 +168,7 @@ public class YeOldePlumberSchool implements PlumberSchool dirToPersist ); - indexToPersist.swapAdapter(null); + indexToPersist.swapSegment(null); metrics.incrementRowOutputCount(rowsToPersist); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index 37187904ace..b89142ef19a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -22,7 +22,6 @@ package com.metamx.druid.merger.common.task; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; -import com.metamx.common.exception.FormattedException; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.input.InputRow; @@ -35,7 +34,7 @@ import com.metamx.druid.realtime.Firehose; import com.metamx.druid.realtime.FirehoseFactory; import com.metamx.druid.realtime.Plumber; import com.metamx.druid.realtime.Schema; -import com.metamx.druid.realtime.SegmentPusher; +import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.realtime.Sink; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 859352c2a7d..7388058bad0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -34,7 +34,7 @@ import com.google.common.collect.Sets; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.loading.SegmentGetter; +import com.metamx.druid.loading.SegmentPuller; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; @@ -142,12 +142,12 @@ public abstract class MergeTask extends AbstractTask // download segments to merge - final Map segmentGetters = toolbox.getSegmentGetters(this); + final Map segmentGetters = toolbox.getSegmentGetters(this); final Map gettedSegments = Maps.newHashMap(); for (final DataSegment segment : segments) { Map loadSpec = segment.getLoadSpec(); - SegmentGetter segmentGetter = segmentGetters.get(loadSpec.get("type")); - gettedSegments.put(segment, segmentGetter.getSegmentFiles(loadSpec)); + SegmentPuller segmentPuller = segmentGetters.get(loadSpec.get("type")); + gettedSegments.put(segment, segmentPuller.getSegmentFiles(segment)); } // merge files together diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java index ed7ac9f3f25..78326d3a3cc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java @@ -106,7 +106,7 @@ public class TaskConsumer implements Runnable .addData("interval", task.getInterval()) .emit(); - // TODO - Retry would be nice, but only after we have a way to throttle and limit them + // Retry would be nice, but only after we have a way to throttle and limit them. Just fail for now. if(!shutdown) { queue.done(task, TaskStatus.failure(task.getId())); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 1c11c62cb7b..bc48c7c71de 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -69,9 +69,9 @@ import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.NoopScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; -import com.metamx.druid.realtime.S3SegmentPusher; -import com.metamx.druid.realtime.S3SegmentPusherConfig; -import com.metamx.druid.realtime.SegmentPusher; +import com.metamx.druid.loading.S3SegmentPusher; +import com.metamx.druid.loading.S3SegmentPusherConfig; +import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 7c218d4dcdc..0799a8de37c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -30,7 +30,6 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.RegisteringNode; import com.metamx.druid.http.StatusServlet; -import com.metamx.druid.index.v1.serde.Registererer; import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; @@ -43,9 +42,9 @@ import com.metamx.druid.merger.worker.TaskMonitor; import com.metamx.druid.merger.worker.Worker; import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; import com.metamx.druid.merger.worker.config.WorkerConfig; -import com.metamx.druid.realtime.S3SegmentPusher; -import com.metamx.druid.realtime.S3SegmentPusherConfig; -import com.metamx.druid.realtime.SegmentPusher; +import com.metamx.druid.loading.S3SegmentPusher; +import com.metamx.druid.loading.S3SegmentPusherConfig; +import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; diff --git a/pom.xml b/pom.xml index b94153ac887..d5cdd164498 100644 --- a/pom.xml +++ b/pom.xml @@ -68,7 +68,7 @@ com.metamx java-util - 0.16.0 + 0.16.1-SNAPSHOT com.metamx @@ -364,14 +364,6 @@ pub-libs-local https://metamx.artifactoryonline.com/metamx/pub-libs-releases-local - - repo.codahale.com - http://repo.codahale.com - - - nativelibs4java - http://nativelibs4java.sourceforge.net/maven - thirdparty-uploads JBoss Thirdparty Uploads diff --git a/realtime/pom.xml b/realtime/pom.xml index 6a0e30f2c8e..502f9c8b182 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -139,13 +139,6 @@ zkclient - - - com.codahale - jerkson_${scala.version} - 0.5.0 - - commons-codec diff --git a/realtime/src/main/java/com/metamx/druid/realtime/FireHydrant.java b/realtime/src/main/java/com/metamx/druid/realtime/FireHydrant.java index 633f7b1ac62..366d3e51a28 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/FireHydrant.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/FireHydrant.java @@ -19,16 +19,16 @@ package com.metamx.druid.realtime; -import com.metamx.druid.StorageAdapter; +import com.metamx.druid.index.IncrementalIndexSegment; +import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.IncrementalIndex; -import com.metamx.druid.index.v1.IncrementalIndexStorageAdapter; /** */ public class FireHydrant { private volatile IncrementalIndex index; - private volatile StorageAdapter adapter; + private volatile Segment adapter; private final int count; public FireHydrant( @@ -37,12 +37,12 @@ public class FireHydrant ) { this.index = index; - this.adapter = new IncrementalIndexStorageAdapter(index); + this.adapter = new IncrementalIndexSegment(index); this.count = count; } public FireHydrant( - StorageAdapter adapter, + Segment adapter, int count ) { @@ -56,7 +56,7 @@ public class FireHydrant return index; } - public StorageAdapter getAdapter() + public Segment getSegment() { return adapter; } @@ -71,7 +71,7 @@ public class FireHydrant return index == null; } - public void swapAdapter(StorageAdapter adapter) + public void swapSegment(Segment adapter) { this.adapter = adapter; this.index = null; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java index 60d290992d5..f503e80ade3 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -40,6 +40,9 @@ import com.metamx.druid.http.QueryServlet; import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.loading.S3SegmentPusher; +import com.metamx.druid.loading.S3SegmentPusherConfig; +import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java index fb4713f344b..603b9ee1a36 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java @@ -38,11 +38,16 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; import com.metamx.druid.client.ServerView; import com.metamx.druid.guava.ThreadRenamingRunnable; +import com.metamx.druid.index.QueryableIndex; +import com.metamx.druid.index.QueryableIndexSegment; +import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.IndexGranularity; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; import com.metamx.druid.index.v1.MMappedIndex; +import com.metamx.druid.index.v1.MMappedIndexQueryableIndex; import com.metamx.druid.index.v1.MMappedIndexStorageAdapter; +import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.query.MetricsEmittingQueryRunner; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerFactory; @@ -185,7 +190,7 @@ public class RealtimePlumberSchool implements PlumberSchool log.info("Loading previously persisted segment at [%s]", segmentDir); hydrants.add( new FireHydrant( - new MMappedIndexStorageAdapter(IndexIO.mapDir(segmentDir)), + new QueryableIndexSegment(null, IndexIO.loadIndex(segmentDir)), Integer.parseInt(segmentDir.getName()) ) ); @@ -305,15 +310,16 @@ public class RealtimePlumberSchool implements PlumberSchool try { List indexes = Lists.newArrayList(); for (FireHydrant fireHydrant : sink) { - StorageAdapter adapter = fireHydrant.getAdapter(); - if (adapter instanceof MMappedIndexStorageAdapter) { + Segment segment = fireHydrant.getSegment(); + final QueryableIndex queryableIndex = segment.asQueryableIndex(); + if (queryableIndex instanceof MMappedIndexQueryableIndex) { log.info("Adding hydrant[%s]", fireHydrant); - indexes.add(((MMappedIndexStorageAdapter) adapter).getIndex()); + indexes.add(((MMappedIndexQueryableIndex) queryableIndex).getIndex()); } else { log.makeAlert("[%s] Failure to merge-n-push", schema.getDataSource()) - .addData("type", "Unknown adapter type") - .addData("adapterClass", adapter.getClass().toString()) + .addData("type", "Unknown segment type") + .addData("adapterClass", segment.getClass().toString()) .emit(); return; } @@ -420,7 +426,7 @@ public class RealtimePlumberSchool implements PlumberSchool @Override public QueryRunner apply(@Nullable FireHydrant input) { - return factory.createRunner(input.getAdapter()); + return factory.createRunner(input.getSegment()); } } ) @@ -442,6 +448,8 @@ public class RealtimePlumberSchool implements PlumberSchool } } + log.info("Submitting persist runnable for dataSource[%s]", schema.getDataSource()); + persistExecutor.execute( new ThreadRenamingRunnable(String.format("%s-incremental-persist", schema.getDataSource())) { @@ -495,7 +503,9 @@ public class RealtimePlumberSchool implements PlumberSchool new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())) ); - indexToPersist.swapAdapter(new MMappedIndexStorageAdapter(IndexIO.mapDir(persistedFile))); + indexToPersist.swapSegment( + new QueryableIndexSegment(null, new MMappedIndexQueryableIndex(IndexIO.mapDir(persistedFile))) + ); return numRows; } diff --git a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java index 6ae83c5f9bf..1b22f1e3a78 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java @@ -19,133 +19,23 @@ package com.metamx.druid.realtime; -import com.google.common.base.Joiner; -import com.google.common.collect.ImmutableMap; -import com.google.common.io.Closeables; -import com.metamx.common.ISE; -import com.metamx.common.StreamUtils; -import com.metamx.druid.client.DataSegment; -import com.metamx.emitter.EmittingLogger; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.IOUtils; +import com.metamx.druid.loading.S3SegmentPusherConfig; +import com.metamx.druid.loading.SegmentPusher; import org.codehaus.jackson.map.ObjectMapper; -import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Object; - -import java.io.ByteArrayInputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; -import java.security.NoSuchAlgorithmException; -import java.util.zip.ZipEntry; -import java.util.zip.ZipOutputStream; /** + * A placeholder class to make the move of the SegmentPushers to a new package backwards compatible */ -public class S3SegmentPusher implements SegmentPusher +@Deprecated +public class S3SegmentPusher extends com.metamx.druid.loading.S3SegmentPusher implements SegmentPusher { - private static final EmittingLogger log = new EmittingLogger(S3SegmentPusher.class); - private static final Joiner JOINER = Joiner.on("/").skipNulls(); - - private final RestS3Service s3Client; - private final S3SegmentPusherConfig config; - private final ObjectMapper jsonMapper; - public S3SegmentPusher( - RestS3Service s3Client, - S3SegmentPusherConfig config, - ObjectMapper jsonMapper + RestS3Service s3Client, + S3SegmentPusherConfig config, + ObjectMapper jsonMapper ) { - this.s3Client = s3Client; - this.config = config; - this.jsonMapper = jsonMapper; - } - - @Override - public DataSegment push(File file, DataSegment segment) throws IOException - { - log.info("Uploading [%s] to S3", file); - String outputKey = JOINER.join( - config.getBaseKey().isEmpty() ? null : config.getBaseKey(), - segment.getDataSource(), - String.format( - "%s_%s", - segment.getInterval().getStart(), - segment.getInterval().getEnd() - ), - segment.getVersion(), - segment.getShardSpec().getPartitionNum() - ); - - File indexFilesDir = file; - - long indexSize = 0; - final File zipOutFile = File.createTempFile("druid", "index.zip"); - ZipOutputStream zipOut = null; - try { - zipOut = new ZipOutputStream(new FileOutputStream(zipOutFile)); - File[] indexFiles = indexFilesDir.listFiles(); - for (File indexFile : indexFiles) { - log.info("Adding indexFile[%s] with size[%,d]. Total size[%,d]", indexFile, indexFile.length(), indexSize); - if (indexFile.length() >= Integer.MAX_VALUE) { - throw new ISE("indexFile[%s] too large [%,d]", indexFile, indexFile.length()); - } - zipOut.putNextEntry(new ZipEntry(indexFile.getName())); - IOUtils.copy(new FileInputStream(indexFile), zipOut); - indexSize += indexFile.length(); - } - } - finally { - Closeables.closeQuietly(zipOut); - } - - try { - S3Object toPush = new S3Object(zipOutFile); - - final String outputBucket = config.getBucket(); - toPush.setBucketName(outputBucket); - toPush.setKey(outputKey + "/index.zip"); - - log.info("Pushing %s.", toPush); - s3Client.putObject(outputBucket, toPush); - - DataSegment outputSegment = segment.withSize(indexSize) - .withLoadSpec( - ImmutableMap.of( - "type", "s3_zip", - "bucket", outputBucket, - "key", toPush.getKey() - ) - ); - - File descriptorFile = File.createTempFile("druid", "descriptor.json"); - StreamUtils.copyToFileAndClose(new ByteArrayInputStream(jsonMapper.writeValueAsBytes(segment)), descriptorFile); - S3Object descriptorObject = new S3Object(descriptorFile); - descriptorObject.setBucketName(outputBucket); - descriptorObject.setKey(outputKey + "/descriptor.json"); - - log.info("Pushing %s", descriptorObject); - s3Client.putObject(outputBucket, descriptorObject); - - log.info("Deleting Index File[%s]", indexFilesDir); - FileUtils.deleteDirectory(indexFilesDir); - - log.info("Deleting zipped index File[%s]", zipOutFile); - zipOutFile.delete(); - - log.info("Deleting descriptor file[%s]", descriptorFile); - descriptorFile.delete(); - - return outputSegment; - } - catch (NoSuchAlgorithmException e) { - throw new IOException(e); - } - catch (S3ServiceException e) { - throw new IOException(e); - } + super(s3Client, config, jsonMapper); } } diff --git a/server/pom.xml b/server/pom.xml index 88bfc7e2982..7db3faba232 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -209,13 +209,6 @@ easymock test - - com.metamx.druid - druid-client - ${project.parent.version} - test-jar - test - com.metamx.druid druid-index-common diff --git a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java index 9d0e8bbdfa2..ce4e5f4accb 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java +++ b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java @@ -24,13 +24,13 @@ import com.google.common.collect.Ordering; import com.metamx.common.ISE; import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.Query; -import com.metamx.druid.StorageAdapter; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.collect.CountingMap; +import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.SegmentIdAttachedStorageAdapter; -import com.metamx.druid.loading.StorageAdapterLoader; +import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.loading.StorageAdapterLoadingException; import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.partition.PartitionHolder; @@ -66,29 +66,29 @@ public class ServerManager implements QuerySegmentWalker private final Object lock = new Object(); - private final StorageAdapterLoader storageAdapterLoader; + private final SegmentLoader segmentLoader; private final QueryRunnerFactoryConglomerate conglomerate; private final ServiceEmitter emitter; private final ExecutorService exec; - private final Map> dataSources; + private final Map> dataSources; private final CountingMap dataSourceSizes = new CountingMap(); private final CountingMap dataSourceCounts = new CountingMap(); public ServerManager( - StorageAdapterLoader storageAdapterLoader, + SegmentLoader segmentLoader, QueryRunnerFactoryConglomerate conglomerate, ServiceEmitter emitter, ExecutorService exec ) { - this.storageAdapterLoader = storageAdapterLoader; + this.segmentLoader = segmentLoader; this.conglomerate = conglomerate; this.emitter = emitter; this.exec = exec; - this.dataSources = new HashMap>(); + this.dataSources = new HashMap>(); } public Map getDataSourceSizes() @@ -107,13 +107,13 @@ public class ServerManager implements QuerySegmentWalker public void loadSegment(final DataSegment segment) throws StorageAdapterLoadingException { - StorageAdapter adapter = null; + final Segment adapter; try { - adapter = storageAdapterLoader.getAdapter(segment.getLoadSpec()); + adapter = segmentLoader.getSegment(segment); } catch (StorageAdapterLoadingException e) { try { - storageAdapterLoader.cleanupAdapter(segment.getLoadSpec()); + segmentLoader.cleanup(segment); } catch (StorageAdapterLoadingException e1) { // ignore @@ -125,18 +125,16 @@ public class ServerManager implements QuerySegmentWalker throw new StorageAdapterLoadingException("Null adapter from loadSpec[%s]", segment.getLoadSpec()); } - adapter = new SegmentIdAttachedStorageAdapter(segment.getIdentifier(), adapter); - synchronized (lock) { String dataSource = segment.getDataSource(); - VersionedIntervalTimeline loadedIntervals = dataSources.get(dataSource); + VersionedIntervalTimeline loadedIntervals = dataSources.get(dataSource); if (loadedIntervals == null) { - loadedIntervals = new VersionedIntervalTimeline(Ordering.natural()); + loadedIntervals = new VersionedIntervalTimeline(Ordering.natural()); dataSources.put(dataSource, loadedIntervals); } - PartitionHolder entry = loadedIntervals.findEntry( + PartitionHolder entry = loadedIntervals.findEntry( segment.getInterval(), segment.getVersion() ); @@ -161,17 +159,17 @@ public class ServerManager implements QuerySegmentWalker { String dataSource = segment.getDataSource(); synchronized (lock) { - VersionedIntervalTimeline loadedIntervals = dataSources.get(dataSource); + VersionedIntervalTimeline loadedIntervals = dataSources.get(dataSource); if (loadedIntervals == null) { log.info("Told to delete a queryable for a dataSource[%s] that doesn't exist.", dataSource); return; } - PartitionChunk removed = loadedIntervals.remove( - segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk((StorageAdapter) null) + PartitionChunk removed = loadedIntervals.remove( + segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk((Segment) null) ); - StorageAdapter oldQueryable = (removed == null) ? null : removed.getObject(); + Segment oldQueryable = (removed == null) ? null : removed.getObject(); if (oldQueryable != null) { synchronized (dataSourceSizes) { @@ -189,7 +187,7 @@ public class ServerManager implements QuerySegmentWalker ); } } - storageAdapterLoader.cleanupAdapter(segment.getLoadSpec()); + segmentLoader.cleanup(segment); } @Override @@ -202,7 +200,7 @@ public class ServerManager implements QuerySegmentWalker final QueryToolChest> toolChest = factory.getToolchest(); - final VersionedIntervalTimeline timeline = dataSources.get(query.getDataSource()); + final VersionedIntervalTimeline timeline = dataSources.get(query.getDataSource()); if (timeline == null) { return new NoopQueryRunner(); @@ -211,20 +209,20 @@ public class ServerManager implements QuerySegmentWalker FunctionalIterable> adapters = FunctionalIterable .create(intervals) .transformCat( - new Function>>() + new Function>>() { @Override - public Iterable> apply(Interval input) + public Iterable> apply(Interval input) { return timeline.lookup(input); } } ) .transformCat( - new Function, Iterable>>() + new Function, Iterable>>() { @Override - public Iterable> apply(@Nullable final TimelineObjectHolder holder) + public Iterable> apply(@Nullable final TimelineObjectHolder holder) { if (holder == null) { return null; @@ -233,10 +231,10 @@ public class ServerManager implements QuerySegmentWalker return FunctionalIterable .create(holder.getObject()) .transform( - new Function, QueryRunner>() + new Function, QueryRunner>() { @Override - public QueryRunner apply(PartitionChunk input) + public QueryRunner apply(PartitionChunk input) { return buildAndDecorateQueryRunner( factory, @@ -274,7 +272,7 @@ public class ServerManager implements QuerySegmentWalker final QueryToolChest> toolChest = factory.getToolchest(); - final VersionedIntervalTimeline timeline = dataSources.get(query.getDataSource()); + final VersionedIntervalTimeline timeline = dataSources.get(query.getDataSource()); if (timeline == null) { return new NoopQueryRunner(); @@ -289,7 +287,7 @@ public class ServerManager implements QuerySegmentWalker @SuppressWarnings("unchecked") public Iterable> apply(@Nullable SegmentDescriptor input) { - final PartitionHolder entry = timeline.findEntry( + final PartitionHolder entry = timeline.findEntry( input.getInterval(), input.getVersion() ); @@ -297,13 +295,13 @@ public class ServerManager implements QuerySegmentWalker return null; } - final PartitionChunk chunk = entry.getChunk(input.getPartitionNumber()); + final PartitionChunk chunk = entry.getChunk(input.getPartitionNumber()); if (chunk == null) { return null; } - final StorageAdapter adapter = chunk.getObject(); - return Arrays.>asList( + final Segment adapter = chunk.getObject(); + return Arrays.asList( buildAndDecorateQueryRunner(factory, toolChest, adapter, new SpecificSegmentSpec(input)) ); } @@ -316,7 +314,7 @@ public class ServerManager implements QuerySegmentWalker private QueryRunner buildAndDecorateQueryRunner( QueryRunnerFactory> factory, final QueryToolChest> toolChest, - StorageAdapter adapter, + Segment adapter, QuerySegmentSpec segmentSpec ) { @@ -333,7 +331,7 @@ public class ServerManager implements QuerySegmentWalker }, new BySegmentQueryRunner( adapter.getSegmentIdentifier(), - adapter.getInterval().getStart(), + adapter.getDataInterval().getStart(), factory.createRunner(adapter) ) ).withWaitMeasuredFromNow(), diff --git a/server/src/main/java/com/metamx/druid/http/ComputeNode.java b/server/src/main/java/com/metamx/druid/http/ComputeNode.java index 39f43bfcde2..c559a82d3bc 100644 --- a/server/src/main/java/com/metamx/druid/http/ComputeNode.java +++ b/server/src/main/java/com/metamx/druid/http/ComputeNode.java @@ -38,7 +38,7 @@ import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.QueryableLoaderConfig; -import com.metamx.druid.loading.StorageAdapterLoader; +import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.metrics.ServerMonitor; import com.metamx.druid.query.MetricsEmittingExecutorService; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; @@ -71,7 +71,7 @@ public class ComputeNode extends BaseServerNode } private DruidServer druidServer; - private StorageAdapterLoader adapterLoader; + private SegmentLoader segmentLoader; public ComputeNode( Properties props, @@ -84,10 +84,10 @@ public class ComputeNode extends BaseServerNode super(log, props, lifecycle, jsonMapper, smileMapper, configFactory); } - public ComputeNode setAdapterLoader(StorageAdapterLoader storageAdapterLoader) + public ComputeNode setSegmentLoader(SegmentLoader segmentLoader) { - Preconditions.checkState(this.adapterLoader == null, "Cannot set adapterLoader once it has already been set."); - this.adapterLoader = storageAdapterLoader; + Preconditions.checkState(this.segmentLoader == null, "Cannot set segmentLoader once it has already been set."); + this.segmentLoader = segmentLoader; return this; } @@ -104,10 +104,10 @@ public class ComputeNode extends BaseServerNode return druidServer; } - public StorageAdapterLoader getAdapterLoader() + public SegmentLoader getSegmentLoader() { initializeAdapterLoader(); - return adapterLoader; + return segmentLoader; } protected void doInit() throws Exception @@ -129,7 +129,7 @@ public class ComputeNode extends BaseServerNode ), emitter, new ServiceMetricEvent.Builder() ); - final ServerManager serverManager = new ServerManager(adapterLoader, conglomerate, emitter, executorService); + final ServerManager serverManager = new ServerManager(segmentLoader, conglomerate, emitter, executorService); final ZkCoordinator coordinator = new ZkCoordinator( getJsonMapper(), @@ -157,7 +157,7 @@ public class ComputeNode extends BaseServerNode private void initializeAdapterLoader() { - if (adapterLoader == null) { + if (segmentLoader == null) { final Properties props = getProps(); try { final RestS3Service s3Client = new RestS3Service( @@ -167,7 +167,7 @@ public class ComputeNode extends BaseServerNode ) ); - setAdapterLoader( + setSegmentLoader( ServerInit.makeDefaultQueryableLoader(s3Client, getConfigFactory().build(QueryableLoaderConfig.class)) ); } diff --git a/server/src/main/java/com/metamx/druid/loading/SingleStorageAdapterLoader.java b/server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java similarity index 54% rename from server/src/main/java/com/metamx/druid/loading/SingleStorageAdapterLoader.java rename to server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java index 8c58ac1c2bd..19b71c0db6b 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleStorageAdapterLoader.java +++ b/server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java @@ -17,39 +17,47 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.loading; +package com.metamx.druid.index; -import com.google.inject.Inject; import com.metamx.druid.StorageAdapter; - -import java.util.Map; +import com.metamx.druid.index.v1.IncrementalIndex; +import com.metamx.druid.index.v1.IncrementalIndexStorageAdapter; +import org.joda.time.Interval; /** */ -public class SingleStorageAdapterLoader implements StorageAdapterLoader +public class IncrementalIndexSegment implements Segment { - private final SegmentGetter segmentGetter; - private final StorageAdapterFactory factory; + private final IncrementalIndex index; - @Inject - public SingleStorageAdapterLoader( - SegmentGetter segmentGetter, - StorageAdapterFactory factory + public IncrementalIndexSegment( + IncrementalIndex index ) { - this.segmentGetter = segmentGetter; - this.factory = factory; + this.index = index; } @Override - public StorageAdapter getAdapter(Map loadSpec) throws StorageAdapterLoadingException + public String getSegmentIdentifier() { - return factory.factorize(segmentGetter.getSegmentFiles(loadSpec)); + throw new UnsupportedOperationException(); } @Override - public void cleanupAdapter(Map loadSpec) throws StorageAdapterLoadingException + public Interval getDataInterval() { - segmentGetter.cleanSegmentFiles(loadSpec); + return index.getInterval(); + } + + @Override + public QueryableIndex asQueryableIndex() + { + return null; + } + + @Override + public StorageAdapter asStorageAdapter() + { + return new IncrementalIndexStorageAdapter(index); } } diff --git a/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java b/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java new file mode 100644 index 00000000000..8df88fcdad1 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java @@ -0,0 +1,62 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index; + +import com.metamx.druid.StorageAdapter; +import com.metamx.druid.index.v1.QueryableIndexStorageAdapter; +import org.joda.time.Interval; + +/** +*/ +public class QueryableIndexSegment implements Segment +{ + private final QueryableIndex index; + private final String identifier; + + public QueryableIndexSegment(final String segmentIdentifier, QueryableIndex index) + { + this.index = index; + identifier = segmentIdentifier; + } + + @Override + public String getSegmentIdentifier() + { + return identifier; + } + + @Override + public Interval getDataInterval() + { + return index.getDataInterval(); + } + + @Override + public QueryableIndex asQueryableIndex() + { + return index; + } + + @Override + public StorageAdapter asStorageAdapter() + { + return new QueryableIndexStorageAdapter(index); + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/StorageAdapterLoader.java b/server/src/main/java/com/metamx/druid/index/Segment.java similarity index 74% rename from server/src/main/java/com/metamx/druid/loading/StorageAdapterLoader.java rename to server/src/main/java/com/metamx/druid/index/Segment.java index 96afd41e2ce..7bf0acce96c 100644 --- a/server/src/main/java/com/metamx/druid/loading/StorageAdapterLoader.java +++ b/server/src/main/java/com/metamx/druid/index/Segment.java @@ -17,16 +17,17 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.loading; +package com.metamx.druid.index; import com.metamx.druid.StorageAdapter; - -import java.util.Map; +import org.joda.time.Interval; /** */ -public interface StorageAdapterLoader +public interface Segment { - public StorageAdapter getAdapter(Map loadSpec) throws StorageAdapterLoadingException; - public void cleanupAdapter(Map loadSpec) throws StorageAdapterLoadingException; + public String getSegmentIdentifier(); + public Interval getDataInterval(); + public QueryableIndex asQueryableIndex(); + public StorageAdapter asStorageAdapter(); } diff --git a/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java b/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java index 23c6dbb6842..10613561daa 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java +++ b/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java @@ -384,6 +384,7 @@ public class IndexMerger final Function>, Iterable> rowMergerFn ) throws IOException { + // TODO: make v9 index, complain to Eric when you see this, cause he should be doing it. Map metricTypes = Maps.newTreeMap(Ordering.natural().nullsFirst()); for (IndexableAdapter adapter : indexes) { for (String metric : adapter.getAvailableMetrics()) { @@ -704,7 +705,7 @@ public class IndexMerger if (!dimension.equals(serializerUtils.readString(dimValsMapped))) { throw new ISE("dimensions[%s] didn't equate!? This is a major WTF moment.", dimension); } - Indexed dimVals = GenericIndexed.readFromByteBuffer(dimValsMapped, GenericIndexed.stringStrategy); + Indexed dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.stringStrategy); log.info("Starting dimension[%s] with cardinality[%,d]", dimension, dimVals.size()); FlattenedArrayWriter writer = new FlattenedArrayWriter( diff --git a/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexQueryableIndex.java b/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexQueryableIndex.java new file mode 100644 index 00000000000..8aed193edd7 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexQueryableIndex.java @@ -0,0 +1,97 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.v1; + +import com.metamx.druid.index.QueryableIndex; +import com.metamx.druid.index.column.Column; +import com.metamx.druid.index.column.ComplexColumnImpl; +import com.metamx.druid.index.column.FloatColumn; +import com.metamx.druid.index.column.LongColumn; +import com.metamx.druid.index.column.StringMultiValueColumn; +import com.metamx.druid.kv.Indexed; +import com.metamx.druid.kv.VSizeIndexed; +import org.joda.time.Interval; + +/** + */ +public class MMappedIndexQueryableIndex implements QueryableIndex +{ + private final MMappedIndex index; + + public MMappedIndexQueryableIndex( + MMappedIndex index + ) + { + this.index = index; + } + + public MMappedIndex getIndex() + { + return index; + } + + @Override + public Interval getDataInterval() + { + return index.getDataInterval(); + } + + @Override + public Indexed getColumnNames() + { + return null; + } + + @Override + public Indexed getAvailableDimensions() + { + return index.getAvailableDimensions(); + } + + @Override + public Column getTimeColumn() + { + return new LongColumn(index.timestamps); + } + + @Override + public Column getColumn(String columnName) + { + final MetricHolder metricHolder = index.getMetricHolder(columnName); + if (metricHolder == null) { + final VSizeIndexed dimColumn = index.getDimColumn(columnName); + if (dimColumn == null) { + return null; + } + + return new StringMultiValueColumn( + index.getDimValueLookup(columnName), + dimColumn, + index.getInvertedIndexes().get(columnName) + ); + } + else if (metricHolder.getType() == MetricHolder.MetricType.FLOAT) { + return new FloatColumn(metricHolder.floatType); + } + else { + return new ComplexColumnImpl(metricHolder.getComplexType()); + } + } +} diff --git a/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexStorageAdapter.java b/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexStorageAdapter.java index 1e64950d308..6cffaf14141 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexStorageAdapter.java +++ b/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexStorageAdapter.java @@ -248,7 +248,7 @@ public class MMappedIndexStorageAdapter extends BaseStorageAdapter @Override public DimensionSelector makeDimensionSelector(String dimension) { - final String dimensionName = dimension; + final String dimensionName = dimension.toLowerCase(); final Indexed rowVals = index.getDimColumn(dimensionName); final Indexed dimValueLookup = index.getDimValueLookup(dimensionName); diff --git a/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java b/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java new file mode 100644 index 00000000000..7e7f8a33f54 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java @@ -0,0 +1,864 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.v1; + +import com.google.common.base.Function; +import com.google.common.base.Functions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; +import com.google.common.io.Closeables; +import com.metamx.common.collect.MoreIterators; +import com.metamx.common.guava.FunctionalIterable; +import com.metamx.common.guava.FunctionalIterator; +import com.metamx.druid.BaseStorageAdapter; +import com.metamx.druid.Capabilities; +import com.metamx.druid.QueryGranularity; +import com.metamx.druid.index.QueryableIndex; +import com.metamx.druid.index.brita.BitmapIndexSelector; +import com.metamx.druid.index.brita.Filter; +import com.metamx.druid.index.column.Column; +import com.metamx.druid.index.column.ColumnSelector; +import com.metamx.druid.index.column.ComplexColumn; +import com.metamx.druid.index.column.DictionaryEncodedColumn; +import com.metamx.druid.index.column.GenericColumn; +import com.metamx.druid.index.column.ValueType; +import com.metamx.druid.index.v1.processing.Cursor; +import com.metamx.druid.index.v1.processing.DimensionSelector; +import com.metamx.druid.index.v1.processing.Offset; +import com.metamx.druid.kv.Indexed; +import com.metamx.druid.kv.IndexedInts; +import com.metamx.druid.kv.IndexedIterable; +import com.metamx.druid.processing.ComplexMetricSelector; +import com.metamx.druid.processing.FloatMetricSelector; +import com.sun.xml.internal.xsom.impl.scd.Iterators; +import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import java.util.Iterator; +import java.util.Map; + +/** + */ +public class QueryableIndexStorageAdapter extends BaseStorageAdapter +{ + private final QueryableIndex index; + + public QueryableIndexStorageAdapter( + QueryableIndex index + ) + { + this.index = index; + } + + @Override + public String getSegmentIdentifier() + { + throw new UnsupportedOperationException(); + } + + @Override + public Interval getInterval() + { + return index.getDataInterval(); + } + + @Override + public int getDimensionCardinality(String dimension) + { + Column column = null; + column = index.getColumn(dimension); + if (column == null) { + return 0; + } + if (!column.getCapabilities().isDictionaryEncoded()) { + throw new UnsupportedOperationException("Only know cardinality of dictionary encoded columns."); + } + return column.getDictionaryEncoding().getCardinality(); + } + + @Override + public DateTime getMinTime() + { + GenericColumn column = null; + try { + column = index.getTimeColumn().getGenericColumn(); + return new DateTime(column.getLongSingleValueRow(0)); + } + finally { + Closeables.closeQuietly(column); + } + } + + @Override + public DateTime getMaxTime() + { + GenericColumn column = null; + try { + column = index.getTimeColumn().getGenericColumn(); + return new DateTime(column.getLongSingleValueRow(column.size() - 1)); + } + finally { + Closeables.closeQuietly(column); + } + } + + @Override + public Capabilities getCapabilities() + { + return Capabilities.builder().dimensionValuesSorted(true).build(); + } + + @Override + public Iterable makeCursors(Filter filter, Interval interval, QueryGranularity gran) + { + Interval actualInterval = interval; + final Interval dataInterval = getInterval(); + if (!actualInterval.overlaps(dataInterval)) { + return ImmutableList.of(); + } + + if (actualInterval.getStart().isBefore(dataInterval.getStart())) { + actualInterval = actualInterval.withStart(dataInterval.getStart()); + } + if (actualInterval.getEnd().isAfter(dataInterval.getEnd())) { + actualInterval = actualInterval.withEnd(dataInterval.getEnd()); + } + + final Iterable iterable; + if (filter == null) { + iterable = new NoFilterCursorIterable(index, actualInterval, gran); + } else { + Offset offset = new ConciseOffset(filter.goConcise(new MMappedBitmapIndexSelector(index))); + + iterable = new CursorIterable(index, actualInterval, gran, offset); + } + + return FunctionalIterable.create(iterable).keep(Functions.identity()); + } + + @Override + public Indexed getAvailableDimensions() + { + return index.getAvailableDimensions(); + } + + @Override + public Indexed getDimValueLookup(String dimension) + { + final Column column = index.getColumn(dimension.toLowerCase()); + + if (column == null || !column.getCapabilities().isDictionaryEncoded()) { + return null; + } + + final DictionaryEncodedColumn dictionary = column.getDictionaryEncoding(); + return new Indexed() + { + @Override + public Class getClazz() + { + return String.class; + } + + @Override + public int size() + { + return dictionary.getCardinality(); + } + + @Override + public String get(int index) + { + return dictionary.lookupName(index); + } + + @Override + public int indexOf(String value) + { + return dictionary.lookupId(value); + } + + @Override + public Iterator iterator() + { + return IndexedIterable.create(this).iterator(); + } + }; + } + + @Override + public ImmutableConciseSet getInvertedIndex(String dimension, String dimVal) + { + final Column column = index.getColumn(dimension.toLowerCase()); + if (column == null) { + return new ImmutableConciseSet(); + } + if (!column.getCapabilities().hasBitmapIndexes()) { + return new ImmutableConciseSet(); + } + + return column.getBitmapIndex().getConciseSet(dimVal); + } + + @Override + public Offset getFilterOffset(Filter filter) + { + return new ConciseOffset(filter.goConcise(new MMappedBitmapIndexSelector(index))); + } + + private static class CursorIterable implements Iterable + { + private final ColumnSelector index; + private final Interval interval; + private final QueryGranularity gran; + private final Offset offset; + + public CursorIterable( + ColumnSelector index, + Interval interval, + QueryGranularity gran, + Offset offset + ) + { + this.index = index; + this.interval = interval; + this.gran = gran; + this.offset = offset; + } + + @Override + public Iterator iterator() + { + final Offset baseOffset = offset.clone(); + + final Map genericColumnCache = Maps.newHashMap(); + final Map complexColumnCache = Maps.newHashMap(); + final GenericColumn timestamps = index.getTimeColumn().getGenericColumn(); + + final FunctionalIterator retVal = FunctionalIterator + .create(gran.iterable(interval.getStartMillis(), interval.getEndMillis()).iterator()) + .transform( + new Function() + { + + @Override + public Cursor apply(final Long input) + { + final long timeStart = Math.max(interval.getStartMillis(), input); + while (baseOffset.withinBounds() + && timestamps.getLongSingleValueRow(baseOffset.getOffset()) < timeStart) { + baseOffset.increment(); + } + + final Offset offset = new TimestampCheckingOffset( + baseOffset, timestamps, Math.min(interval.getEndMillis(), gran.next(timeStart)) + ); + + return new Cursor() + { + private final Offset initOffset = offset.clone(); + private final DateTime myBucket = gran.toDateTime(input); + private Offset cursorOffset = offset; + + @Override + public DateTime getTime() + { + return myBucket; + } + + @Override + public void advance() + { + cursorOffset.increment(); + } + + @Override + public boolean isDone() + { + return !cursorOffset.withinBounds(); + } + + @Override + public void reset() + { + cursorOffset = initOffset.clone(); + } + + @Override + public DimensionSelector makeDimensionSelector(String dimension) + { + final String dimensionName = dimension.toLowerCase(); + final Column columnDesc = index.getColumn(dimensionName); + if (columnDesc == null) { + return null; + } + + final DictionaryEncodedColumn column = columnDesc.getDictionaryEncoding(); + + if (columnDesc.getCapabilities().hasMultipleValues()) { + return new DimensionSelector() + { + @Override + public IndexedInts getRow() + { + return column.getMultiValueRow(cursorOffset.getOffset()); + } + + @Override + public int getValueCardinality() + { + return column.getCardinality(); + } + + @Override + public String lookupName(int id) + { + final String retVal = column.lookupName(id); + return retVal == null ? "" : retVal; + } + + @Override + public int lookupId(String name) + { + return column.lookupId(name); + } + }; + } + else { + return new DimensionSelector() + { + @Override + public IndexedInts getRow() + { + final int value = column.getSingleValueRow(cursorOffset.getOffset()); + return new IndexedInts() + { + @Override + public int size() + { + return 1; + } + + @Override + public int get(int index) + { + return value; + } + + @Override + public Iterator iterator() + { + return Iterators.singleton(value); + } + }; + } + + @Override + public int getValueCardinality() + { + return column.getCardinality(); + } + + @Override + public String lookupName(int id) + { + return column.lookupName(id); + } + + @Override + public int lookupId(String name) + { + return column.lookupId(name); + } + }; + } + } + + @Override + public FloatMetricSelector makeFloatMetricSelector(String metric) + { + final String metricName = metric.toLowerCase(); + GenericColumn cachedMetricVals = genericColumnCache.get(metricName); + + if (cachedMetricVals == null) { + Column holder = index.getColumn(metricName); + if (holder != null && holder.getCapabilities().getType() == ValueType.FLOAT) { + cachedMetricVals = holder.getGenericColumn(); + genericColumnCache.put(metricName, cachedMetricVals); + } + } + + if (cachedMetricVals == null) { + return new FloatMetricSelector() + { + @Override + public float get() + { + return 0.0f; + } + }; + } + + final GenericColumn metricVals = cachedMetricVals; + return new FloatMetricSelector() + { + @Override + public float get() + { + return metricVals.getFloatSingleValueRow(cursorOffset.getOffset()); + } + }; + } + + @Override + public ComplexMetricSelector makeComplexMetricSelector(String metric) + { + final String metricName = metric.toLowerCase(); + ComplexColumn cachedMetricVals = complexColumnCache.get(metricName); + + if (cachedMetricVals == null) { + Column holder = index.getColumn(metricName); + if (holder != null && holder.getCapabilities().getType() == ValueType.COMPLEX) { + cachedMetricVals = holder.getComplexColumn(); + complexColumnCache.put(metricName, cachedMetricVals); + } + } + + if (cachedMetricVals == null) { + return null; + } + + final ComplexColumn metricVals = cachedMetricVals; + return new ComplexMetricSelector() + { + @Override + public Class classOfObject() + { + return metricVals.getClazz(); + } + + @Override + public Object get() + { + return metricVals.getRowValue(cursorOffset.getOffset()); + } + }; + } + }; + } + } + ); + + // This after call is not perfect, if there is an exception during processing, it will never get called, + // but it's better than nothing and doing this properly all the time requires a lot more fixerating + return MoreIterators.after( + retVal, + new Runnable() + { + @Override + public void run() + { + Closeables.closeQuietly(timestamps); + for (GenericColumn column : genericColumnCache.values()) { + Closeables.closeQuietly(column); + } + for (ComplexColumn complexColumn : complexColumnCache.values()) { + Closeables.closeQuietly(complexColumn); + } + } + } + ); + } + } + + private static class TimestampCheckingOffset implements Offset + { + private final Offset baseOffset; + private final GenericColumn timestamps; + private final long threshold; + + public TimestampCheckingOffset( + Offset baseOffset, + GenericColumn timestamps, + long threshold + ) + { + this.baseOffset = baseOffset; + this.timestamps = timestamps; + this.threshold = threshold; + } + + @Override + public int getOffset() + { + return baseOffset.getOffset(); + } + + @Override + public Offset clone() + { + return new TimestampCheckingOffset(baseOffset.clone(), timestamps, threshold); + } + + @Override + public boolean withinBounds() + { + return baseOffset.withinBounds() && timestamps.getLongSingleValueRow(baseOffset.getOffset()) < threshold; + } + + @Override + public void increment() + { + baseOffset.increment(); + } + } + + private static class NoFilterCursorIterable implements Iterable + { + private final ColumnSelector index; + private final Interval interval; + private final QueryGranularity gran; + + public NoFilterCursorIterable( + ColumnSelector index, + Interval interval, + QueryGranularity gran + ) + { + this.index = index; + this.interval = interval; + this.gran = gran; + } + + /** + * This produces iterators of Cursor objects that must be fully processed (until isDone() returns true) before the + * next Cursor is processed. It is *not* safe to pass these cursors off to another thread for parallel processing + * + * @return + */ + @Override + public Iterator iterator() + { + final Map genericColumnCache = Maps.newHashMap(); + final Map complexColumnCache = Maps.newHashMap(); + final GenericColumn timestamps = index.getTimeColumn().getGenericColumn(); + + final FunctionalIterator retVal = FunctionalIterator + .create(gran.iterable(interval.getStartMillis(), interval.getEndMillis()).iterator()) + .transform( + new Function() + { + private int currRow = 0; + + @Override + public Cursor apply(final Long input) + { + final long timeStart = Math.max(interval.getStartMillis(), input); + while (currRow < timestamps.size() && timestamps.getLongSingleValueRow(currRow) < timeStart) { + ++currRow; + } + + return new Cursor() + { + private final DateTime myBucket = gran.toDateTime(input); + private final long nextBucket = Math.min(gran.next(myBucket.getMillis()), interval.getEndMillis()); + private final int initRow = currRow; + + @Override + public DateTime getTime() + { + return myBucket; + } + + @Override + public void advance() + { + ++currRow; + } + + @Override + public boolean isDone() + { + return currRow >= timestamps.size() || timestamps.getLongSingleValueRow(currRow) >= nextBucket; + } + + @Override + public void reset() + { + currRow = initRow; + } + + @Override + public DimensionSelector makeDimensionSelector(String dimension) + { + final String dimensionName = dimension.toLowerCase(); + final Column columnDesc = index.getColumn(dimensionName); + if (columnDesc == null) { + return null; + } + + final DictionaryEncodedColumn column = columnDesc.getDictionaryEncoding(); + + if (columnDesc.getCapabilities().hasMultipleValues()) { + return new DimensionSelector() + { + @Override + public IndexedInts getRow() + { + return column.getMultiValueRow(currRow); + } + + @Override + public int getValueCardinality() + { + return column.getCardinality(); + } + + @Override + public String lookupName(int id) + { + final String retVal = column.lookupName(id); + return retVal == null ? "" : retVal; + } + + @Override + public int lookupId(String name) + { + return column.lookupId(name); + } + }; + } + else { + return new DimensionSelector() + { + @Override + public IndexedInts getRow() + { + final int value = column.getSingleValueRow(currRow); + return new IndexedInts() + { + @Override + public int size() + { + return 1; + } + + @Override + public int get(int index) + { + return value; + } + + @Override + public Iterator iterator() + { + return Iterators.singleton(value); + } + }; + } + + @Override + public int getValueCardinality() + { + return column.getCardinality(); + } + + @Override + public String lookupName(int id) + { + return column.lookupName(id); + } + + @Override + public int lookupId(String name) + { + return column.lookupId(name); + } + }; + } + } + + @Override + public FloatMetricSelector makeFloatMetricSelector(String metric) + { + final String metricName = metric.toLowerCase(); + GenericColumn cachedMetricVals = genericColumnCache.get(metricName); + + if (cachedMetricVals == null) { + Column holder = index.getColumn(metricName); + if (holder != null && holder.getCapabilities().getType() == ValueType.FLOAT) { + cachedMetricVals = holder.getGenericColumn(); + genericColumnCache.put(metricName, cachedMetricVals); + } + } + + if (cachedMetricVals == null) { + return new FloatMetricSelector() + { + @Override + public float get() + { + return 0.0f; + } + }; + } + + final GenericColumn metricVals = cachedMetricVals; + return new FloatMetricSelector() + { + @Override + public float get() + { + return metricVals.getFloatSingleValueRow(currRow); + } + }; + } + + @Override + public ComplexMetricSelector makeComplexMetricSelector(String metric) + { + final String metricName = metric.toLowerCase(); + ComplexColumn cachedMetricVals = complexColumnCache.get(metricName); + + if (cachedMetricVals == null) { + Column holder = index.getColumn(metricName); + if (holder != null && holder.getCapabilities().getType() == ValueType.COMPLEX) { + cachedMetricVals = holder.getComplexColumn(); + complexColumnCache.put(metricName, cachedMetricVals); + } + } + + if (cachedMetricVals == null) { + return null; + } + + final ComplexColumn metricVals = cachedMetricVals; + return new ComplexMetricSelector() + { + @Override + public Class classOfObject() + { + return metricVals.getClazz(); + } + + @Override + public Object get() + { + return metricVals.getRowValue(currRow); + } + }; + } + }; + } + } + ); + + return MoreIterators.after( + retVal, + new Runnable() + { + @Override + public void run() + { + Closeables.closeQuietly(timestamps); + for (GenericColumn column : genericColumnCache.values()) { + Closeables.closeQuietly(column); + } + for (ComplexColumn complexColumn : complexColumnCache.values()) { + Closeables.closeQuietly(complexColumn); + } + } + } + ); + } + } + + private class MMappedBitmapIndexSelector implements BitmapIndexSelector + { + private final ColumnSelector index; + + public MMappedBitmapIndexSelector(final ColumnSelector index) + { + this.index = index; + } + + @Override + public Indexed getDimensionValues(String dimension) + { + final Column columnDesc = index.getColumn(dimension.toLowerCase()); + if (columnDesc == null || !columnDesc.getCapabilities().isDictionaryEncoded()) { + return null; + } + final DictionaryEncodedColumn column = columnDesc.getDictionaryEncoding(); + return new Indexed() + { + @Override + public Class getClazz() + { + return String.class; + } + + @Override + public int size() + { + return column.getCardinality(); + } + + @Override + public String get(int index) + { + return column.lookupName(index); + } + + @Override + public int indexOf(String value) + { + return column.lookupId(value); + } + + @Override + public Iterator iterator() + { + return IndexedIterable.create(this).iterator(); + } + }; + } + + @Override + public int getNumRows() + { + GenericColumn column = null; + try { + column = index.getTimeColumn().getGenericColumn(); + return column.size(); + } + finally { + Closeables.closeQuietly(column); + } + } + + @Override + public ImmutableConciseSet getConciseInvertedIndex(String dimension, String value) + { + return getInvertedIndex(dimension, value); + } + } +} diff --git a/server/src/main/java/com/metamx/druid/index/v1/processing/DimensionSelector.java b/server/src/main/java/com/metamx/druid/index/v1/processing/DimensionSelector.java index c2efd331c64..8ced8f388dd 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/processing/DimensionSelector.java +++ b/server/src/main/java/com/metamx/druid/index/v1/processing/DimensionSelector.java @@ -57,18 +57,12 @@ public interface DimensionSelector * A,B * B * - * would be turned into (per lookupExpansion) - * - * 0 - * 1 - * 0 - * 2 + * getRow() would return * - * at which point lookupExpansion would really return: - * - * lookupExpansion(1) => [0 1] - * lookupExpansion(2) => [0] - * lookupExpansion(3) => [1] + * getRow(0) => [0 1] + * getRow(1) => [0] + * getRow(2) => [0 1] + * getRow(3) => [1] * * and then lookupName would return: * @@ -82,7 +76,7 @@ public interface DimensionSelector /** * The ID is the int id value of the field. - * + * * @param name * @return */ diff --git a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java index a0dcbf94735..1d727f9abe3 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -25,19 +25,18 @@ import com.google.common.collect.Maps; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.DruidProcessingConfig; -import com.metamx.druid.GroupByQueryEngine; -import com.metamx.druid.GroupByQueryEngineConfig; +import com.metamx.druid.loading.DelegatingSegmentLoader; +import com.metamx.druid.loading.MMappedQueryableIndexFactory; +import com.metamx.druid.loading.QueryableIndexFactory; +import com.metamx.druid.loading.S3SegmentPuller; +import com.metamx.druid.loading.SingleSegmentLoader; +import com.metamx.druid.query.group.GroupByQueryEngine; +import com.metamx.druid.query.group.GroupByQueryEngineConfig; import com.metamx.druid.Query; import com.metamx.druid.collect.StupidPool; -import com.metamx.druid.loading.DelegatingStorageAdapterLoader; -import com.metamx.druid.loading.MMappedStorageAdapterFactory; import com.metamx.druid.loading.QueryableLoaderConfig; -import com.metamx.druid.loading.RealtimeSegmentGetter; -import com.metamx.druid.loading.S3SegmentGetter; -import com.metamx.druid.loading.S3ZippedSegmentGetter; -import com.metamx.druid.loading.SingleStorageAdapterLoader; -import com.metamx.druid.loading.StorageAdapterFactory; -import com.metamx.druid.loading.StorageAdapterLoader; +import com.metamx.druid.loading.S3ZippedSegmentPuller; +import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.group.GroupByQuery; import com.metamx.druid.query.group.GroupByQueryRunnerFactory; @@ -63,28 +62,26 @@ public class ServerInit { private static Logger log = new Logger(ServerInit.class); - public static StorageAdapterLoader makeDefaultQueryableLoader( + public static SegmentLoader makeDefaultQueryableLoader( RestS3Service s3Client, QueryableLoaderConfig config ) { - DelegatingStorageAdapterLoader delegateLoader = new DelegatingStorageAdapterLoader(); + DelegatingSegmentLoader delegateLoader = new DelegatingSegmentLoader(); - final S3SegmentGetter segmentGetter = new S3SegmentGetter(s3Client, config); - final S3ZippedSegmentGetter zippedGetter = new S3ZippedSegmentGetter(s3Client, config); - final RealtimeSegmentGetter realtimeGetter = new RealtimeSegmentGetter(config); - final StorageAdapterFactory factory; + final S3SegmentPuller segmentGetter = new S3SegmentPuller(s3Client, config); + final S3ZippedSegmentPuller zippedGetter = new S3ZippedSegmentPuller(s3Client, config); + final QueryableIndexFactory factory; if ("mmap".equals(config.getQueryableFactoryType())) { - factory = new MMappedStorageAdapterFactory(); + factory = new MMappedQueryableIndexFactory(); } else { throw new ISE("Unknown queryableFactoryType[%s]", config.getQueryableFactoryType()); } delegateLoader.setLoaderTypes( - ImmutableMap.builder() - .put("s3", new SingleStorageAdapterLoader(segmentGetter, factory)) - .put("s3_zip", new SingleStorageAdapterLoader(zippedGetter, factory)) - .put("realtime", new SingleStorageAdapterLoader(realtimeGetter, factory)) + ImmutableMap.builder() + .put("s3", new SingleSegmentLoader(segmentGetter, factory)) + .put("s3_zip", new SingleSegmentLoader(zippedGetter, factory)) .build() ); diff --git a/server/src/main/java/com/metamx/druid/loading/DelegatingStorageAdapterLoader.java b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java similarity index 63% rename from server/src/main/java/com/metamx/druid/loading/DelegatingStorageAdapterLoader.java rename to server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java index 5d421df7a0a..d576e59ae82 100644 --- a/server/src/main/java/com/metamx/druid/loading/DelegatingStorageAdapterLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java @@ -21,43 +21,44 @@ package com.metamx.druid.loading; import com.metamx.common.MapUtils; import com.metamx.common.logger.Logger; -import com.metamx.druid.StorageAdapter; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.Segment; import javax.inject.Inject; import java.util.Map; /** */ -public class DelegatingStorageAdapterLoader implements StorageAdapterLoader +public class DelegatingSegmentLoader implements SegmentLoader { - private static final Logger log = new Logger(DelegatingStorageAdapterLoader.class); + private static final Logger log = new Logger(DelegatingSegmentLoader.class); - private volatile Map loaderTypes; + private volatile Map loaderTypes; @Inject public void setLoaderTypes( - Map loaderTypes + Map loaderTypes ) { this.loaderTypes = loaderTypes; } @Override - public StorageAdapter getAdapter(Map loadSpec) throws StorageAdapterLoadingException + public Segment getSegment(DataSegment segment) throws StorageAdapterLoadingException { - return getLoader(loadSpec).getAdapter(loadSpec); + return getLoader(segment.getLoadSpec()).getSegment(segment); } @Override - public void cleanupAdapter(Map loadSpec) throws StorageAdapterLoadingException + public void cleanup(DataSegment segment) throws StorageAdapterLoadingException { - getLoader(loadSpec).cleanupAdapter(loadSpec); + getLoader(segment.getLoadSpec()).cleanup(segment); } - private StorageAdapterLoader getLoader(Map loadSpec) throws StorageAdapterLoadingException + private SegmentLoader getLoader(Map loadSpec) throws StorageAdapterLoadingException { String type = MapUtils.getString(loadSpec, "type"); - StorageAdapterLoader loader = loaderTypes.get(type); + SegmentLoader loader = loaderTypes.get(type); if (loader == null) { throw new StorageAdapterLoadingException("Unknown loader type[%s]. Known types are %s", type, loaderTypes.keySet()); diff --git a/server/src/main/java/com/metamx/druid/loading/ConvertingBaseQueryableFactory.java b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java similarity index 85% rename from server/src/main/java/com/metamx/druid/loading/ConvertingBaseQueryableFactory.java rename to server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java index 8cb720affd2..9e71e3aeb04 100644 --- a/server/src/main/java/com/metamx/druid/loading/ConvertingBaseQueryableFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java @@ -20,7 +20,7 @@ package com.metamx.druid.loading; import com.metamx.common.logger.Logger; -import com.metamx.druid.StorageAdapter; +import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.v1.IndexIO; import org.apache.commons.io.FileUtils; @@ -29,12 +29,12 @@ import java.io.IOException; /** */ -public abstract class ConvertingBaseQueryableFactory implements StorageAdapterFactory +public class MMappedQueryableIndexFactory implements QueryableIndexFactory { - private static final Logger log = new Logger(ConvertingBaseQueryableFactory.class); + private static final Logger log = new Logger(MMappedQueryableIndexFactory.class); @Override - public StorageAdapter factorize(File parentDir) throws StorageAdapterLoadingException + public QueryableIndex factorize(File parentDir) throws StorageAdapterLoadingException { File indexFile = new File(parentDir, "index.drd"); if (!indexFile.exists()) { @@ -61,7 +61,7 @@ public abstract class ConvertingBaseQueryableFactory implements StorageAdapterFa FileUtils.deleteDirectory(canBeMappedDir); } - return factorizeConverted(parentDir); + return IndexIO.loadIndex(parentDir); } catch (IOException e) { log.warn(e, "Got exception, deleting index[%s]", indexFile); @@ -74,6 +74,4 @@ public abstract class ConvertingBaseQueryableFactory implements StorageAdapterFa throw new StorageAdapterLoadingException(e, e.getMessage()); } } - - protected abstract StorageAdapter factorizeConverted(File parentDir) throws IOException; } diff --git a/server/src/main/java/com/metamx/druid/loading/StorageAdapterFactory.java b/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java similarity index 85% rename from server/src/main/java/com/metamx/druid/loading/StorageAdapterFactory.java rename to server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java index 92fec4e7632..d7f60309aa6 100644 --- a/server/src/main/java/com/metamx/druid/loading/StorageAdapterFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java @@ -19,13 +19,13 @@ package com.metamx.druid.loading; -import com.metamx.druid.StorageAdapter; +import com.metamx.druid.index.QueryableIndex; import java.io.File; /** */ -public interface StorageAdapterFactory +public interface QueryableIndexFactory { - public StorageAdapter factorize(File parentDir) throws StorageAdapterLoadingException; + public QueryableIndex factorize(File parentDir) throws StorageAdapterLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentGetter.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java similarity index 92% rename from server/src/main/java/com/metamx/druid/loading/S3SegmentGetter.java rename to server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java index d0d9fb7a0bd..380489548d5 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentGetter.java +++ b/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java @@ -23,6 +23,7 @@ import com.google.inject.Inject; import com.metamx.common.MapUtils; import com.metamx.common.StreamUtils; import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; import com.metamx.druid.common.s3.S3Utils; import org.apache.commons.io.FileUtils; import org.jets3t.service.impl.rest.httpclient.RestS3Service; @@ -38,9 +39,9 @@ import java.util.zip.GZIPInputStream; /** */ -public class S3SegmentGetter implements SegmentGetter +public class S3SegmentPuller implements SegmentPuller { - private static final Logger log = new Logger(S3SegmentGetter.class); + private static final Logger log = new Logger(S3SegmentPuller.class); private static final long DEFAULT_TIMEOUT = 5 * 60 * 1000; private static final String BUCKET = "bucket"; @@ -50,7 +51,7 @@ public class S3SegmentGetter implements SegmentGetter private final S3SegmentGetterConfig config; @Inject - public S3SegmentGetter( + public S3SegmentPuller( RestS3Service s3Client, S3SegmentGetterConfig config ) @@ -60,8 +61,9 @@ public class S3SegmentGetter implements SegmentGetter } @Override - public File getSegmentFiles(Map loadSpec) throws StorageAdapterLoadingException + public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException { + Map loadSpec = segment.getLoadSpec(); String s3Bucket = MapUtils.getString(loadSpec, "bucket"); String s3Path = MapUtils.getString(loadSpec, "key"); @@ -156,8 +158,9 @@ public class S3SegmentGetter implements SegmentGetter } @Override - public boolean cleanSegmentFiles(Map loadSpec) throws StorageAdapterLoadingException + public boolean cleanSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException { + Map loadSpec = segment.getLoadSpec(); File cacheFile = new File( config.getCacheDirectory(), computeCacheFilePath(MapUtils.getString(loadSpec, BUCKET), MapUtils.getString(loadSpec, KEY)) diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java new file mode 100644 index 00000000000..f4099154d2d --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java @@ -0,0 +1,149 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.loading; + +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.Closeables; +import com.metamx.common.ISE; +import com.metamx.common.StreamUtils; +import com.metamx.druid.client.DataSegment; +import com.metamx.emitter.EmittingLogger; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.codehaus.jackson.map.ObjectMapper; +import org.jets3t.service.S3ServiceException; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.model.S3Object; + +import java.io.ByteArrayInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.security.NoSuchAlgorithmException; +import java.util.zip.ZipEntry; +import java.util.zip.ZipOutputStream; + +public class S3SegmentPusher implements SegmentPusher +{ + private static final EmittingLogger log = new EmittingLogger(S3SegmentPusher.class); + private static final Joiner JOINER = Joiner.on("/").skipNulls(); + + private final RestS3Service s3Client; + private final S3SegmentPusherConfig config; + private final ObjectMapper jsonMapper; + + public S3SegmentPusher( + RestS3Service s3Client, + S3SegmentPusherConfig config, + ObjectMapper jsonMapper + ) + { + this.s3Client = s3Client; + this.config = config; + this.jsonMapper = jsonMapper; + } + + @Override + public DataSegment push(File file, DataSegment segment) throws IOException + { + log.info("Uploading [%s] to S3", file); + String outputKey = JOINER.join( + config.getBaseKey().isEmpty() ? null : config.getBaseKey(), + segment.getDataSource(), + String.format( + "%s_%s", + segment.getInterval().getStart(), + segment.getInterval().getEnd() + ), + segment.getVersion(), + segment.getShardSpec().getPartitionNum() + ); + + File indexFilesDir = file; + + long indexSize = 0; + final File zipOutFile = File.createTempFile("druid", "index.zip"); + ZipOutputStream zipOut = null; + try { + zipOut = new ZipOutputStream(new FileOutputStream(zipOutFile)); + File[] indexFiles = indexFilesDir.listFiles(); + for (File indexFile : indexFiles) { + log.info("Adding indexFile[%s] with size[%,d]. Total size[%,d]", indexFile, indexFile.length(), indexSize); + if (indexFile.length() >= Integer.MAX_VALUE) { + throw new ISE("indexFile[%s] too large [%,d]", indexFile, indexFile.length()); + } + zipOut.putNextEntry(new ZipEntry(indexFile.getName())); + IOUtils.copy(new FileInputStream(indexFile), zipOut); + indexSize += indexFile.length(); + } + } + finally { + Closeables.closeQuietly(zipOut); + } + + try { + S3Object toPush = new S3Object(zipOutFile); + + final String outputBucket = config.getBucket(); + toPush.setBucketName(outputBucket); + toPush.setKey(outputKey + "/index.zip"); + + log.info("Pushing %s.", toPush); + s3Client.putObject(outputBucket, toPush); + + DataSegment outputSegment = segment.withSize(indexSize) + .withLoadSpec( + ImmutableMap.of( + "type", "s3_zip", + "bucket", outputBucket, + "key", toPush.getKey() + ) + ); + + File descriptorFile = File.createTempFile("druid", "descriptor.json"); + StreamUtils.copyToFileAndClose(new ByteArrayInputStream(jsonMapper.writeValueAsBytes(segment)), descriptorFile); + S3Object descriptorObject = new S3Object(descriptorFile); + descriptorObject.setBucketName(outputBucket); + descriptorObject.setKey(outputKey + "/descriptor.json"); + + log.info("Pushing %s", descriptorObject); + s3Client.putObject(outputBucket, descriptorObject); + + log.info("Deleting Index File[%s]", indexFilesDir); + FileUtils.deleteDirectory(indexFilesDir); + + log.info("Deleting zipped index File[%s]", zipOutFile); + zipOutFile.delete(); + + log.info("Deleting descriptor file[%s]", descriptorFile); + descriptorFile.delete(); + + return outputSegment; + } + catch (NoSuchAlgorithmException e) { + throw new IOException(e); + } + catch (S3ServiceException e) { + throw new IOException(e); + } + } +} \ No newline at end of file diff --git a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusherConfig.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentPusherConfig.java similarity index 96% rename from realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusherConfig.java rename to server/src/main/java/com/metamx/druid/loading/S3SegmentPusherConfig.java index 96a96eeea10..0bd66a1a913 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusherConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/S3SegmentPusherConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.realtime; +package com.metamx.druid.loading; import org.skife.config.Config; import org.skife.config.Default; diff --git a/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentGetter.java b/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java similarity index 91% rename from server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentGetter.java rename to server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java index d90d33c586d..8fd8ebd4542 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentGetter.java +++ b/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java @@ -23,6 +23,7 @@ import com.google.common.io.Closeables; import com.metamx.common.MapUtils; import com.metamx.common.StreamUtils; import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; import com.metamx.druid.common.s3.S3Utils; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; @@ -43,9 +44,9 @@ import java.util.zip.ZipInputStream; /** */ -public class S3ZippedSegmentGetter implements SegmentGetter +public class S3ZippedSegmentPuller implements SegmentPuller { - private static final Logger log = new Logger(S3ZippedSegmentGetter.class); + private static final Logger log = new Logger(S3ZippedSegmentPuller.class); private static final String BUCKET = "bucket"; private static final String KEY = "key"; @@ -53,7 +54,7 @@ public class S3ZippedSegmentGetter implements SegmentGetter private final RestS3Service s3Client; private final S3SegmentGetterConfig config; - public S3ZippedSegmentGetter( + public S3ZippedSegmentPuller( RestS3Service s3Client, S3SegmentGetterConfig config ) @@ -63,8 +64,9 @@ public class S3ZippedSegmentGetter implements SegmentGetter } @Override - public File getSegmentFiles(Map loadSpec) throws StorageAdapterLoadingException + public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException { + Map loadSpec = segment.getLoadSpec(); String s3Bucket = MapUtils.getString(loadSpec, "bucket"); String s3Path = MapUtils.getString(loadSpec, "key"); @@ -161,8 +163,9 @@ public class S3ZippedSegmentGetter implements SegmentGetter } @Override - public boolean cleanSegmentFiles(Map loadSpec) throws StorageAdapterLoadingException + public boolean cleanSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException { + Map loadSpec = segment.getLoadSpec(); File cacheFile = new File( config.getCacheDirectory(), computeCacheFilePath( diff --git a/server/src/main/java/com/metamx/druid/loading/MMappedStorageAdapterFactory.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java similarity index 65% rename from server/src/main/java/com/metamx/druid/loading/MMappedStorageAdapterFactory.java rename to server/src/main/java/com/metamx/druid/loading/SegmentLoader.java index 13c65ad5c4d..1ca54b89106 100644 --- a/server/src/main/java/com/metamx/druid/loading/MMappedStorageAdapterFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java @@ -19,20 +19,13 @@ package com.metamx.druid.loading; -import com.metamx.druid.StorageAdapter; -import com.metamx.druid.index.v1.IndexIO; -import com.metamx.druid.index.v1.MMappedIndexStorageAdapter; - -import java.io.File; -import java.io.IOException; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.Segment; /** */ -public class MMappedStorageAdapterFactory extends ConvertingBaseQueryableFactory +public interface SegmentLoader { - @Override - protected StorageAdapter factorizeConverted(File parentDir) throws IOException - { - return new MMappedIndexStorageAdapter(IndexIO.mapDir(parentDir)); - } + public Segment getSegment(DataSegment loadSpec) throws StorageAdapterLoadingException; + public void cleanup(DataSegment loadSpec) throws StorageAdapterLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentGetter.java b/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java similarity index 77% rename from server/src/main/java/com/metamx/druid/loading/SegmentGetter.java rename to server/src/main/java/com/metamx/druid/loading/SegmentPuller.java index 53e7481ef1a..9cba65f425c 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentGetter.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java @@ -19,13 +19,15 @@ package com.metamx.druid.loading; +import com.metamx.druid.client.DataSegment; + import java.io.File; import java.util.Map; /** */ -public interface SegmentGetter +public interface SegmentPuller { - public File getSegmentFiles(Map loadSpec) throws StorageAdapterLoadingException; - public boolean cleanSegmentFiles(Map loadSpec) throws StorageAdapterLoadingException; + public File getSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException; + public boolean cleanSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException; } diff --git a/realtime/src/main/java/com/metamx/druid/realtime/SegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/SegmentPusher.java similarity index 95% rename from realtime/src/main/java/com/metamx/druid/realtime/SegmentPusher.java rename to server/src/main/java/com/metamx/druid/loading/SegmentPusher.java index 16a9253137c..3700215efc1 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/SegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentPusher.java @@ -17,15 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.realtime; +package com.metamx.druid.loading; import com.metamx.druid.client.DataSegment; import java.io.File; import java.io.IOException; -/** - */ public interface SegmentPusher { public DataSegment push(File file, DataSegment segment) throws IOException; diff --git a/server/src/main/java/com/metamx/druid/loading/RealtimeSegmentGetter.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java similarity index 50% rename from server/src/main/java/com/metamx/druid/loading/RealtimeSegmentGetter.java rename to server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index 15a37ebf611..19c3981e988 100644 --- a/server/src/main/java/com/metamx/druid/loading/RealtimeSegmentGetter.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -20,46 +20,39 @@ package com.metamx.druid.loading; import com.google.inject.Inject; -import com.metamx.common.logger.Logger; - -import java.io.File; -import java.util.Map; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.QueryableIndex; +import com.metamx.druid.index.QueryableIndexSegment; +import com.metamx.druid.index.Segment; /** */ -public class RealtimeSegmentGetter implements SegmentGetter +public class SingleSegmentLoader implements SegmentLoader { - private static final Logger log = new Logger(RealtimeSegmentGetter.class); - - private final S3SegmentGetterConfig config; + private final SegmentPuller segmentPuller; + private final QueryableIndexFactory factory; @Inject - public RealtimeSegmentGetter( - S3SegmentGetterConfig config + public SingleSegmentLoader( + SegmentPuller segmentPuller, + QueryableIndexFactory factory ) { - this.config = config; + this.segmentPuller = segmentPuller; + this.factory = factory; } @Override - public File getSegmentFiles(final Map loadSpec) throws StorageAdapterLoadingException + public Segment getSegment(DataSegment segment) throws StorageAdapterLoadingException { - try { - File cacheFile = (File) loadSpec.get("file"); + final QueryableIndex index = factory.factorize(segmentPuller.getSegmentFiles(segment)); - if (!cacheFile.exists()) { - throw new StorageAdapterLoadingException("Unable to find persisted file!"); - } - return cacheFile; - } - catch (Exception e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); - } + return new QueryableIndexSegment(segment.getIdentifier(), index); } @Override - public boolean cleanSegmentFiles(Map loadSpec) throws StorageAdapterLoadingException + public void cleanup(DataSegment segment) throws StorageAdapterLoadingException { - throw new UnsupportedOperationException(); + segmentPuller.cleanSegmentFiles(segment); } } diff --git a/server/src/main/java/com/metamx/druid/query/QueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/QueryRunnerFactory.java index 5eda9f40514..083ebae543e 100644 --- a/server/src/main/java/com/metamx/druid/query/QueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/QueryRunnerFactory.java @@ -21,6 +21,7 @@ package com.metamx.druid.query; import com.metamx.druid.Query; import com.metamx.druid.StorageAdapter; +import com.metamx.druid.index.Segment; import java.util.concurrent.ExecutorService; @@ -28,7 +29,7 @@ import java.util.concurrent.ExecutorService; */ public interface QueryRunnerFactory> { - public QueryRunner createRunner(StorageAdapter adapter); + public QueryRunner createRunner(Segment segment); public QueryRunner mergeRunners(ExecutorService queryExecutor, Iterable> queryRunners); public QueryToolChest getToolchest(); } diff --git a/server/src/main/java/com/metamx/druid/GroupByQueryEngine.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java similarity index 99% rename from server/src/main/java/com/metamx/druid/GroupByQueryEngine.java rename to server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java index 48afd63ac49..b8d13be6674 100644 --- a/server/src/main/java/com/metamx/druid/GroupByQueryEngine.java +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package com.metamx.druid.query.group; import com.google.common.base.Function; import com.google.common.collect.ImmutableList; @@ -31,6 +31,7 @@ import com.metamx.common.ISE; import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.FunctionalIterator; import com.metamx.common.guava.Sequence; +import com.metamx.druid.StorageAdapter; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.BufferAggregator; import com.metamx.druid.aggregation.post.PostAggregator; @@ -42,7 +43,6 @@ import com.metamx.druid.index.v1.processing.DimensionSelector; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; import com.metamx.druid.query.dimension.DimensionSpec; -import com.metamx.druid.query.group.GroupByQuery; import org.joda.time.Interval; import javax.annotation.Nullable; diff --git a/server/src/main/java/com/metamx/druid/GroupByQueryEngineConfig.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngineConfig.java similarity index 96% rename from server/src/main/java/com/metamx/druid/GroupByQueryEngineConfig.java rename to server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngineConfig.java index 3236a64f684..2422849d96a 100644 --- a/server/src/main/java/com/metamx/druid/GroupByQueryEngineConfig.java +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngineConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package com.metamx.druid.query.group; import org.skife.config.Config; import org.skife.config.Default; diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java index 01327a44c80..c1fcc22786a 100644 --- a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java @@ -26,9 +26,9 @@ import com.metamx.common.ISE; import com.metamx.common.guava.ExecutorExecutingSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.GroupByQueryEngine; import com.metamx.druid.Query; import com.metamx.druid.StorageAdapter; +import com.metamx.druid.index.Segment; import com.metamx.druid.input.Row; import com.metamx.druid.query.ConcatQueryRunner; import com.metamx.druid.query.QueryRunner; @@ -62,22 +62,9 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory createRunner(final StorageAdapter adapter) + public QueryRunner createRunner(final Segment segment) { - return new QueryRunner() - { - @Override - public Sequence run(Query input) - { - if (! (input instanceof GroupByQuery)) { - throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class); - } - - GroupByQuery query = (GroupByQuery) input; - - return engine.process(query, adapter); - } - }; + return new GroupByQueryRunner(segment, engine); } @Override @@ -132,4 +119,26 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory + { + private final StorageAdapter adapter; + private final GroupByQueryEngine engine; + + public GroupByQueryRunner(Segment segment, final GroupByQueryEngine engine) + { + this.adapter = segment.asStorageAdapter(); + this.engine = engine; + } + + @Override + public Sequence run(Query input) + { + if (! (input instanceof GroupByQuery)) { + throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class); + } + + return engine.process((GroupByQuery) input, adapter); + } + } } diff --git a/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java index ce624a944f4..b616456f4f2 100644 --- a/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java @@ -28,6 +28,7 @@ import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.Query; import com.metamx.druid.StorageAdapter; +import com.metamx.druid.index.Segment; import com.metamx.druid.query.ConcatQueryRunner; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerFactory; @@ -56,7 +57,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory> createRunner(final StorageAdapter adapter) + public QueryRunner> createRunner(final Segment adapter) { return new QueryRunner>() { @@ -66,7 +67,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory> createRunner(final StorageAdapter adapter) + public QueryRunner> createRunner(final Segment segment) { - return new QueryRunner>() - { - @Override - public Sequence> run(final Query> input) - { - if (!(input instanceof SearchQuery)) { - throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class); - } - - final SearchQuery query = (SearchQuery) input; - - return new BaseSequence, Iterator>>( - new BaseSequence.IteratorMaker, Iterator>>() - { - @Override - public Iterator> make() - { - return Lists.newArrayList( - new SearchResultBuilder( - adapter.getInterval().getStart(), - adapter.searchDimensions( - query, - Filters.convertDimensionFilters(query.getDimensionsFilter()) - ) - ).build() - ).iterator(); - } - - @Override - public void cleanup(Iterator> toClean) - { - - } - } - ); - } - }; - + return new SearchQueryRunner(segment); } @Override @@ -102,4 +66,49 @@ public class SearchQueryRunnerFactory implements QueryRunnerFactory> + { + private final StorageAdapter adapter; + + public SearchQueryRunner(Segment segment) + { + this.adapter = segment.asStorageAdapter(); + } + + @Override + public Sequence> run(final Query> input) + { + if (!(input instanceof SearchQuery)) { + throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class); + } + + final SearchQuery query = (SearchQuery) input; + + return new BaseSequence, Iterator>>( + new BaseSequence.IteratorMaker, Iterator>>() + { + @Override + public Iterator> make() + { + return Iterators.singleton( + new SearchResultBuilder( + adapter.getInterval().getStart(), + adapter.searchDimensions( + query, + Filters.convertDimensionFilters(query.getDimensionsFilter()) + ) + ).build() + ); + } + + @Override + public void cleanup(Iterator> toClean) + { + + } + } + ); + } + } } diff --git a/server/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index 459a520c916..4006238786d 100644 --- a/server/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -24,7 +24,7 @@ import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; import com.metamx.druid.StorageAdapter; -import com.metamx.druid.collect.StupidPool; +import com.metamx.druid.index.Segment; import com.metamx.druid.query.ChainedExecutionQueryRunner; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerFactory; @@ -33,7 +33,6 @@ import com.metamx.druid.query.group.GroupByQuery; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; -import java.nio.ByteBuffer; import java.util.Iterator; import java.util.concurrent.ExecutorService; @@ -45,41 +44,9 @@ public class TimeBoundaryQueryRunnerFactory private static final TimeBoundaryQueryQueryToolChest toolChest = new TimeBoundaryQueryQueryToolChest(); @Override - public QueryRunner> createRunner(final StorageAdapter adapter) + public QueryRunner> createRunner(final Segment segment) { - return new QueryRunner>() - { - @Override - public Sequence> run(Query> input) - { - if (!(input instanceof TimeBoundaryQuery)) { - throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class); - } - - final TimeBoundaryQuery legacyQuery = (TimeBoundaryQuery) input; - - return new BaseSequence, Iterator>>( - new BaseSequence.IteratorMaker, Iterator>>() - { - @Override - public Iterator> make() - { - return legacyQuery.buildResult( - adapter.getInterval().getStart(), - adapter.getMinTime(), - adapter.getMaxTime() - ).iterator(); - } - - @Override - public void cleanup(Iterator> toClean) - { - - } - } - ); - } - }; + return new TimeBoundaryQueryRunner(segment); } @Override @@ -97,4 +64,45 @@ public class TimeBoundaryQueryRunnerFactory { return toolChest; } + + private static class TimeBoundaryQueryRunner implements QueryRunner> + { + private final StorageAdapter adapter; + + public TimeBoundaryQueryRunner(Segment segment) + { + this.adapter = segment.asStorageAdapter(); + } + + @Override + public Sequence> run(Query> input) + { + if (!(input instanceof TimeBoundaryQuery)) { + throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class); + } + + final TimeBoundaryQuery legacyQuery = (TimeBoundaryQuery) input; + + return new BaseSequence, Iterator>>( + new BaseSequence.IteratorMaker, Iterator>>() + { + @Override + public Iterator> make() + { + return legacyQuery.buildResult( + adapter.getInterval().getStart(), + adapter.getMinTime(), + adapter.getMaxTime() + ).iterator(); + } + + @Override + public void cleanup(Iterator> toClean) + { + + } + } + ); + } + } } diff --git a/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryEngine.java b/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryEngine.java new file mode 100644 index 00000000000..018dc329fc2 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryEngine.java @@ -0,0 +1,100 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.query.timeseries; + +import com.google.common.base.Function; +import com.metamx.common.guava.BaseSequence; +import com.metamx.common.guava.Sequence; +import com.metamx.druid.StorageAdapter; +import com.metamx.druid.TimeseriesResultBuilder; +import com.metamx.druid.aggregation.Aggregator; +import com.metamx.druid.aggregation.AggregatorFactory; +import com.metamx.druid.aggregation.post.PostAggregator; +import com.metamx.druid.index.brita.Filters; +import com.metamx.druid.index.v1.processing.Cursor; +import com.metamx.druid.query.QueryRunnerHelper; +import com.metamx.druid.result.Result; +import com.metamx.druid.result.TimeseriesResultValue; + +import java.util.Iterator; +import java.util.List; + +/** + */ +public class TimeseriesQueryEngine +{ + public Sequence> process(final TimeseriesQuery query, final StorageAdapter adapter) + { + return new BaseSequence, Iterator>>( + new BaseSequence.IteratorMaker, Iterator>>() + { + @Override + public Iterator> make() + { + return QueryRunnerHelper.makeCursorBasedQuery( + adapter, + query.getQuerySegmentSpec().getIntervals(), + Filters.convertDimensionFilters(query.getDimensionsFilter()), + query.getGranularity(), + new Function>() + { + private final List aggregatorSpecs = query.getAggregatorSpecs(); + private final List postAggregatorSpecs = query.getPostAggregatorSpecs(); + + @Override + public Result apply(Cursor cursor) + { + Aggregator[] aggregators = QueryRunnerHelper.makeAggregators(cursor, aggregatorSpecs); + + while (!cursor.isDone()) { + for (Aggregator aggregator : aggregators) { + aggregator.aggregate(); + } + cursor.advance(); + } + + TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime()); + + for (Aggregator aggregator : aggregators) { + bob.addMetric(aggregator); + } + + for (PostAggregator postAgg : postAggregatorSpecs) { + bob.addMetric(postAgg); + } + + return bob.build(); + } + } + ).iterator(); + } + + @Override + public void cleanup(Iterator> toClean) + { + // TODO: Let's fix this to actually use Sequences for the closing of stuff + while (toClean.hasNext()) { + toClean.next(); + } + } + } + ); + } +} diff --git a/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerFactory.java index 32e612f12e8..8d413730e09 100644 --- a/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerFactory.java @@ -19,29 +19,19 @@ package com.metamx.druid.query.timeseries; -import com.google.common.base.Function; import com.metamx.common.ISE; -import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; import com.metamx.druid.StorageAdapter; -import com.metamx.druid.TimeseriesResultBuilder; -import com.metamx.druid.aggregation.Aggregator; -import com.metamx.druid.aggregation.AggregatorFactory; -import com.metamx.druid.aggregation.post.PostAggregator; -import com.metamx.druid.index.brita.Filters; -import com.metamx.druid.index.v1.processing.Cursor; +import com.metamx.druid.index.Segment; import com.metamx.druid.query.ChainedExecutionQueryRunner; import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryRunnerHelper; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.group.GroupByQuery; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; -import java.util.Iterator; -import java.util.List; import java.util.concurrent.ExecutorService; /** @@ -50,74 +40,12 @@ public class TimeseriesQueryRunnerFactory implements QueryRunnerFactory, TimeseriesQuery> { private static final TimeseriesQueryQueryToolChest toolChest = new TimeseriesQueryQueryToolChest(); + private static final TimeseriesQueryEngine engine = new TimeseriesQueryEngine(); @Override - public QueryRunner> createRunner(final StorageAdapter adapter) + public QueryRunner> createRunner(final Segment segment) { - return new QueryRunner>() - { - @Override - public Sequence> run(Query> input) - { - if (!(input instanceof TimeseriesQuery)) { - throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class); - } - - final TimeseriesQuery query = (TimeseriesQuery) input; - - return new BaseSequence, Iterator>>( - new BaseSequence.IteratorMaker, Iterator>>() - { - @Override - public Iterator> make() - { - return QueryRunnerHelper.makeCursorBasedQuery( - adapter, - query.getQuerySegmentSpec().getIntervals(), - Filters.convertDimensionFilters(query.getDimensionsFilter()), - query.getGranularity(), - new Function>() - { - private final List aggregatorSpecs = query.getAggregatorSpecs(); - private final List postAggregatorSpecs = query.getPostAggregatorSpecs(); - - @Override - public Result apply(Cursor cursor) - { - Aggregator[] aggregators = QueryRunnerHelper.makeAggregators(cursor, aggregatorSpecs); - - while (!cursor.isDone()) { - for (Aggregator aggregator : aggregators) { - aggregator.aggregate(); - } - cursor.advance(); - } - - TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime()); - - for (Aggregator aggregator : aggregators) { - bob.addMetric(aggregator); - } - - for (PostAggregator postAgg : postAggregatorSpecs) { - bob.addMetric(postAgg); - } - - return bob.build(); - } - } - ).iterator(); - } - - @Override - public void cleanup(Iterator> toClean) - { - - } - } - ); - } - }; + return new TimeseriesQueryRunner(segment); } @Override @@ -135,4 +63,24 @@ public class TimeseriesQueryRunnerFactory { return toolChest; } + + private static class TimeseriesQueryRunner implements QueryRunner> + { + private final StorageAdapter adapter; + + public TimeseriesQueryRunner(Segment segment) + { + this.adapter = segment.asStorageAdapter(); + } + + @Override + public Sequence> run(Query> input) + { + if (!(input instanceof TimeseriesQuery)) { + throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class); + } + + return engine.process((TimeseriesQuery) input, adapter); + } + } } diff --git a/client/src/test/java/com/metamx/druid/TestHelper.java b/server/src/test/java/com/metamx/druid/TestHelper.java similarity index 99% rename from client/src/test/java/com/metamx/druid/TestHelper.java rename to server/src/test/java/com/metamx/druid/TestHelper.java index 342b3b4111c..cf6c60a38a1 100644 --- a/client/src/test/java/com/metamx/druid/TestHelper.java +++ b/server/src/test/java/com/metamx/druid/TestHelper.java @@ -90,4 +90,4 @@ public class TestHelper { Assert.assertEquals(msg, expected, actual); } -} \ No newline at end of file +} diff --git a/client/src/test/java/com/metamx/druid/client/RangeIterable.java b/server/src/test/java/com/metamx/druid/client/RangeIterable.java similarity index 100% rename from client/src/test/java/com/metamx/druid/client/RangeIterable.java rename to server/src/test/java/com/metamx/druid/client/RangeIterable.java diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index 1f44bd3e65f..3e3a36b77ed 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -36,10 +36,12 @@ import com.metamx.druid.Query; import com.metamx.druid.QueryGranularity; import com.metamx.druid.StorageAdapter; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.QueryableIndex; +import com.metamx.druid.index.Segment; import com.metamx.druid.index.brita.Filter; import com.metamx.druid.index.v1.SegmentIdAttachedStorageAdapter; import com.metamx.druid.index.v1.processing.Cursor; -import com.metamx.druid.loading.StorageAdapterLoader; +import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.loading.StorageAdapterLoadingException; import com.metamx.druid.metrics.NoopServiceEmitter; import com.metamx.druid.query.CacheStrategy; @@ -85,19 +87,19 @@ public class ServerManagerTest factory = new MyQueryRunnerFactory(); serverManager = new ServerManager( - new StorageAdapterLoader() + new SegmentLoader() { @Override - public StorageAdapter getAdapter(final Map loadSpec) + public Segment getSegment(final DataSegment segment) { - return new StorageAdapterForTesting( - MapUtils.getString(loadSpec, "version"), - (Interval) loadSpec.get("interval") + return new SegmentForTesting( + MapUtils.getString(segment.getLoadSpec(), "version"), + (Interval) segment.getLoadSpec().get("interval") ); } @Override - public void cleanupAdapter(Map loadSpec) throws StorageAdapterLoadingException + public void cleanup(DataSegment segment) throws StorageAdapterLoadingException { } @@ -285,11 +287,11 @@ public class ServerManagerTest QueryRunner> runner = serverManager.getQueryRunnerForIntervals(query, intervals); final Sequence> seq = runner.run(query); Sequences.toList(seq, Lists.>newArrayList()); - Iterator adaptersIter = factory.getAdapters().iterator(); + Iterator adaptersIter = factory.getAdapters().iterator(); while (expectedIter.hasNext() && adaptersIter.hasNext()) { Pair expectedVals = expectedIter.next(); - StorageAdapterForTesting value = adaptersIter.next(); + SegmentForTesting value = adaptersIter.next(); Assert.assertEquals(expectedVals.lhs, value.getVersion()); Assert.assertEquals(expectedVals.rhs, value.getInterval()); @@ -301,12 +303,12 @@ public class ServerManagerTest factory.clearAdapters(); } - private static class StorageAdapterForTesting implements StorageAdapter + private static class SegmentForTesting implements Segment { private final String version; private final Interval interval; - StorageAdapterForTesting( + SegmentForTesting( String version, Interval interval ) @@ -328,41 +330,23 @@ public class ServerManagerTest @Override public String getSegmentIdentifier() { - throw new UnsupportedOperationException(); + return version; } @Override - public int getDimensionCardinality(String dimension) + public Interval getDataInterval() + { + return interval; + } + + @Override + public QueryableIndex asQueryableIndex() { throw new UnsupportedOperationException(); } @Override - public DateTime getMinTime() - { - throw new UnsupportedOperationException(); - } - - @Override - public DateTime getMaxTime() - { - throw new UnsupportedOperationException(); - } - - @Override - public Capabilities getCapabilities() - { - throw new UnsupportedOperationException(); - } - - @Override - public Iterable makeCursors(Filter filter, Interval interval, QueryGranularity gran) - { - throw new UnsupportedOperationException(); - } - - @Override - public Iterable searchDimensions(SearchQuery query, Filter filter) + public StorageAdapter asStorageAdapter() { throw new UnsupportedOperationException(); } @@ -370,12 +354,12 @@ public class ServerManagerTest public static class MyQueryRunnerFactory implements QueryRunnerFactory, SearchQuery> { - private List adapters = Lists.newArrayList(); + private List adapters = Lists.newArrayList(); @Override - public QueryRunner> createRunner(StorageAdapter adapter) + public QueryRunner> createRunner(Segment adapter) { - adapters.add((StorageAdapterForTesting) ((SegmentIdAttachedStorageAdapter) adapter).getDelegate()); + adapters.add((SegmentForTesting) adapter); return new NoopQueryRunner>(); } @@ -393,7 +377,7 @@ public class ServerManagerTest return new NoopQueryToolChest, SearchQuery>(); } - public List getAdapters() + public List getAdapters() { return adapters; } diff --git a/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java b/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java index 040673707b0..c493193372c 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java @@ -28,7 +28,7 @@ import com.metamx.druid.client.DruidServer; import com.metamx.druid.client.DruidServerConfig; import com.metamx.druid.client.ZKPhoneBook; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.loading.NoopStorageAdapterLoader; +import com.metamx.druid.loading.NoopSegmentLoader; import com.metamx.druid.metrics.NoopServiceEmitter; import com.metamx.druid.query.NoopQueryRunnerFactoryConglomerate; import com.metamx.druid.shard.NoneShardSpec; @@ -72,7 +72,7 @@ public class ZkCoordinatorTest } serverManager = new ServerManager( - new NoopStorageAdapterLoader(), + new NoopSegmentLoader(), new NoopQueryRunnerFactoryConglomerate(), new NoopServiceEmitter(), MoreExecutors.sameThreadExecutor() diff --git a/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java b/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java index 2a2013b68fc..6b07bfa5902 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java +++ b/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java @@ -21,38 +21,20 @@ package com.metamx.druid.index.v1; import com.google.common.base.Charsets; import com.google.common.base.Function; -import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Maps; import com.google.common.io.CharStreams; -import com.google.common.io.Closeables; import com.google.common.io.InputSupplier; import com.google.common.io.LineProcessor; -import com.google.common.primitives.Ints; -import com.metamx.common.ISE; import com.metamx.common.logger.Logger; -import com.metamx.common.parsers.DelimitedParser; -import com.metamx.common.parsers.Parser; -import com.metamx.common.parsers.ToLowerCaseParser; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; import com.metamx.druid.client.RangeIterable; -import com.metamx.druid.guava.GuavaUtils; -import com.metamx.druid.index.v1.serde.ComplexMetricSerde; -import com.metamx.druid.index.v1.serde.ComplexMetrics; +import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.indexer.data.DelimitedDataSpec; import com.metamx.druid.indexer.data.StringInputRowParser; import com.metamx.druid.indexer.data.TimestampSpec; -import com.metamx.druid.input.InputRow; -import com.metamx.druid.input.MapBasedInputRow; -import com.metamx.druid.kv.ArrayIndexed; -import com.metamx.druid.kv.Indexed; -import com.metamx.druid.kv.IndexedFloats; -import com.metamx.druid.kv.IndexedInts; -import com.metamx.druid.kv.IndexedLongs; -import com.metamx.druid.kv.Indexedids; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -61,10 +43,7 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; import java.net.URL; -import java.nio.ByteOrder; -import java.nio.FloatBuffer; import java.util.Arrays; -import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicLong; @@ -74,195 +53,18 @@ public class TestIndex { private static final Logger log = new Logger(TestIndex.class); - private static Index index = null; - private static Index unionIndexTop = null; - private static Index unionIndexBottom = null; private static IncrementalIndex realtimeIndex = null; - private static MMappedIndex mmappedIndex = null; - private static MMappedIndex mergedRealtime = null; + private static QueryableIndex mmappedIndex = null; + private static QueryableIndex mergedRealtime = null; public static final String[] COLUMNS = new String[]{"ts", "provider", "quALIty", "plAcEmEnT", "pLacementish", "iNdEx"}; public static final String[] DIMENSIONS = new String[]{"provider", "quALIty", "plAcEmEnT", "pLacementish"}; public static final String[] METRICS = new String[]{"iNdEx"}; - public static final Map dimIds = Maps.uniqueIndex( - new RangeIterable(4), - new Function() - { - @Override - public String apply(@Nullable Integer input) - { - return DIMENSIONS[input]; - } - } - ); private static final Interval DATA_INTERVAL = new Interval("2011-01-12T00:00:00.000Z/2011-04-16T00:00:00.000Z"); private static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ new DoubleSumAggregatorFactory(METRICS[0], METRICS[0]) }; - public static Index convertMMapToIndex(MMappedIndex mmappedIndex) - { - Indexed dimsIndexed = mmappedIndex.getAvailableDimensions(); - String[] dimensions = new String[dimsIndexed.size()]; - for (int i = 0; i < dimsIndexed.size(); ++i) { - dimensions[i] = dimsIndexed.get(i); - } - - Indexed metricsIndexed = mmappedIndex.getAvailableMetrics(); - String[] metrics = new String[metricsIndexed.size()]; - for (int i = 0; i < metricsIndexed.size(); ++i) { - metrics[i] = metricsIndexed.get(i); - } - - IndexedLongs timeBuf = mmappedIndex.getReadOnlyTimestamps(); - long[] timestamps = new long[timeBuf.size()]; - timeBuf.fill(0, timestamps); - Closeables.closeQuietly(timeBuf); - - Map metricVals = Maps.newLinkedHashMap(); - for (String metric : metrics) { - MetricHolder holder = mmappedIndex.getMetricHolder(metric); - switch (holder.getType()) { - case FLOAT: - IndexedFloats mmappedFloats = holder.getFloatType(); - float[] metricValsArray = new float[mmappedFloats.size()]; - mmappedFloats.fill(0, metricValsArray); - Closeables.closeQuietly(mmappedFloats); - - metricVals.put( - metric, - MetricHolder.floatMetric( - metric, - CompressedFloatsIndexedSupplier.fromFloatBuffer( - FloatBuffer.wrap(metricValsArray), - ByteOrder.nativeOrder() - ) - ) - ); - break; - case COMPLEX: - Indexed complexObjects = holder.getComplexType(); - Object[] vals = new Object[complexObjects.size()]; - for (int i = 0; i < complexObjects.size(); ++i) { - vals[i] = complexObjects.get(i); - } - - final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(holder.getTypeName()); - if (serde == null) { - throw new ISE("Unknown type[%s]", holder.getTypeName()); - } - - metricVals.put( - metric, - MetricHolder.complexMetric( - metric, - holder.getTypeName(), - new ArrayIndexed(vals, serde.getObjectStrategy().getClazz()) - ) - ); - break; - } - } - - Map> dimIdLookup = Maps.newHashMap(); - Map reverseDimLookup = Maps.newHashMap(); - Map invertedIndexesMap = Maps.newHashMap(); - Map dimensionColumns = Maps.newHashMap(); - - for (String dimension : dimensions) { - final Indexed dimValueLookup = mmappedIndex.getDimValueLookup(dimension); - String[] values = new String[dimValueLookup.size()]; - for (int i = 0; i < dimValueLookup.size(); ++i) { - values[i] = dimValueLookup.get(i); - } - - Map lookupMap = Maps.newHashMapWithExpectedSize(dimValueLookup.size()); - for (int i = 0; i < values.length; i++) { - lookupMap.put(values[i], i); - } - - ImmutableConciseSet[] invertedIndexes = new ImmutableConciseSet[values.length]; - final Indexed dimValuesIndexed = mmappedIndex.getDimValueLookup(dimension); - for (int i = 0; i < dimValuesIndexed.size(); ++i) { - invertedIndexes[i] = mmappedIndex.getInvertedIndex(dimension, dimValuesIndexed.get(i)); - } - - int[] dimValues = new int[timestamps.length]; - Map, Integer> rowGroupings = Maps.newHashMap(); - final Indexed dimColumn = mmappedIndex.getDimColumn(dimension); - for (int i = 0; i < dimColumn.size(); ++i) { - int[] expansionValue = Indexedids.arrayFromIndexedInts(dimColumn.get(i)); - Integer value = rowGroupings.get(Ints.asList(expansionValue)); - if (value == null) { - value = rowGroupings.size(); - rowGroupings.put(Ints.asList(expansionValue), value); - } - dimValues[i] = value; - } - - int[][] expansionValues = new int[rowGroupings.size()][]; - for (Map.Entry, Integer> entry : rowGroupings.entrySet()) { - expansionValues[entry.getValue()] = Ints.toArray(entry.getKey()); - } - - dimIdLookup.put(dimension, lookupMap); - reverseDimLookup.put(dimension, values); - invertedIndexesMap.put(dimension, invertedIndexes); - dimensionColumns.put(dimension, new DimensionColumn(expansionValues, dimValues)); - } - - return new Index( - dimensions, - metrics, - mmappedIndex.getDataInterval(), - timestamps, - metricVals, - dimIdLookup, - reverseDimLookup, - invertedIndexesMap, - dimensionColumns - ); - } - - public static Index getTestIndex() throws IOException - { - synchronized (log) { - if (index != null) { - return index; - } - } - - return index = convertMMapToIndex(getMMappedTestIndex()); - } - - public static Index getTestUnionIndexTop() throws IOException - { - synchronized (log) { - if (unionIndexTop != null) { - return unionIndexTop; - } - } - - IncrementalIndex incrementalIndex = makeRealtimeIndex("druid.sample.tsv.top"); - MMappedIndex mmapped = persistRealtimeAndLoadMMapped(incrementalIndex); - - return unionIndexTop = convertMMapToIndex(mmapped); - } - - public static Index getTestUnionIndexBottom() throws IOException - { - synchronized (log) { - if (unionIndexBottom != null) { - return unionIndexBottom; - } - } - - IncrementalIndex incrementalIndex = makeRealtimeIndex("druid.sample.tsv.bottom"); - MMappedIndex mmapped = persistRealtimeAndLoadMMapped(incrementalIndex); - - return unionIndexBottom = convertMMapToIndex(mmapped); - } - public static IncrementalIndex getIncrementalTestIndex() { synchronized (log) { @@ -274,7 +76,7 @@ public class TestIndex return realtimeIndex = makeRealtimeIndex("druid.sample.tsv"); } - public static MMappedIndex getMMappedTestIndex() + public static QueryableIndex getMMappedTestIndex() { synchronized (log) { if (mmappedIndex != null) { @@ -288,7 +90,7 @@ public class TestIndex return mmappedIndex; } - public static MMappedIndex mergedRealtimeIndex() + public static QueryableIndex mergedRealtimeIndex() { synchronized (log) { if (mergedRealtime != null) { @@ -316,12 +118,9 @@ public class TestIndex IndexMerger.persist(top, DATA_INTERVAL, topFile); IndexMerger.persist(bottom, DATA_INTERVAL, bottomFile); - mergedRealtime = com.metamx.druid.index.v1.IndexIO.mapDir( + mergedRealtime = IndexIO.loadIndex( IndexMerger.mergeMMapped( - Arrays.asList( - com.metamx.druid.index.v1.IndexIO.mapDir(topFile), - com.metamx.druid.index.v1.IndexIO.mapDir(bottomFile) - ), + Arrays.asList(IndexIO.mapDir(topFile), IndexIO.mapDir(bottomFile)), METRIC_AGGS, mergedFile ) @@ -377,8 +176,6 @@ public class TestIndex runOnce = true; } - final String[] splits = line.split("\t"); - retVal.add(parser.parse(line)); ++lineCount; @@ -403,7 +200,7 @@ public class TestIndex return retVal; } - public static MMappedIndex persistRealtimeAndLoadMMapped(IncrementalIndex index) + public static QueryableIndex persistRealtimeAndLoadMMapped(IncrementalIndex index) { try { File someTmpFile = File.createTempFile("billy", "yay"); @@ -412,7 +209,7 @@ public class TestIndex someTmpFile.deleteOnExit(); IndexMerger.persist(index, someTmpFile); - return com.metamx.druid.index.v1.IndexIO.mapDir(someTmpFile); + return IndexIO.loadIndex(someTmpFile); } catch (IOException e) { throw Throwables.propagate(e); diff --git a/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java b/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java new file mode 100644 index 00000000000..dc1f640fa5b --- /dev/null +++ b/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java @@ -0,0 +1,67 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.loading; + +import com.metamx.druid.StorageAdapter; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.QueryableIndex; +import com.metamx.druid.index.Segment; +import org.joda.time.Interval; + +/** +*/ +public class NoopSegmentLoader implements SegmentLoader +{ + @Override + public Segment getSegment(final DataSegment segment) throws StorageAdapterLoadingException + { + return new Segment() + { + @Override + public String getSegmentIdentifier() + { + return segment.getIdentifier(); + } + + @Override + public Interval getDataInterval() + { + return segment.getInterval(); + } + + @Override + public QueryableIndex asQueryableIndex() + { + throw new UnsupportedOperationException(); + } + + @Override + public StorageAdapter asStorageAdapter() + { + throw new UnsupportedOperationException(); + } + }; + } + + @Override + public void cleanup(DataSegment loadSpec) throws StorageAdapterLoadingException + { + } +} diff --git a/server/src/test/java/com/metamx/druid/loading/NoopStorageAdapterLoader.java b/server/src/test/java/com/metamx/druid/loading/NoopStorageAdapterLoader.java deleted file mode 100644 index a5bcf8b0b34..00000000000 --- a/server/src/test/java/com/metamx/druid/loading/NoopStorageAdapterLoader.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.loading; - -import com.metamx.druid.Capabilities; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.StorageAdapter; -import com.metamx.druid.index.brita.Filter; -import com.metamx.druid.index.v1.processing.Cursor; -import com.metamx.druid.query.search.SearchHit; -import com.metamx.druid.query.search.SearchQuery; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -import java.util.Map; - -/** -*/ -public class NoopStorageAdapterLoader implements StorageAdapterLoader -{ - @Override - public StorageAdapter getAdapter(final Map loadSpec) - { - return new StorageAdapter() - { - @Override - public String getSegmentIdentifier() - { - throw new UnsupportedOperationException(); - } - - @Override - public Interval getInterval() - { - throw new UnsupportedOperationException(); - } - - @Override - public int getDimensionCardinality(String dimension) - { - throw new UnsupportedOperationException(); - } - - @Override - public DateTime getMinTime() - { - throw new UnsupportedOperationException(); - } - - @Override - public DateTime getMaxTime() - { - throw new UnsupportedOperationException(); - } - - @Override - public Capabilities getCapabilities() - { - throw new UnsupportedOperationException(); - } - - @Override - public Iterable makeCursors(Filter filter, Interval interval, QueryGranularity gran) - { - throw new UnsupportedOperationException(); - } - - @Override - public Iterable searchDimensions(SearchQuery query, Filter filter) - { - throw new UnsupportedOperationException(); - } - }; - } - - @Override - public void cleanupAdapter(Map loadSpec) throws StorageAdapterLoadingException - { - - } -} diff --git a/server/src/test/java/com/metamx/druid/query/QueryRunnerTestHelper.java b/server/src/test/java/com/metamx/druid/query/QueryRunnerTestHelper.java index 89d7802d044..de3734c39f8 100644 --- a/server/src/test/java/com/metamx/druid/query/QueryRunnerTestHelper.java +++ b/server/src/test/java/com/metamx/druid/query/QueryRunnerTestHelper.java @@ -30,12 +30,18 @@ import com.metamx.druid.aggregation.LongSumAggregatorFactory; import com.metamx.druid.aggregation.post.ArithmeticPostAggregator; import com.metamx.druid.aggregation.post.ConstantPostAggregator; import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; +import com.metamx.druid.index.IncrementalIndexSegment; +import com.metamx.druid.index.QueryableIndex; +import com.metamx.druid.index.QueryableIndexSegment; +import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.index.v1.IncrementalIndexStorageAdapter; import com.metamx.druid.index.v1.Index; import com.metamx.druid.index.v1.IndexStorageAdapter; import com.metamx.druid.index.v1.MMappedIndex; +import com.metamx.druid.index.v1.MMappedIndexQueryableIndex; import com.metamx.druid.index.v1.MMappedIndexStorageAdapter; +import com.metamx.druid.index.v1.QueryableIndexStorageAdapter; import com.metamx.druid.index.v1.TestIndex; import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; import com.metamx.druid.query.segment.QuerySegmentSpec; @@ -108,26 +114,19 @@ public class QueryRunnerTestHelper ) throws IOException { - final Index testIndex = TestIndex.getTestIndex(); final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex(); - final MMappedIndex persistedRTIndex = TestIndex.getMMappedTestIndex(); - final MMappedIndex mergedRT = TestIndex.mergedRealtimeIndex(); + final QueryableIndex mMappedTestIndex = TestIndex.getMMappedTestIndex(); + final QueryableIndex mergedRealtimeIndex = TestIndex.mergedRealtimeIndex(); return Arrays.asList( new Object[][]{ { - makeQueryRunner(factory, new IndexStorageAdapter(testIndex)) + makeQueryRunner(factory, new IncrementalIndexSegment(rtIndex)) }, { - makeQueryRunner(factory, new MMappedIndexStorageAdapter(MMappedIndex.fromIndex(testIndex))) + makeQueryRunner(factory, new QueryableIndexSegment(null, mMappedTestIndex)) }, { - makeQueryRunner(factory, new IncrementalIndexStorageAdapter(rtIndex)) - }, - { - makeQueryRunner(factory, new MMappedIndexStorageAdapter(persistedRTIndex)) - }, - { - makeQueryRunner(factory, new MMappedIndexStorageAdapter(mergedRT)) + makeQueryRunner(factory, new QueryableIndexSegment(null, mergedRealtimeIndex)) } } ); @@ -135,7 +134,7 @@ public class QueryRunnerTestHelper private static QueryRunner makeQueryRunner( QueryRunnerFactory> factory, - StorageAdapter adapter + Segment adapter ) { return new FinalizeResultsQueryRunner( diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java index 61c84f9c447..0860715963b 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java @@ -24,17 +24,13 @@ import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.GroupByQueryEngine; -import com.metamx.druid.GroupByQueryEngineConfig; import com.metamx.druid.Query; import com.metamx.druid.collect.StupidPool; -import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerTestHelper; import com.metamx.druid.query.timeseries.TimeseriesQuery; -import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerFactory; import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerTest; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; @@ -42,7 +38,6 @@ import org.joda.time.DateTime; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; @@ -165,4 +160,11 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest // Skip this test because the timeseries test expects a day that doesn't have a filter match to be filled in, // but group by just doesn't return a value if the filter doesn't match. } + + @Override + public void testTimeseriesWithFilterOnNonExistentDimension() + { + // Skip this test because the timeseries test expects a day that doesn't have a filter match to be filled in, + // but group by just doesn't return a value if the filter doesn't match. + } } diff --git a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java index 0368745ef8b..b6479983634 100644 --- a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -958,6 +958,48 @@ public class TimeseriesQueryRunnerTest TestHelper.assertExpectedResults(expectedResults, results); } + @Test + public void testTimeseriesWithFilterOnNonExistentDimension() + { + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .filters("bobby", "billy") + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01"), + new TimeseriesResultValue( + ImmutableMap.of( + "rows", 0L, + "index", 0.0, + "addRowsIndexConstant", 1.0 + ) + ) + ), + new Result( + new DateTime("2011-04-02"), + new TimeseriesResultValue( + ImmutableMap.of( + "rows", 0L, + "index", 0.0, + "addRowsIndexConstant", 1.0 + ) + ) + ) + ); + + Iterable> results = Sequences.toList( + runner.run(query), + Lists.>newArrayList() + ); + TestHelper.assertExpectedResults(expectedResults, results); + } + @Test public void testTimeseriesWithNonExistentFilter() { From 72db507cf8e043f11e51b45b12cec4e3f2dba0b7 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Sun, 6 Jan 2013 23:37:52 -0800 Subject: [PATCH 11/92] 1) Update java-util version to be a non-SNAPSHOT version --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d5cdd164498..5a165a16ba2 100644 --- a/pom.xml +++ b/pom.xml @@ -68,7 +68,7 @@ com.metamx java-util - 0.16.1-SNAPSHOT + 0.18.0 com.metamx From b62bdf915b90aab90e8c1f2198804b77d66abc6d Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Mon, 7 Jan 2013 10:09:19 -0800 Subject: [PATCH 12/92] 1) Fix stray import... yay IntelliJ --- .../metamx/druid/index/v1/QueryableIndexStorageAdapter.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java b/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java index 7e7f8a33f54..445dd937ce5 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java +++ b/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java @@ -22,6 +22,7 @@ package com.metamx.druid.index.v1; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterators; import com.google.common.collect.Maps; import com.google.common.io.Closeables; import com.metamx.common.collect.MoreIterators; @@ -47,7 +48,6 @@ import com.metamx.druid.kv.IndexedInts; import com.metamx.druid.kv.IndexedIterable; import com.metamx.druid.processing.ComplexMetricSelector; import com.metamx.druid.processing.FloatMetricSelector; -import com.sun.xml.internal.xsom.impl.scd.Iterators; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -366,7 +366,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter @Override public Iterator iterator() { - return Iterators.singleton(value); + return Iterators.singletonIterator(value); } }; } @@ -670,7 +670,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter @Override public Iterator iterator() { - return Iterators.singleton(value); + return Iterators.singletonIterator(value); } }; } From 4ef31de62d78fcc4ba1a3f884424257c43d2dafb Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Mon, 7 Jan 2013 13:11:28 -0800 Subject: [PATCH 13/92] 1) Fix stray import, take 2... yay IntelliJ --- .../metamx/druid/query/search/SearchQueryRunnerFactory.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java index 8282b16aeb6..21cc70aa1e6 100644 --- a/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java @@ -19,6 +19,7 @@ package com.metamx.druid.query.search; +import com.google.common.collect.Iterators; import com.metamx.common.ISE; import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; @@ -34,7 +35,6 @@ import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.group.GroupByQuery; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; -import com.sun.xml.internal.xsom.impl.scd.Iterators; import java.util.Iterator; import java.util.concurrent.ExecutorService; @@ -91,7 +91,7 @@ public class SearchQueryRunnerFactory implements QueryRunnerFactory> make() { - return Iterators.singleton( + return Iterators.singletonIterator( new SearchResultBuilder( adapter.getInterval().getStart(), adapter.searchDimensions( From aff4a05ca385adbd4e3e0986aa18be155fa2b01d Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 7 Jan 2013 13:41:26 -0800 Subject: [PATCH 14/92] db based configs for indexer workers --- .../java/com/metamx/druid/db/DbConnector.java | 12 + .../merger/coordinator/RemoteTaskRunner.java | 23 +- .../config/EC2AutoScalingStrategyConfig.java | 17 -- .../config/RemoteTaskRunnerConfig.java | 7 - .../config/WorkerSetupManagerConfig.java | 17 ++ .../http/IndexerCoordinatorNode.java | 50 +++- .../http/IndexerCoordinatorResource.java | 28 ++- .../http/IndexerCoordinatorServletModule.java | 7 +- .../scaling/EC2AutoScalingStrategy.java | 36 +-- .../merger/coordinator/setup/EC2NodeData.java | 52 +++++ .../coordinator/setup/GalaxyUserData.java | 43 ++++ .../coordinator/setup/WorkerNodeData.java | 14 ++ .../coordinator/setup/WorkerSetupData.java | 52 +++++ .../coordinator/setup/WorkerSetupManager.java | 216 ++++++++++++++++++ .../coordinator/setup/WorkerUserData.java | 14 ++ .../coordinator/RemoteTaskRunnerTest.java | 37 +-- .../scaling/EC2AutoScalingStrategyTest.java | 65 +++--- 17 files changed, 582 insertions(+), 108 deletions(-) create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerNodeData.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerUserData.java diff --git a/common/src/main/java/com/metamx/druid/db/DbConnector.java b/common/src/main/java/com/metamx/druid/db/DbConnector.java index 99712df22a5..75cf0ba27e7 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnector.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnector.java @@ -59,6 +59,18 @@ public class DbConnector ); } + public static void createWorkerSetupTable(final DBI dbi, final String workerTableName) + { + createTable( + dbi, + workerTableName, + String.format( + "CREATE table %s (minVersion TINYTEXT NOT NULL, minNumWorkers SMALLINT NOT NULL, nodeData LONGTEXT NOT NULL, userData LONGTEXT NOT NULL)", + workerTableName + ) + ); + } + public static void createTable( final DBI dbi, final String tableName, diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 2a235b88d86..addb789762f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -39,6 +39,7 @@ import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.scaling.AutoScalingData; import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.druid.merger.worker.Worker; import com.metamx.emitter.EmittingLogger; import com.netflix.curator.framework.CuratorFramework; @@ -88,6 +89,7 @@ public class RemoteTaskRunner implements TaskRunner private final ScheduledExecutorService scheduledExec; private final RetryPolicyFactory retryPolicyFactory; private final ScalingStrategy strategy; + private final WorkerSetupManager workerSetupManager; // all workers that exist in ZK private final Map zkWorkers = new ConcurrentHashMap(); @@ -109,7 +111,8 @@ public class RemoteTaskRunner implements TaskRunner PathChildrenCache workerPathCache, ScheduledExecutorService scheduledExec, RetryPolicyFactory retryPolicyFactory, - ScalingStrategy strategy + ScalingStrategy strategy, + WorkerSetupManager workerSetupManager ) { this.jsonMapper = jsonMapper; @@ -119,6 +122,7 @@ public class RemoteTaskRunner implements TaskRunner this.scheduledExec = scheduledExec; this.retryPolicyFactory = retryPolicyFactory; this.strategy = strategy; + this.workerSetupManager = workerSetupManager; } @LifecycleStart @@ -169,7 +173,7 @@ public class RemoteTaskRunner implements TaskRunner public void run() { if (currentlyTerminating.isEmpty()) { - if (zkWorkers.size() <= config.getMinNumWorkers()) { + if (zkWorkers.size() <= workerSetupManager.getWorkerSetupData().getMinNumWorkers()) { return; } @@ -388,8 +392,7 @@ public class RemoteTaskRunner implements TaskRunner synchronized (statusLock) { try { if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) || - event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) - { + event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); final TaskStatus taskStatus; @@ -399,7 +402,7 @@ public class RemoteTaskRunner implements TaskRunner event.getData().getData(), TaskStatus.class ); - if(!taskStatus.getId().equals(taskId)) { + if (!taskStatus.getId().equals(taskId)) { // Sanity check throw new ISE( "Worker[%s] status id does not match payload id: %s != %s", @@ -408,7 +411,8 @@ public class RemoteTaskRunner implements TaskRunner taskStatus.getId() ); } - } catch (Exception e) { + } + catch (Exception e) { log.warn(e, "Worker[%s] wrote bogus status for task: %s", worker.getHost(), taskId); retryTask(new CleanupPaths(worker.getHost(), taskId), tasks.get(taskId)); throw Throwables.propagate(e); @@ -446,7 +450,8 @@ public class RemoteTaskRunner implements TaskRunner } } } - } catch(Exception e) { + } + catch (Exception e) { log.makeAlert(e, "Failed to handle new worker status") .addData("worker", worker.getHost()) .addData("znode", event.getData().getPath()) @@ -526,7 +531,9 @@ public class RemoteTaskRunner implements TaskRunner public boolean apply(WorkerWrapper input) { return (!input.isAtCapacity() && - input.getWorker().getVersion().compareTo(config.getMinWorkerVersion()) >= 0); + input.getWorker() + .getVersion() + .compareTo(workerSetupManager.getWorkerSetupData().getMinVersion()) >= 0); } } ) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/EC2AutoScalingStrategyConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/EC2AutoScalingStrategyConfig.java index c364070e313..a8cfcf8df22 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/EC2AutoScalingStrategyConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/EC2AutoScalingStrategyConfig.java @@ -26,24 +26,7 @@ import org.skife.config.Default; */ public abstract class EC2AutoScalingStrategyConfig { - @Config("druid.indexer.amiId") - public abstract String getAmiId(); - @Config("druid.indexer.worker.port") @Default("8080") public abstract String getWorkerPort(); - - @Config("druid.indexer.instanceType") - public abstract String getInstanceType(); - - @Config("druid.indexer.minNumInstancesToProvision") - @Default("1") - public abstract int getMinNumInstancesToProvision(); - - @Config("druid.indexer.maxNumInstancesToProvision") - @Default("1") - public abstract int getMaxNumInstancesToProvision(); - - @Config("druid.indexer.userDataFile") - public abstract String getUserDataFile(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java index 00b869ea6da..2e20c4ffff2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java @@ -37,13 +37,6 @@ public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig @Default("2012-01-01T00:55:00.000Z") public abstract DateTime getTerminateResourcesOriginDateTime(); - @Config("druid.indexer.minWorkerVersion") - public abstract String getMinWorkerVersion(); - - @Config("druid.indexer.minNumWorkers") - @Default("1") - public abstract int getMinNumWorkers(); - @Config("druid.indexer.maxWorkerIdleTimeMillisBeforeDeletion") @Default("1") public abstract int getMaxWorkerIdleTimeMillisBeforeDeletion(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java new file mode 100644 index 00000000000..ad7444b657e --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java @@ -0,0 +1,17 @@ +package com.metamx.druid.merger.coordinator.config; + +import org.joda.time.Duration; +import org.skife.config.Config; +import org.skife.config.Default; + +/** + */ +public abstract class WorkerSetupManagerConfig +{ + @Config("druid.indexer.workerSetupTable") + public abstract String getWorkerSetupTable(); + + @Config("druid.indexer.poll.duration") + @Default("PT1M") + public abstract Duration getPollDuration(); +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 1c11c62cb7b..348a62b9a28 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -69,6 +69,8 @@ import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.NoopScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; +import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig; import com.metamx.druid.realtime.S3SegmentPusher; import com.metamx.druid.realtime.S3SegmentPusherConfig; import com.metamx.druid.realtime.SegmentPusher; @@ -98,6 +100,7 @@ import org.mortbay.jetty.servlet.DefaultServlet; import org.mortbay.jetty.servlet.FilterHolder; import org.mortbay.jetty.servlet.ServletHolder; import org.skife.config.ConfigurationObjectFactory; +import org.skife.jdbi.v2.DBI; import java.net.URL; import java.util.Arrays; @@ -133,6 +136,7 @@ public class IndexerCoordinatorNode extends RegisteringNode private CuratorFramework curatorFramework = null; private ScheduledExecutorFactory scheduledExecutorFactory = null; private IndexerZkConfig indexerZkConfig; + private WorkerSetupManager workerSetupManager = null; private TaskRunnerFactory taskRunnerFactory = null; private TaskMaster taskMaster = null; private Server server = null; @@ -160,14 +164,16 @@ public class IndexerCoordinatorNode extends RegisteringNode return this; } - public void setMergerDBCoordinator(MergerDBCoordinator mergerDBCoordinator) + public IndexerCoordinatorNode setMergerDBCoordinator(MergerDBCoordinator mergerDBCoordinator) { this.mergerDBCoordinator = mergerDBCoordinator; + return this; } - public void setTaskQueue(TaskQueue taskQueue) + public IndexerCoordinatorNode setTaskQueue(TaskQueue taskQueue) { this.taskQueue = taskQueue; + return this; } public IndexerCoordinatorNode setMergeDbCoordinator(MergerDBCoordinator mergeDbCoordinator) @@ -182,9 +188,16 @@ public class IndexerCoordinatorNode extends RegisteringNode return this; } - public void setTaskRunnerFactory(TaskRunnerFactory taskRunnerFactory) + public IndexerCoordinatorNode setWorkerSetupManager(WorkerSetupManager workerSetupManager) + { + this.workerSetupManager = workerSetupManager; + return this; + } + + public IndexerCoordinatorNode setTaskRunnerFactory(TaskRunnerFactory taskRunnerFactory) { this.taskRunnerFactory = taskRunnerFactory; + return this; } public void init() throws Exception @@ -202,6 +215,7 @@ public class IndexerCoordinatorNode extends RegisteringNode initializeJacksonSubtypes(); initializeCurator(); initializeIndexerZkConfig(); + initializeWorkerSetupManager(); initializeTaskRunnerFactory(); initializeTaskMaster(); initializeServer(); @@ -220,7 +234,8 @@ public class IndexerCoordinatorNode extends RegisteringNode jsonMapper, config, emitter, - taskQueue + taskQueue, + workerSetupManager ) ); @@ -447,6 +462,27 @@ public class IndexerCoordinatorNode extends RegisteringNode } } + public void initializeWorkerSetupManager() + { + if (workerSetupManager == null) { + final DbConnectorConfig dbConnectorConfig = configFactory.build(DbConnectorConfig.class); + final DBI dbi = new DbConnector(dbConnectorConfig).getDBI(); + final WorkerSetupManagerConfig workerSetupManagerConfig = configFactory.build(WorkerSetupManagerConfig.class); + + DbConnector.createWorkerSetupTable(dbi, workerSetupManagerConfig.getWorkerSetupTable()); + workerSetupManager = new WorkerSetupManager( + dbi, Executors.newScheduledThreadPool( + 1, + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat("WorkerSetupManagerExec--%d") + .build() + ), jsonMapper, workerSetupManagerConfig + ); + } + lifecycle.addManagedInstance(workerSetupManager); + } + public void initializeTaskRunnerFactory() { if (taskRunnerFactory == null) { @@ -476,7 +512,8 @@ public class IndexerCoordinatorNode extends RegisteringNode PropUtils.getProperty(props, "com.metamx.aws.secretKey") ) ), - configFactory.build(EC2AutoScalingStrategyConfig.class) + configFactory.build(EC2AutoScalingStrategyConfig.class), + workerSetupManager ); } else if (config.getStrategyImpl().equalsIgnoreCase("noop")) { strategy = new NoopScalingStrategy(); @@ -491,7 +528,8 @@ public class IndexerCoordinatorNode extends RegisteringNode new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), true), retryScheduledExec, new RetryPolicyFactory(configFactory.build(RetryPolicyConfig.class)), - strategy + strategy, + workerSetupManager ); } }; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index 6cf9b0a7c16..e4acd93514f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -28,6 +28,8 @@ import com.metamx.druid.merger.common.task.MergeTask; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.service.ServiceEmitter; import javax.ws.rs.Consumes; @@ -48,18 +50,21 @@ public class IndexerCoordinatorResource private final IndexerCoordinatorConfig config; private final ServiceEmitter emitter; private final TaskQueue tasks; + private final WorkerSetupManager workerSetupManager; @Inject public IndexerCoordinatorResource( IndexerCoordinatorConfig config, ServiceEmitter emitter, - TaskQueue tasks + TaskQueue tasks, + WorkerSetupManager workerSetupManager ) throws Exception { this.config = config; this.emitter = emitter; this.tasks = tasks; + this.workerSetupManager = workerSetupManager; } @POST @@ -115,4 +120,25 @@ public class IndexerCoordinatorResource { return Response.ok(ImmutableMap.of("task", taskid)).build(); } + + @GET + @Path("/worker/setup") + @Produces("application/json") + public Response getWorkerSetupData() + { + return Response.ok(workerSetupManager.getWorkerSetupData()).build(); + } + + @POST + @Path("/worker/setup") + @Consumes("application/json") + public Response setWorkerSetupData( + final WorkerSetupData workerSetupData + ) + { + if (!workerSetupManager.setWorkerSetupData(workerSetupData)) { + return Response.status(Response.Status.BAD_REQUEST).build(); + } + return Response.ok().build(); + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java index 9c657bdc292..4cc1df9fa6f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java @@ -22,6 +22,7 @@ package com.metamx.druid.merger.coordinator.http; import com.google.inject.Provides; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.service.ServiceEmitter; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; @@ -38,18 +39,21 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule private final IndexerCoordinatorConfig indexerCoordinatorConfig; private final ServiceEmitter emitter; private final TaskQueue tasks; + private final WorkerSetupManager workerSetupManager; public IndexerCoordinatorServletModule( ObjectMapper jsonMapper, IndexerCoordinatorConfig indexerCoordinatorConfig, ServiceEmitter emitter, - TaskQueue tasks + TaskQueue tasks, + WorkerSetupManager workerSetupManager ) { this.jsonMapper = jsonMapper; this.indexerCoordinatorConfig = indexerCoordinatorConfig; this.emitter = emitter; this.tasks = tasks; + this.workerSetupManager = workerSetupManager; } @Override @@ -60,6 +64,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule bind(IndexerCoordinatorConfig.class).toInstance(indexerCoordinatorConfig); bind(ServiceEmitter.class).toInstance(emitter); bind(TaskQueue.class).toInstance(tasks); + bind(WorkerSetupManager.class).toInstance(workerSetupManager); serve("/*").with(GuiceContainer.class); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index 265fe62287c..eed69ae7f9e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -24,19 +24,20 @@ import com.amazonaws.services.ec2.model.DescribeInstancesRequest; import com.amazonaws.services.ec2.model.DescribeInstancesResult; import com.amazonaws.services.ec2.model.Filter; import com.amazonaws.services.ec2.model.Instance; -import com.amazonaws.services.ec2.model.InstanceType; import com.amazonaws.services.ec2.model.Reservation; import com.amazonaws.services.ec2.model.RunInstancesRequest; import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; import com.google.common.base.Function; import com.google.common.collect.Lists; +import com.metamx.common.ISE; import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; +import com.metamx.druid.merger.coordinator.setup.EC2NodeData; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.EmittingLogger; import org.codehaus.jackson.map.ObjectMapper; -import javax.annotation.Nullable; -import java.io.File; import java.util.List; /** @@ -48,31 +49,40 @@ public class EC2AutoScalingStrategy implements ScalingStrategy private final ObjectMapper jsonMapper; private final AmazonEC2Client amazonEC2Client; private final EC2AutoScalingStrategyConfig config; + private final WorkerSetupManager workerSetupManager; public EC2AutoScalingStrategy( ObjectMapper jsonMapper, AmazonEC2Client amazonEC2Client, - EC2AutoScalingStrategyConfig config + EC2AutoScalingStrategyConfig config, + WorkerSetupManager workerSetupManager ) { this.jsonMapper = jsonMapper; this.amazonEC2Client = amazonEC2Client; this.config = config; + this.workerSetupManager = workerSetupManager; } @Override public AutoScalingData provision() { try { + WorkerSetupData setupData = workerSetupManager.getWorkerSetupData(); + if (!(setupData.getNodeData() instanceof EC2NodeData)) { + throw new ISE("DB misconfiguration! Node data is an instance of [%s]", setupData.getNodeData().getClass()); + } + EC2NodeData workerConfig = (EC2NodeData) setupData.getNodeData(); + log.info("Creating new instance(s)..."); RunInstancesResult result = amazonEC2Client.runInstances( new RunInstancesRequest( - config.getAmiId(), - config.getMinNumInstancesToProvision(), - config.getMaxNumInstancesToProvision() + workerConfig.getAmiId(), + workerConfig.getMinInstances(), + workerConfig.getMaxInstances() ) - .withInstanceType(InstanceType.fromValue(config.getInstanceType())) - .withUserData(jsonMapper.writeValueAsString(new File(config.getUserDataFile()))) + .withInstanceType(workerConfig.getInstanceType()) + .withUserData(jsonMapper.writeValueAsString(setupData.getUserData())) ); List instanceIds = Lists.transform( @@ -80,7 +90,7 @@ public class EC2AutoScalingStrategy implements ScalingStrategy new Function() { @Override - public String apply(@Nullable Instance input) + public String apply(Instance input) { return input.getInstanceId(); } @@ -95,7 +105,7 @@ public class EC2AutoScalingStrategy implements ScalingStrategy new Function() { @Override - public String apply(@Nullable Instance input) + public String apply(Instance input) { return String.format("%s:%s", input.getPrivateIpAddress(), config.getWorkerPort()); } @@ -135,7 +145,7 @@ public class EC2AutoScalingStrategy implements ScalingStrategy new Function() { @Override - public String apply(@Nullable Instance input) + public String apply(Instance input) { return input.getInstanceId(); } @@ -150,7 +160,7 @@ public class EC2AutoScalingStrategy implements ScalingStrategy new Function() { @Override - public String apply(@Nullable Instance input) + public String apply(Instance input) { return String.format("%s:%s", input.getPrivateIpAddress(), config.getWorkerPort()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java new file mode 100644 index 00000000000..209444c6731 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java @@ -0,0 +1,52 @@ +package com.metamx.druid.merger.coordinator.setup; + +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +/** + */ +public class EC2NodeData implements WorkerNodeData +{ + private final String amiId; + private final String instanceType; + private final int minInstances; + private final int maxInstances; + + @JsonCreator + public EC2NodeData( + @JsonProperty("amiId") String amiId, + @JsonProperty("instanceType") String instanceType, + @JsonProperty("minInstances") int minInstances, + @JsonProperty("maxInstances") int maxInstances + ) + { + this.amiId = amiId; + this.instanceType = instanceType; + this.minInstances = minInstances; + this.maxInstances = maxInstances; + } + + @JsonProperty + public String getAmiId() + { + return amiId; + } + + @JsonProperty + public String getInstanceType() + { + return instanceType; + } + + @JsonProperty + public int getMinInstances() + { + return minInstances; + } + + @JsonProperty + public int getMaxInstances() + { + return maxInstances; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java new file mode 100644 index 00000000000..0baa0ddfb9a --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java @@ -0,0 +1,43 @@ +package com.metamx.druid.merger.coordinator.setup; + +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +/** + */ +public class GalaxyUserData implements WorkerUserData +{ + public final String env; + public final String ver; + public final String type; + + @JsonCreator + public GalaxyUserData( + @JsonProperty("env") String env, + @JsonProperty("ver") String ver, + @JsonProperty("type") String type + ) + { + this.env = env; + this.ver = ver; + this.type = type; + } + + @JsonProperty + public String getEnv() + { + return env; + } + + @JsonProperty + public String getVer() + { + return ver; + } + + @JsonProperty + public String getType() + { + return type; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerNodeData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerNodeData.java new file mode 100644 index 00000000000..8068a4c267b --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerNodeData.java @@ -0,0 +1,14 @@ +package com.metamx.druid.merger.coordinator.setup; + +import org.codehaus.jackson.annotate.JsonSubTypes; +import org.codehaus.jackson.annotate.JsonTypeInfo; + +/** + */ +@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type") +@JsonSubTypes(value={ + @JsonSubTypes.Type(name="ec2", value=EC2NodeData.class) +}) +public interface WorkerNodeData +{ +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java new file mode 100644 index 00000000000..224b22167ae --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java @@ -0,0 +1,52 @@ +package com.metamx.druid.merger.coordinator.setup; + +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +/** + */ +public class WorkerSetupData +{ + private final String minVersion; + private final int minNumWorkers; + private final WorkerNodeData nodeData; + private final WorkerUserData userData; + + @JsonCreator + public WorkerSetupData( + @JsonProperty("minVersion") String minVersion, + @JsonProperty("minNumWorkers") int minNumWorkers, + @JsonProperty("nodeData") WorkerNodeData nodeData, + @JsonProperty("userData") WorkerUserData userData + ) + { + this.minVersion = minVersion; + this.minNumWorkers = minNumWorkers; + this.nodeData = nodeData; + this.userData = userData; + } + + @JsonProperty + public String getMinVersion() + { + return minVersion; + } + + @JsonProperty + public int getMinNumWorkers() + { + return minNumWorkers; + } + + @JsonProperty + public WorkerNodeData getNodeData() + { + return nodeData; + } + + @JsonProperty + public WorkerUserData getUserData() + { + return userData; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java new file mode 100644 index 00000000000..a4ab8fe8d5d --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java @@ -0,0 +1,216 @@ +package com.metamx.druid.merger.coordinator.setup; + +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; +import com.metamx.common.ISE; +import com.metamx.common.concurrent.ScheduledExecutors; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.common.logger.Logger; +import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig; +import org.apache.commons.collections.MapUtils; +import org.codehaus.jackson.map.ObjectMapper; +import org.joda.time.Duration; +import org.skife.jdbi.v2.DBI; +import org.skife.jdbi.v2.FoldController; +import org.skife.jdbi.v2.Folder3; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.StatementContext; +import org.skife.jdbi.v2.tweak.HandleCallback; + +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; + +/** + */ +public class WorkerSetupManager +{ + private static final Logger log = new Logger(WorkerSetupManager.class); + + private final DBI dbi; + private final ObjectMapper jsonMapper; + private final ScheduledExecutorService exec; + private final WorkerSetupManagerConfig config; + + private final Object lock = new Object(); + + private volatile AtomicReference workerSetupData = new AtomicReference(null); + private volatile boolean started = false; + + public WorkerSetupManager( + DBI dbi, + ScheduledExecutorService exec, + ObjectMapper jsonMapper, + WorkerSetupManagerConfig config + ) + { + this.dbi = dbi; + this.exec = exec; + this.jsonMapper = jsonMapper; + this.config = config; + } + + @LifecycleStart + public void start() + { + synchronized (lock) { + if (started) { + return; + } + + ScheduledExecutors.scheduleWithFixedDelay( + exec, + new Duration(0), + config.getPollDuration(), + new Runnable() + { + @Override + public void run() + { + poll(); + } + } + ); + + started = true; + } + } + + @LifecycleStop + public void stop() + { + synchronized (lock) { + if (!started) { + return; + } + + started = false; + } + } + + public void poll() + { + try { + List setupDataList = dbi.withHandle( + new HandleCallback>() + { + @Override + public List withHandle(Handle handle) throws Exception + { + return handle.createQuery( + String.format( + "SELECT minVersion, minNumWorkers, nodeData, userData FROM %s", + config.getWorkerSetupTable() + ) + ).fold( + Lists.newArrayList(), + new Folder3, Map>() + { + @Override + public ArrayList fold( + ArrayList workerNodeConfigurations, + Map stringObjectMap, + FoldController foldController, + StatementContext statementContext + ) throws SQLException + { + try { + // stringObjectMap lowercases and jackson may fail serde + workerNodeConfigurations.add( + new WorkerSetupData( + MapUtils.getString(stringObjectMap, "minVersion"), + MapUtils.getInteger(stringObjectMap, "minNumWorkers"), + jsonMapper.readValue( + MapUtils.getString(stringObjectMap, "nodeData"), + WorkerNodeData.class + ), + jsonMapper.readValue( + MapUtils.getString(stringObjectMap, "userData"), + WorkerUserData.class + ) + ) + ); + return workerNodeConfigurations; + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + ); + } + } + ); + + if (setupDataList.isEmpty()) { + throw new ISE("WTF?! No configuration found for worker nodes!"); + } else if (setupDataList.size() != 1) { + throw new ISE("WTF?! Found more than one configuration for worker nodes"); + } + + workerSetupData.set(setupDataList.get(0)); + } + catch (Exception e) { + log.error(e, "Exception while polling for worker setup data!"); + } + } + + @SuppressWarnings("unchecked") + public WorkerSetupData getWorkerSetupData() + { + synchronized (lock) { + if (!started) { + throw new ISE("Must start WorkerSetupManager first!"); + } + + return workerSetupData.get(); + } + } + + public boolean setWorkerSetupData(final WorkerSetupData value) + { + synchronized (lock) { + try { + if (!started) { + throw new ISE("Must start WorkerSetupManager first!"); + } + + dbi.withHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws Exception + { + handle.createStatement(String.format("DELETE FROM %s", config.getWorkerSetupTable())).execute(); + handle.createStatement( + String.format( + "INSERT INTO %s (minVersion, minNumWorkers, nodeData, userData) VALUES (:minVersion, :minNumWorkers, :nodeData, :userData)", + config.getWorkerSetupTable() + ) + ) + .bind("minVersion", value.getMinVersion()) + .bind("minNumWorkers", value.getMinNumWorkers()) + .bind("nodeData", jsonMapper.writeValueAsString(value.getNodeData())) + .bind("userData", jsonMapper.writeValueAsString(value.getUserData())) + .execute(); + + return null; + } + } + ); + + workerSetupData.set(value); + } + catch (Exception e) { + log.error(e, "Exception updating worker config"); + return false; + } + } + + return true; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerUserData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerUserData.java new file mode 100644 index 00000000000..80857fb58d1 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerUserData.java @@ -0,0 +1,14 @@ +package com.metamx.druid.merger.coordinator.setup; + +import org.codehaus.jackson.annotate.JsonSubTypes; +import org.codehaus.jackson.annotate.JsonTypeInfo; + +/** + */ +@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="classType") +@JsonSubTypes(value={ + @JsonSubTypes.Type(name="galaxy", value=GalaxyUserData.class) +}) +public interface WorkerUserData +{ +} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index eb10731abd9..3282ca5dd3a 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -17,6 +17,8 @@ import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; import com.metamx.druid.merger.coordinator.scaling.AutoScalingData; import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.druid.merger.worker.TaskMonitor; import com.metamx.druid.merger.worker.Worker; import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; @@ -62,6 +64,7 @@ public class RemoteTaskRunnerTest private PathChildrenCache pathChildrenCache; private RemoteTaskRunner remoteTaskRunner; private TaskMonitor taskMonitor; + private WorkerSetupManager workerSetupManager; private ScheduledExecutorService scheduledExec; @@ -69,7 +72,6 @@ public class RemoteTaskRunnerTest private Worker worker1; - @Before public void setUp() throws Exception { @@ -141,9 +143,10 @@ public class RemoteTaskRunnerTest { remoteTaskRunner.run(task1, new TaskContext(new DateTime().toString(), Sets.newHashSet()), null); try { - remoteTaskRunner.run(task1, new TaskContext(new DateTime().toString(), Sets.newHashSet()), null); - fail("ISE expected"); - } catch (ISE expected) { + remoteTaskRunner.run(task1, new TaskContext(new DateTime().toString(), Sets.newHashSet()), null); + fail("ISE expected"); + } + catch (ISE expected) { } } @@ -333,6 +336,17 @@ public class RemoteTaskRunnerTest private void makeRemoteTaskRunner() throws Exception { scheduledExec = EasyMock.createMock(ScheduledExecutorService.class); + workerSetupManager = EasyMock.createMock(WorkerSetupManager.class); + + EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn( + new WorkerSetupData( + "0", + 0, + null, + null + ) + ); + EasyMock.replay(workerSetupManager); remoteTaskRunner = new RemoteTaskRunner( jsonMapper, @@ -341,7 +355,8 @@ public class RemoteTaskRunnerTest pathChildrenCache, scheduledExec, new RetryPolicyFactory(new TestRetryPolicyConfig()), - new TestScalingStrategy() + new TestScalingStrategy(), + workerSetupManager ); // Create a single worker and wait for things for be ready @@ -405,18 +420,6 @@ public class RemoteTaskRunnerTest return null; } - @Override - public String getMinWorkerVersion() - { - return "0"; - } - - @Override - public int getMinNumWorkers() - { - return 0; - } - @Override public int getMaxWorkerIdleTimeMillisBeforeDeletion() { diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java index 958a2c1d836..2cbdfe83efe 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java @@ -29,6 +29,10 @@ import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; +import com.metamx.druid.merger.coordinator.setup.EC2NodeData; +import com.metamx.druid.merger.coordinator.setup.GalaxyUserData; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; @@ -52,6 +56,7 @@ public class EC2AutoScalingStrategyTest private Reservation reservation; private Instance instance; private EC2AutoScalingStrategy strategy; + private WorkerSetupManager workerSetupManager; @Before public void setUp() throws Exception @@ -60,6 +65,7 @@ public class EC2AutoScalingStrategyTest runInstancesResult = EasyMock.createMock(RunInstancesResult.class); describeInstancesResult = EasyMock.createMock(DescribeInstancesResult.class); reservation = EasyMock.createMock(Reservation.class); + workerSetupManager = EasyMock.createMock(WorkerSetupManager.class); instance = new Instance() .withInstanceId(INSTANCE_ID) @@ -69,44 +75,16 @@ public class EC2AutoScalingStrategyTest strategy = new EC2AutoScalingStrategy( new DefaultObjectMapper(), - amazonEC2Client, new EC2AutoScalingStrategyConfig() - { - @Override - public String getAmiId() - { - return AMI_ID; - } - - @Override - public String getWorkerPort() - { - return "8080"; - } - - @Override - public String getInstanceType() - { - return "t1.micro"; - } - - @Override - public int getMinNumInstancesToProvision() - { - return 1; - } - - @Override - public int getMaxNumInstancesToProvision() - { - return 1; - } - - @Override - public String getUserDataFile() - { - return ""; - } - } + amazonEC2Client, + new EC2AutoScalingStrategyConfig() + { + @Override + public String getWorkerPort() + { + return "8080"; + } + }, + workerSetupManager ); } @@ -117,11 +95,22 @@ public class EC2AutoScalingStrategyTest EasyMock.verify(runInstancesResult); EasyMock.verify(describeInstancesResult); EasyMock.verify(reservation); + EasyMock.verify(workerSetupManager); } @Test public void testScale() { + EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn( + new WorkerSetupData( + "0", + 0, + new EC2NodeData(AMI_ID, INSTANCE_ID, 1, 1), + new GalaxyUserData("env", "ver", "type") + ) + ); + EasyMock.replay(workerSetupManager); + EasyMock.expect(amazonEC2Client.runInstances(EasyMock.anyObject(RunInstancesRequest.class))).andReturn( runInstancesResult ); From 79e9ff94a67753dfb0619e040d7965b1dba1fa27 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 7 Jan 2013 14:07:47 -0800 Subject: [PATCH 15/92] add missing headers --- .../config/WorkerSetupManagerConfig.java | 19 +++++++++++++++++++ .../coordinator/scaling/AutoScalingData.java | 19 +++++++++++++++++++ .../scaling/NoopScalingStrategy.java | 19 +++++++++++++++++++ .../merger/coordinator/setup/EC2NodeData.java | 19 +++++++++++++++++++ .../coordinator/setup/GalaxyUserData.java | 19 +++++++++++++++++++ .../coordinator/setup/WorkerNodeData.java | 19 +++++++++++++++++++ .../coordinator/setup/WorkerSetupData.java | 19 +++++++++++++++++++ .../coordinator/setup/WorkerSetupManager.java | 19 +++++++++++++++++++ .../coordinator/setup/WorkerUserData.java | 19 +++++++++++++++++++ 9 files changed, 171 insertions(+) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java index ad7444b657e..97368c9f77e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.merger.coordinator.config; import org.joda.time.Duration; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java index 6cce08f8731..5a1bb4980e5 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.merger.coordinator.scaling; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java index 67eb99293e4..d723df537dc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.merger.coordinator.scaling; import com.metamx.emitter.EmittingLogger; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java index 209444c6731..d1531f7974a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.merger.coordinator.setup; import org.codehaus.jackson.annotate.JsonCreator; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java index 0baa0ddfb9a..21e09f5fd40 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.merger.coordinator.setup; import org.codehaus.jackson.annotate.JsonCreator; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerNodeData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerNodeData.java index 8068a4c267b..b7cec6d68d0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerNodeData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerNodeData.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.merger.coordinator.setup; import org.codehaus.jackson.annotate.JsonSubTypes; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java index 224b22167ae..f54818f8437 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.merger.coordinator.setup; import org.codehaus.jackson.annotate.JsonCreator; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java index a4ab8fe8d5d..03bdd770671 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.merger.coordinator.setup; import com.google.common.base.Throwables; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerUserData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerUserData.java index 80857fb58d1..4a42c9b3bac 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerUserData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerUserData.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.merger.coordinator.setup; import org.codehaus.jackson.annotate.JsonSubTypes; From 700ce241067ee0c5ad56587a12218296bf2750cf Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 7 Jan 2013 16:52:57 -0800 Subject: [PATCH 16/92] fix JVM optimizing out the benchmarking code --- .../druid/client/cache/MemcachedCacheBrokerBenchmark.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java index 2fa7d3b4193..d1839098e8b 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java @@ -56,7 +56,7 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark broker = new MemcachedCacheBroker( client, 500, // 500 milliseconds - 3600 * 24 * 365 // 1 year + 3600 // 1 hour ); cache = broker.provideCache("default"); @@ -84,15 +84,17 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark } } - public byte[] timeGetObject(int reps) { + public long timeGetObject(int reps) { byte[] bytes = null; + long count = 0; for (int i = 0; i < reps; i++) { for(int k = 0; k < objectCount; ++k) { String key = BASE_KEY + i; bytes = cache.get(key.getBytes()); + count += bytes.length; } } - return bytes; + return count; } public static void main(String[] args) throws Exception { From 4c2da933895477110d50a23bcfcb858fe2b13776 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 9 Jan 2013 14:51:35 -0800 Subject: [PATCH 17/92] bug fixes with ec2 auto scaling --- .../java/com/metamx/druid/db/DbConnector.java | 2 +- .../merger/coordinator/RemoteTaskRunner.java | 23 ++++---- .../http/IndexerCoordinatorNode.java | 6 +- .../scaling/EC2AutoScalingStrategy.java | 57 +++++++++++++------ .../scaling/NoopScalingStrategy.java | 7 +++ .../coordinator/scaling/ScalingStrategy.java | 7 +++ .../merger/coordinator/setup/EC2NodeData.java | 22 ++++++- .../coordinator/setup/GalaxyUserData.java | 10 ++-- .../coordinator/setup/WorkerSetupData.java | 22 ++++++- .../coordinator/setup/WorkerSetupManager.java | 16 +++++- .../coordinator/RemoteTaskRunnerTest.java | 8 +++ .../scaling/EC2AutoScalingStrategyTest.java | 9 ++- pom.xml | 2 +- 13 files changed, 146 insertions(+), 45 deletions(-) diff --git a/common/src/main/java/com/metamx/druid/db/DbConnector.java b/common/src/main/java/com/metamx/druid/db/DbConnector.java index 75cf0ba27e7..b918af2b2b5 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnector.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnector.java @@ -65,7 +65,7 @@ public class DbConnector dbi, workerTableName, String.format( - "CREATE table %s (minVersion TINYTEXT NOT NULL, minNumWorkers SMALLINT NOT NULL, nodeData LONGTEXT NOT NULL, userData LONGTEXT NOT NULL)", + "CREATE table %s (minVersion TINYTEXT NOT NULL, minNumWorkers SMALLINT NOT NULL, nodeData LONGTEXT NOT NULL, userData LONGTEXT NOT NULL, securityGroupIds LONGTEXT NOT NULL, keyName TINYTEXT NOT NULL)", workerTableName ) ); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index addb789762f..0e8927cb0b9 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -54,6 +54,7 @@ import org.joda.time.Duration; import org.joda.time.Period; import javax.annotation.Nullable; +import java.util.Arrays; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -148,7 +149,7 @@ public class RemoteTaskRunner implements TaskRunner Worker.class ); log.info("Worker[%s] removed!", worker.getHost()); - removeWorker(worker.getHost()); + removeWorker(worker); } } } @@ -222,7 +223,7 @@ public class RemoteTaskRunner implements TaskRunner } log.info( - "[%s] still terminating. Wait for all nodes to terminate before trying again.", + "%s still terminating. Wait for all nodes to terminate before trying again.", currentlyTerminating ); } @@ -372,7 +373,7 @@ public class RemoteTaskRunner implements TaskRunner private void addWorker(final Worker worker) { try { - currentlyProvisioning.remove(worker.getHost()); + currentlyProvisioning.removeAll(strategy.ipLookup(Arrays.asList(worker.getIp()))); final String workerStatusPath = JOINER.join(config.getStatusPath(), worker.getHost()); final PathChildrenCache statusCache = new PathChildrenCache(cf, workerStatusPath, true); @@ -483,22 +484,22 @@ public class RemoteTaskRunner implements TaskRunner * When a ephemeral worker node disappears from ZK, we have to make sure there are no tasks still assigned * to the worker. If tasks remain, they are retried. * - * @param workerId - id of the removed worker + * @param worker - the removed worker */ - private void removeWorker(final String workerId) + private void removeWorker(final Worker worker) { - currentlyTerminating.remove(workerId); + currentlyTerminating.remove(worker.getHost()); - WorkerWrapper workerWrapper = zkWorkers.get(workerId); + WorkerWrapper workerWrapper = zkWorkers.get(worker.getHost()); if (workerWrapper != null) { try { Set tasksToRetry = Sets.newHashSet(workerWrapper.getRunningTasks()); - tasksToRetry.addAll(cf.getChildren().forPath(JOINER.join(config.getTaskPath(), workerId))); + tasksToRetry.addAll(cf.getChildren().forPath(JOINER.join(config.getTaskPath(), worker.getHost()))); for (String taskId : tasksToRetry) { TaskWrapper taskWrapper = tasks.get(taskId); if (taskWrapper != null) { - retryTask(new CleanupPaths(workerId, taskId), tasks.get(taskId)); + retryTask(new CleanupPaths(worker.getHost(), taskId), tasks.get(taskId)); } } @@ -508,7 +509,7 @@ public class RemoteTaskRunner implements TaskRunner log.error(e, "Failed to cleanly remove worker[%s]"); } } - zkWorkers.remove(workerId); + zkWorkers.remove(worker.getHost()); } private WorkerWrapper findWorkerForTask() @@ -558,7 +559,7 @@ public class RemoteTaskRunner implements TaskRunner } log.info( - "[%s] still provisioning. Wait for all provisioned nodes to complete before requesting new worker.", + "%s still provisioning. Wait for all provisioned nodes to complete before requesting new worker.", currentlyProvisioning ); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 5248b30712e..15d76cf2fec 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -47,6 +47,9 @@ import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.loading.S3SegmentPusher; +import com.metamx.druid.loading.S3SegmentPusherConfig; +import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -70,9 +73,6 @@ import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig; import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.NoopScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; -import com.metamx.druid.loading.S3SegmentPusher; -import com.metamx.druid.loading.S3SegmentPusherConfig; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index eed69ae7f9e..35f9d7d25cb 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -36,6 +36,7 @@ import com.metamx.druid.merger.coordinator.setup.EC2NodeData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.EmittingLogger; +import org.apache.commons.codec.binary.Base64; import org.codehaus.jackson.map.ObjectMapper; import java.util.List; @@ -82,7 +83,15 @@ public class EC2AutoScalingStrategy implements ScalingStrategy workerConfig.getMaxInstances() ) .withInstanceType(workerConfig.getInstanceType()) - .withUserData(jsonMapper.writeValueAsString(setupData.getUserData())) + .withSecurityGroupIds(workerConfig.getSecurityGroupIds()) + .withKeyName(workerConfig.getKeyName()) + .withUserData( + Base64.encodeBase64String( + jsonMapper.writeValueAsBytes( + setupData.getUserData() + ) + ) + ) ); List instanceIds = Lists.transform( @@ -107,7 +116,7 @@ public class EC2AutoScalingStrategy implements ScalingStrategy @Override public String apply(Instance input) { - return String.format("%s:%s", input.getPrivateIpAddress(), config.getWorkerPort()); + return input.getInstanceId(); } } ), @@ -127,7 +136,7 @@ public class EC2AutoScalingStrategy implements ScalingStrategy DescribeInstancesResult result = amazonEC2Client.describeInstances( new DescribeInstancesRequest() .withFilters( - new Filter("private-ip-address", nodeIds) + new Filter("instance-id", nodeIds) ) ); @@ -139,19 +148,7 @@ public class EC2AutoScalingStrategy implements ScalingStrategy try { log.info("Terminating instance[%s]", instances); amazonEC2Client.terminateInstances( - new TerminateInstancesRequest( - Lists.transform( - instances, - new Function() - { - @Override - public String apply(Instance input) - { - return input.getInstanceId(); - } - } - ) - ) + new TerminateInstancesRequest(nodeIds) ); return new AutoScalingData( @@ -175,4 +172,32 @@ public class EC2AutoScalingStrategy implements ScalingStrategy return null; } + + @Override + public List ipLookup(List ips) + { + DescribeInstancesResult result = amazonEC2Client.describeInstances( + new DescribeInstancesRequest() + .withFilters( + new Filter("private-ip-address", ips) + ) + ); + + List instances = Lists.newArrayList(); + for (Reservation reservation : result.getReservations()) { + instances.addAll(reservation.getInstances()); + } + + return Lists.transform( + instances, + new Function() + { + @Override + public String apply(Instance input) + { + return input.getInstanceId(); + } + } + ); + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java index d723df537dc..2b412ca6202 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java @@ -43,4 +43,11 @@ public class NoopScalingStrategy implements ScalingStrategy log.info("If I were a real strategy I'd terminate %s now", nodeIds); return null; } + + @Override + public List ipLookup(List ips) + { + log.info("I'm not a real strategy so I'm returning what I got %s", ips); + return ips; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java index 9b7da8fb3a4..52105451ed3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java @@ -28,4 +28,11 @@ public interface ScalingStrategy public AutoScalingData provision(); public AutoScalingData terminate(List nodeIds); + + /** + * Provides a lookup of ip addresses to node ids + * @param ips + * @return + */ + public List ipLookup(List ips); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java index d1531f7974a..3e986f94d56 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java @@ -22,6 +22,8 @@ package com.metamx.druid.merger.coordinator.setup; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; +import java.util.List; + /** */ public class EC2NodeData implements WorkerNodeData @@ -30,19 +32,25 @@ public class EC2NodeData implements WorkerNodeData private final String instanceType; private final int minInstances; private final int maxInstances; + private final List securityGroupIds; + private final String keyName; @JsonCreator public EC2NodeData( @JsonProperty("amiId") String amiId, @JsonProperty("instanceType") String instanceType, @JsonProperty("minInstances") int minInstances, - @JsonProperty("maxInstances") int maxInstances + @JsonProperty("maxInstances") int maxInstances, + @JsonProperty("securityGroupIds") List securityGroupIds, + @JsonProperty("keyName") String keyName ) { this.amiId = amiId; this.instanceType = instanceType; this.minInstances = minInstances; this.maxInstances = maxInstances; + this.securityGroupIds = securityGroupIds; + this.keyName = keyName; } @JsonProperty @@ -68,4 +76,16 @@ public class EC2NodeData implements WorkerNodeData { return maxInstances; } + + @JsonProperty + public List getSecurityGroupIds() + { + return securityGroupIds; + } + + @JsonProperty + public String getKeyName() + { + return keyName; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java index 21e09f5fd40..046a8b55f32 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java @@ -27,18 +27,18 @@ import org.codehaus.jackson.annotate.JsonProperty; public class GalaxyUserData implements WorkerUserData { public final String env; - public final String ver; + public final String version; public final String type; @JsonCreator public GalaxyUserData( @JsonProperty("env") String env, - @JsonProperty("ver") String ver, + @JsonProperty("version") String version, @JsonProperty("type") String type ) { this.env = env; - this.ver = ver; + this.version = version; this.type = type; } @@ -49,9 +49,9 @@ public class GalaxyUserData implements WorkerUserData } @JsonProperty - public String getVer() + public String getVersion() { - return ver; + return version; } @JsonProperty diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java index f54818f8437..d0173ef06b8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java @@ -22,6 +22,8 @@ package com.metamx.druid.merger.coordinator.setup; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; +import java.util.List; + /** */ public class WorkerSetupData @@ -30,19 +32,25 @@ public class WorkerSetupData private final int minNumWorkers; private final WorkerNodeData nodeData; private final WorkerUserData userData; + private final List securityGroupIds; + private final String keyName; @JsonCreator public WorkerSetupData( @JsonProperty("minVersion") String minVersion, @JsonProperty("minNumWorkers") int minNumWorkers, @JsonProperty("nodeData") WorkerNodeData nodeData, - @JsonProperty("userData") WorkerUserData userData + @JsonProperty("userData") WorkerUserData userData, + @JsonProperty("securityGroupIds") List securityGroupIds, + @JsonProperty("keyName") String keyName ) { this.minVersion = minVersion; this.minNumWorkers = minNumWorkers; this.nodeData = nodeData; this.userData = userData; + this.securityGroupIds = securityGroupIds; + this.keyName = keyName; } @JsonProperty @@ -68,4 +76,16 @@ public class WorkerSetupData { return userData; } + + @JsonProperty + public List getSecurityGroupIds() + { + return securityGroupIds; + } + + @JsonProperty + public String getKeyName() + { + return keyName; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java index 03bdd770671..42f1a880eda 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java @@ -29,6 +29,7 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig; import org.apache.commons.collections.MapUtils; import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.type.TypeReference; import org.joda.time.Duration; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.FoldController; @@ -122,7 +123,7 @@ public class WorkerSetupManager { return handle.createQuery( String.format( - "SELECT minVersion, minNumWorkers, nodeData, userData FROM %s", + "SELECT minVersion, minNumWorkers, nodeData, userData, securityGroupIds, keyName FROM %s", config.getWorkerSetupTable() ) ).fold( @@ -150,7 +151,14 @@ public class WorkerSetupManager jsonMapper.readValue( MapUtils.getString(stringObjectMap, "userData"), WorkerUserData.class - ) + ), + (List) jsonMapper.readValue( + MapUtils.getString(stringObjectMap, "securityGroupIds"), + new TypeReference>() + { + } + ), + MapUtils.getString(stringObjectMap, "keyName") ) ); return workerNodeConfigurations; @@ -207,7 +215,7 @@ public class WorkerSetupManager handle.createStatement(String.format("DELETE FROM %s", config.getWorkerSetupTable())).execute(); handle.createStatement( String.format( - "INSERT INTO %s (minVersion, minNumWorkers, nodeData, userData) VALUES (:minVersion, :minNumWorkers, :nodeData, :userData)", + "INSERT INTO %s (minVersion, minNumWorkers, nodeData, userData, securityGroupIds, keyName) VALUES (:minVersion, :minNumWorkers, :nodeData, :userData, :securityGroupIds, :keyName)", config.getWorkerSetupTable() ) ) @@ -215,6 +223,8 @@ public class WorkerSetupManager .bind("minNumWorkers", value.getMinNumWorkers()) .bind("nodeData", jsonMapper.writeValueAsString(value.getNodeData())) .bind("userData", jsonMapper.writeValueAsString(value.getUserData())) + .bind("securityGroupIds", jsonMapper.writeValueAsString(value.getSecurityGroupIds())) + .bind("keyName", jsonMapper.writeValueAsString(value.getKeyName())) .execute(); return null; diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index 3282ca5dd3a..edf3499ff74 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -343,6 +343,8 @@ public class RemoteTaskRunnerTest "0", 0, null, + null, + Lists.newArrayList(), null ) ); @@ -404,6 +406,12 @@ public class RemoteTaskRunnerTest { return null; } + + @Override + public List ipLookup(List ips) + { + return ips; + } } private static class TestRemoteTaskRunnerConfig extends RemoteTaskRunnerConfig diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java index 2cbdfe83efe..2660a22952e 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java @@ -27,6 +27,7 @@ import com.amazonaws.services.ec2.model.Reservation; import com.amazonaws.services.ec2.model.RunInstancesRequest; import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; +import com.google.common.collect.Lists; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.druid.merger.coordinator.setup.EC2NodeData; @@ -105,8 +106,10 @@ public class EC2AutoScalingStrategyTest new WorkerSetupData( "0", 0, - new EC2NodeData(AMI_ID, INSTANCE_ID, 1, 1), - new GalaxyUserData("env", "ver", "type") + new EC2NodeData(AMI_ID, INSTANCE_ID, 1, 1, Lists.newArrayList(), "foo"), + new GalaxyUserData("env", "version", "type"), + Arrays.asList("foo"), + "foo2" ) ); EasyMock.replay(workerSetupManager); @@ -133,7 +136,7 @@ public class EC2AutoScalingStrategyTest Assert.assertEquals(created.getNodeIds().size(), 1); Assert.assertEquals(created.getNodes().size(), 1); - Assert.assertEquals(String.format("%s:8080", IP), created.getNodeIds().get(0)); + Assert.assertEquals("theInstance", created.getNodeIds().get(0)); AutoScalingData deleted = strategy.terminate(Arrays.asList("dummyHost")); diff --git a/pom.xml b/pom.xml index 5a165a16ba2..898fc197b93 100644 --- a/pom.xml +++ b/pom.xml @@ -84,7 +84,7 @@ commons-codec commons-codec - 1.3 + 1.7 commons-httpclient From 3de62b34b643837a05e33084cc90f121b304748d Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Mon, 14 Jan 2013 15:00:28 -0600 Subject: [PATCH 18/92] 1) Should be on 0.2.0 with the recent API changes for StorageAdapters --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5a165a16ba2..20ca0f1bada 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.1.26-SNAPSHOT + 0.2.0-SNAPSHOT druid druid From d1f4317af770e854c56ccaf46d639d6aca2fdeba Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 14 Jan 2013 14:55:04 -0800 Subject: [PATCH 19/92] updates to autoscaling config based on code review comments --- .../java/com/metamx/druid/db/DbConnector.java | 2 +- .../merger/coordinator/RemoteTaskRunner.java | 7 ++-- .../config/RemoteTaskRunnerConfig.java | 2 +- .../scaling/EC2AutoScalingStrategy.java | 39 ++++++++++++------- .../coordinator/scaling/ScalingStrategy.java | 2 +- .../merger/coordinator/setup/EC2NodeData.java | 2 +- .../coordinator/setup/GalaxyUserData.java | 2 +- .../coordinator/setup/WorkerNodeData.java | 33 ---------------- .../coordinator/setup/WorkerSetupData.java | 30 +++----------- .../coordinator/setup/WorkerSetupManager.java | 32 +++------------ .../coordinator/setup/WorkerUserData.java | 33 ---------------- .../merger/worker/config/WorkerConfig.java | 3 +- .../coordinator/RemoteTaskRunnerTest.java | 2 - .../scaling/EC2AutoScalingStrategyTest.java | 6 +-- 14 files changed, 50 insertions(+), 145 deletions(-) delete mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerNodeData.java delete mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerUserData.java diff --git a/common/src/main/java/com/metamx/druid/db/DbConnector.java b/common/src/main/java/com/metamx/druid/db/DbConnector.java index b918af2b2b5..45a0b937964 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnector.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnector.java @@ -65,7 +65,7 @@ public class DbConnector dbi, workerTableName, String.format( - "CREATE table %s (minVersion TINYTEXT NOT NULL, minNumWorkers SMALLINT NOT NULL, nodeData LONGTEXT NOT NULL, userData LONGTEXT NOT NULL, securityGroupIds LONGTEXT NOT NULL, keyName TINYTEXT NOT NULL)", + "CREATE table %s (config LONGTEXT NOT NULL)", workerTableName ) ); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 0e8927cb0b9..5537a6b6420 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -53,7 +53,6 @@ import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; -import javax.annotation.Nullable; import java.util.Arrays; import java.util.Comparator; import java.util.List; @@ -185,7 +184,7 @@ public class RemoteTaskRunner implements TaskRunner new Predicate() { @Override - public boolean apply(@Nullable WorkerWrapper input) + public boolean apply(WorkerWrapper input) { return input.getRunningTasks().isEmpty() && System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() @@ -201,9 +200,9 @@ public class RemoteTaskRunner implements TaskRunner new Function() { @Override - public String apply(@Nullable WorkerWrapper input) + public String apply(WorkerWrapper input) { - return input.getWorker().getHost(); + return input.getWorker().getIp(); } } ) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java index 2e20c4ffff2..c9badf7ef88 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java @@ -38,7 +38,7 @@ public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig public abstract DateTime getTerminateResourcesOriginDateTime(); @Config("druid.indexer.maxWorkerIdleTimeMillisBeforeDeletion") - @Default("1") + @Default("10000") public abstract int getMaxWorkerIdleTimeMillisBeforeDeletion(); @Config("druid.indexer.maxScalingDuration") diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index 35f9d7d25cb..8d51da61afd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -30,7 +30,6 @@ import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; import com.google.common.base.Function; import com.google.common.collect.Lists; -import com.metamx.common.ISE; import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.druid.merger.coordinator.setup.EC2NodeData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; @@ -39,6 +38,7 @@ import com.metamx.emitter.EmittingLogger; import org.apache.commons.codec.binary.Base64; import org.codehaus.jackson.map.ObjectMapper; +import javax.annotation.Nullable; import java.util.List; /** @@ -70,10 +70,7 @@ public class EC2AutoScalingStrategy implements ScalingStrategy { try { WorkerSetupData setupData = workerSetupManager.getWorkerSetupData(); - if (!(setupData.getNodeData() instanceof EC2NodeData)) { - throw new ISE("DB misconfiguration! Node data is an instance of [%s]", setupData.getNodeData().getClass()); - } - EC2NodeData workerConfig = (EC2NodeData) setupData.getNodeData(); + EC2NodeData workerConfig = setupData.getNodeData(); log.info("Creating new instance(s)..."); RunInstancesResult result = amazonEC2Client.runInstances( @@ -131,12 +128,12 @@ public class EC2AutoScalingStrategy implements ScalingStrategy } @Override - public AutoScalingData terminate(List nodeIds) + public AutoScalingData terminate(List ids) { DescribeInstancesResult result = amazonEC2Client.describeInstances( new DescribeInstancesRequest() .withFilters( - new Filter("instance-id", nodeIds) + new Filter("private-ip-address", ids) ) ); @@ -148,18 +145,30 @@ public class EC2AutoScalingStrategy implements ScalingStrategy try { log.info("Terminating instance[%s]", instances); amazonEC2Client.terminateInstances( - new TerminateInstancesRequest(nodeIds) + new TerminateInstancesRequest( + Lists.transform( + instances, + new Function() + { + @Override + public String apply(Instance input) + { + return input.getInstanceId(); + } + } + ) + ) ); return new AutoScalingData( Lists.transform( - instances, - new Function() + ids, + new Function() { @Override - public String apply(Instance input) + public String apply(@Nullable String input) { - return String.format("%s:%s", input.getPrivateIpAddress(), config.getWorkerPort()); + return String.format("%s:%s", input, config.getWorkerPort()); } } ), @@ -188,7 +197,7 @@ public class EC2AutoScalingStrategy implements ScalingStrategy instances.addAll(reservation.getInstances()); } - return Lists.transform( + List retVal = Lists.transform( instances, new Function() { @@ -199,5 +208,9 @@ public class EC2AutoScalingStrategy implements ScalingStrategy } } ); + + log.info("Performing lookup: %s --> %s", ips, retVal); + + return retVal; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java index 52105451ed3..150de1357e0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java @@ -27,7 +27,7 @@ public interface ScalingStrategy { public AutoScalingData provision(); - public AutoScalingData terminate(List nodeIds); + public AutoScalingData terminate(List ids); /** * Provides a lookup of ip addresses to node ids diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java index 3e986f94d56..8d302df25f6 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java @@ -26,7 +26,7 @@ import java.util.List; /** */ -public class EC2NodeData implements WorkerNodeData +public class EC2NodeData { private final String amiId; private final String instanceType; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java index 046a8b55f32..876a2635273 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java @@ -24,7 +24,7 @@ import org.codehaus.jackson.annotate.JsonProperty; /** */ -public class GalaxyUserData implements WorkerUserData +public class GalaxyUserData { public final String env; public final String version; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerNodeData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerNodeData.java deleted file mode 100644 index b7cec6d68d0..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerNodeData.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.merger.coordinator.setup; - -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; - -/** - */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type") -@JsonSubTypes(value={ - @JsonSubTypes.Type(name="ec2", value=EC2NodeData.class) -}) -public interface WorkerNodeData -{ -} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java index d0173ef06b8..8395fa2d6c8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java @@ -30,27 +30,21 @@ public class WorkerSetupData { private final String minVersion; private final int minNumWorkers; - private final WorkerNodeData nodeData; - private final WorkerUserData userData; - private final List securityGroupIds; - private final String keyName; + private final EC2NodeData nodeData; + private final GalaxyUserData userData; @JsonCreator public WorkerSetupData( @JsonProperty("minVersion") String minVersion, @JsonProperty("minNumWorkers") int minNumWorkers, - @JsonProperty("nodeData") WorkerNodeData nodeData, - @JsonProperty("userData") WorkerUserData userData, - @JsonProperty("securityGroupIds") List securityGroupIds, - @JsonProperty("keyName") String keyName + @JsonProperty("nodeData") EC2NodeData nodeData, + @JsonProperty("userData") GalaxyUserData userData ) { this.minVersion = minVersion; this.minNumWorkers = minNumWorkers; this.nodeData = nodeData; this.userData = userData; - this.securityGroupIds = securityGroupIds; - this.keyName = keyName; } @JsonProperty @@ -66,26 +60,14 @@ public class WorkerSetupData } @JsonProperty - public WorkerNodeData getNodeData() + public EC2NodeData getNodeData() { return nodeData; } @JsonProperty - public WorkerUserData getUserData() + public GalaxyUserData getUserData() { return userData; } - - @JsonProperty - public List getSecurityGroupIds() - { - return securityGroupIds; - } - - @JsonProperty - public String getKeyName() - { - return keyName; - } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java index 42f1a880eda..baa56290af7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java @@ -123,7 +123,7 @@ public class WorkerSetupManager { return handle.createQuery( String.format( - "SELECT minVersion, minNumWorkers, nodeData, userData, securityGroupIds, keyName FROM %s", + "SELECT config FROM %s", config.getWorkerSetupTable() ) ).fold( @@ -141,24 +141,9 @@ public class WorkerSetupManager try { // stringObjectMap lowercases and jackson may fail serde workerNodeConfigurations.add( - new WorkerSetupData( - MapUtils.getString(stringObjectMap, "minVersion"), - MapUtils.getInteger(stringObjectMap, "minNumWorkers"), - jsonMapper.readValue( - MapUtils.getString(stringObjectMap, "nodeData"), - WorkerNodeData.class - ), - jsonMapper.readValue( - MapUtils.getString(stringObjectMap, "userData"), - WorkerUserData.class - ), - (List) jsonMapper.readValue( - MapUtils.getString(stringObjectMap, "securityGroupIds"), - new TypeReference>() - { - } - ), - MapUtils.getString(stringObjectMap, "keyName") + jsonMapper.readValue( + MapUtils.getString(stringObjectMap, "config"), + WorkerSetupData.class ) ); return workerNodeConfigurations; @@ -215,16 +200,11 @@ public class WorkerSetupManager handle.createStatement(String.format("DELETE FROM %s", config.getWorkerSetupTable())).execute(); handle.createStatement( String.format( - "INSERT INTO %s (minVersion, minNumWorkers, nodeData, userData, securityGroupIds, keyName) VALUES (:minVersion, :minNumWorkers, :nodeData, :userData, :securityGroupIds, :keyName)", + "INSERT INTO %s (config) VALUES (:config)", config.getWorkerSetupTable() ) ) - .bind("minVersion", value.getMinVersion()) - .bind("minNumWorkers", value.getMinNumWorkers()) - .bind("nodeData", jsonMapper.writeValueAsString(value.getNodeData())) - .bind("userData", jsonMapper.writeValueAsString(value.getUserData())) - .bind("securityGroupIds", jsonMapper.writeValueAsString(value.getSecurityGroupIds())) - .bind("keyName", jsonMapper.writeValueAsString(value.getKeyName())) + .bind("config", jsonMapper.writeValueAsString(value)) .execute(); return null; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerUserData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerUserData.java deleted file mode 100644 index 4a42c9b3bac..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerUserData.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.merger.coordinator.setup; - -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; - -/** - */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="classType") -@JsonSubTypes(value={ - @JsonSubTypes.Type(name="galaxy", value=GalaxyUserData.class) -}) -public interface WorkerUserData -{ -} diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java index 5b5f3a0a6e7..4689acef261 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java @@ -41,6 +41,7 @@ public abstract class WorkerConfig public int getCapacity() { - return Runtime.getRuntime().availableProcessors() - 1; + return 1; + //return Runtime.getRuntime().availableProcessors() - 1; } } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index edf3499ff74..1e31efa121c 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -343,8 +343,6 @@ public class RemoteTaskRunnerTest "0", 0, null, - null, - Lists.newArrayList(), null ) ); diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java index 2660a22952e..c3aa8378b07 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java @@ -107,9 +107,7 @@ public class EC2AutoScalingStrategyTest "0", 0, new EC2NodeData(AMI_ID, INSTANCE_ID, 1, 1, Lists.newArrayList(), "foo"), - new GalaxyUserData("env", "version", "type"), - Arrays.asList("foo"), - "foo2" + new GalaxyUserData("env", "version", "type") ) ); EasyMock.replay(workerSetupManager); @@ -138,7 +136,7 @@ public class EC2AutoScalingStrategyTest Assert.assertEquals(created.getNodes().size(), 1); Assert.assertEquals("theInstance", created.getNodeIds().get(0)); - AutoScalingData deleted = strategy.terminate(Arrays.asList("dummyHost")); + AutoScalingData deleted = strategy.terminate(Arrays.asList("dummyIP")); Assert.assertEquals(deleted.getNodeIds().size(), 1); Assert.assertEquals(deleted.getNodes().size(), 1); From 998f0bf3c8d1bb0b262592ea99f3c7be29d8d8c1 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Mon, 14 Jan 2013 18:00:51 -0600 Subject: [PATCH 20/92] 1) Better error message when aggregators want more memory than is available. --- .../druid/query/group/GroupByQueryEngine.java | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java index b8d13be6674..88cb5840950 100644 --- a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java @@ -236,6 +236,11 @@ public class GroupByQueryEngine } } + public int getIncrement() + { + return increment; + } + public int[] getIncrements() { return increments; @@ -301,11 +306,11 @@ public class GroupByQueryEngine return delegate.next(); } - if (cursor.isDone()) { + if (unprocessedKeys == null && cursor.isDone()) { throw new NoSuchElementException(); } - final PositionMaintainer positionMaintainer = new PositionMaintainer(0, sizesRequired, metricsBuffer.limit()); + final PositionMaintainer positionMaintainer = new PositionMaintainer(0, sizesRequired, metricsBuffer.remaining()); final RowUpdater rowUpdater = new RowUpdater(metricsBuffer, aggregators, positionMaintainer); if (unprocessedKeys != null) { for (ByteBuffer key : unprocessedKeys) { @@ -327,6 +332,13 @@ public class GroupByQueryEngine cursor.advance(); } + if (rowUpdater.getPositions().isEmpty() && unprocessedKeys != null) { + throw new ISE( + "Not enough memory to process even a single item. Required [%,d] memory, but only have[%,d]", + positionMaintainer.getIncrement(), metricsBuffer.remaining() + ); + } + delegate = FunctionalIterator .create(rowUpdater.getPositions().entrySet().iterator()) .transform( From 7e074e8158e629a5ff6a11da4d35e51985fabb8f Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Tue, 15 Jan 2013 12:04:12 -0800 Subject: [PATCH 21/92] fix pom breakage --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index e737c55bc20..9917a339787 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.26-SNAPSHOT + 0.2.0-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index ec21eb37008..b4b900311dd 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.26-SNAPSHOT + 0.2.0-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 81af25085e9..d514c32514d 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.1.26-SNAPSHOT + 0.2.0-SNAPSHOT com.metamx druid - 0.1.26-SNAPSHOT + 0.2.0-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index d1aae11058e..350660500d1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.26-SNAPSHOT + 0.2.0-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index ae6b1282605..0e186d40b83 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.1.26-SNAPSHOT + 0.2.0-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 40420b8b2f3..f5a8cc28141 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.1.26-SNAPSHOT + 0.2.0-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index c5ca45c1002..a270b9cca22 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.26-SNAPSHOT + 0.2.0-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 3ab32ed0b90..433aea2dc13 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.26-SNAPSHOT + 0.2.0-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 1ea3478f7b9..64beb8d6b84 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.26-SNAPSHOT + 0.2.0-SNAPSHOT diff --git a/realtime/pom.xml b/realtime/pom.xml index 502f9c8b182..5cf54f122d2 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.26-SNAPSHOT + 0.2.0-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 7db3faba232..354aecaee60 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.1.26-SNAPSHOT + 0.2.0-SNAPSHOT From 86277d111424326d06dd120ac5e9665adb73b7cc Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 15 Jan 2013 22:14:20 -0800 Subject: [PATCH 22/92] StringInputRowParser: Treat dimensionExclusions case-insensitively --- .../druid/indexer/data/StringInputRowParser.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java b/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java index 4c5e2dcd9bb..3721f2c14f7 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java @@ -19,6 +19,7 @@ package com.metamx.druid.indexer.data; +import com.google.common.base.Function; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.exception.FormattedException; @@ -56,7 +57,18 @@ public class StringInputRowParser this.dimensionExclusions = Sets.newHashSet(); if (dimensionExclusions != null) { - this.dimensionExclusions.addAll(dimensionExclusions); + this.dimensionExclusions.addAll( + Lists.transform( + dimensionExclusions, new Function() + { + @Override + public String apply(String s) + { + return s.toLowerCase(); + } + } + ) + ); } this.dimensionExclusions.add(timestampSpec.getTimestampColumn()); From 616415cb7ebae44772aba93e17e2a2dc0a1fea5f Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 15 Jan 2013 22:29:03 -0800 Subject: [PATCH 23/92] UniformGranularitySpec: Only return bucketInterval for timestamps that legitimately overlap our input intervals --- .../granularity/UniformGranularitySpec.java | 35 ++++++++++--------- .../granularity/ArbitraryGranularityTest.java | 6 ++++ .../granularity/UniformGranularityTest.java | 6 ++++ 3 files changed, 31 insertions(+), 16 deletions(-) diff --git a/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java b/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java index a1039caba1c..51d2f37d437 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java @@ -20,6 +20,9 @@ package com.metamx.druid.indexer.granularity; import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.Granularity; import com.metamx.common.guava.Comparators; @@ -35,47 +38,47 @@ import java.util.TreeSet; public class UniformGranularitySpec implements GranularitySpec { final private Granularity granularity; - final private List intervals; + final private List inputIntervals; + final private ArbitraryGranularitySpec wrappedSpec; @JsonCreator public UniformGranularitySpec( @JsonProperty("gran") Granularity granularity, - @JsonProperty("intervals") List intervals + @JsonProperty("intervals") List inputIntervals ) { + List granularIntervals = Lists.newArrayList(); + + for (Interval inputInterval : inputIntervals) { + Iterables.addAll(granularIntervals, granularity.getIterable(inputInterval)); + } + this.granularity = granularity; - this.intervals = intervals; + this.inputIntervals = ImmutableList.copyOf(inputIntervals); + this.wrappedSpec = new ArbitraryGranularitySpec(granularIntervals); } @Override public SortedSet bucketIntervals() { - final TreeSet retVal = Sets.newTreeSet(Comparators.intervals()); - - for (Interval interval : intervals) { - for (Interval segmentInterval : granularity.getIterable(interval)) { - retVal.add(segmentInterval); - } - } - - return retVal; + return wrappedSpec.bucketIntervals(); } @Override public Optional bucketInterval(DateTime dt) { - return Optional.of(granularity.bucket(dt)); + return wrappedSpec.bucketInterval(dt); } - @JsonProperty + @JsonProperty("gran") public Granularity getGranularity() { return granularity; } - @JsonProperty + @JsonProperty("intervals") public Iterable getIntervals() { - return intervals; + return inputIntervals; } } diff --git a/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java b/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java index efc8113ee1b..0044d7d13e6 100644 --- a/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java +++ b/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java @@ -69,6 +69,12 @@ public class ArbitraryGranularityTest spec.bucketInterval(new DateTime("2012-01-03T01Z")) ); + Assert.assertEquals( + "2012-01-04T01Z", + Optional.absent(), + spec.bucketInterval(new DateTime("2012-01-04T01Z")) + ); + Assert.assertEquals( "2012-01-07T23:59:59.999Z", Optional.of(new Interval("2012-01-07T00Z/2012-01-08T00Z")), diff --git a/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java b/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java index 1f37da56de0..ab21be5f9f5 100644 --- a/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java +++ b/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java @@ -72,6 +72,12 @@ public class UniformGranularityTest spec.bucketInterval(new DateTime("2012-01-03T01Z")) ); + Assert.assertEquals( + "2012-01-04T01Z", + Optional.absent(), + spec.bucketInterval(new DateTime("2012-01-04T01Z")) + ); + Assert.assertEquals( "2012-01-07T23:59:59.999Z", Optional.of(new Interval("2012-01-07T00Z/2012-01-08T00Z")), From 8b31d8db9f93ec1a2e3441e77c367862c037720c Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Wed, 16 Jan 2013 10:01:46 -0600 Subject: [PATCH 24/92] 1) Adjust IndexMerger to create convert the indexes it creates from the old format to the new. This is done quite sub-optimally, but it will work for now... --- .../com/metamx/druid/index/v1/IndexIO.java | 2 +- .../metamx/druid/index/v1/IndexMerger.java | 44 +++++++++---------- 2 files changed, 23 insertions(+), 23 deletions(-) diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index 2afcbed6446..6836b9233c5 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -332,7 +332,7 @@ public class IndexIO throw new UnsupportedOperationException("Shouldn't ever happen in a cluster that is not owned by MMX."); } - public void convertV8toV9(File v8Dir, File v9Dir) throws IOException + public static void convertV8toV9(File v8Dir, File v9Dir) throws IOException { log.info("Converting v8[%s] to v9[%s]", v8Dir, v9Dir); diff --git a/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java b/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java index 10613561daa..6f9892ee805 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java +++ b/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java @@ -384,7 +384,6 @@ public class IndexMerger final Function>, Iterable> rowMergerFn ) throws IOException { - // TODO: make v9 index, complain to Eric when you see this, cause he should be doing it. Map metricTypes = Maps.newTreeMap(Ordering.natural().nullsFirst()); for (IndexableAdapter adapter : indexes) { for (String metric : adapter.getAvailableMetrics()) { @@ -392,11 +391,12 @@ public class IndexMerger } } final Interval dataInterval; + File v8OutDir = new File(outDir, "v8-tmp"); /************* Main index.drd file **************/ progress.progress(); long startTime = System.currentTimeMillis(); - File indexFile = new File(outDir, "index.drd"); + File indexFile = new File(v8OutDir, "index.drd"); FileOutputStream fileOutputStream = null; FileChannel channel = null; @@ -426,7 +426,7 @@ public class IndexMerger fileOutputStream = null; } IndexIO.checkFileSize(indexFile); - log.info("outDir[%s] completed index.drd in %,d millis.", outDir, System.currentTimeMillis() - startTime); + log.info("outDir[%s] completed index.drd in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime); /************* Setup Dim Conversions **************/ progress.progress(); @@ -499,7 +499,7 @@ public class IndexMerger } dimensionCardinalities.put(dimension, count); - FileOutputSupplier dimOut = new FileOutputSupplier(IndexIO.makeDimFile(outDir, dimension), true); + FileOutputSupplier dimOut = new FileOutputSupplier(IndexIO.makeDimFile(v8OutDir, dimension), true); dimOuts.add(dimOut); writer.close(); @@ -514,7 +514,7 @@ public class IndexMerger ioPeon.cleanup(); } - log.info("outDir[%s] completed dim conversions in %,d millis.", outDir, System.currentTimeMillis() - startTime); + log.info("outDir[%s] completed dim conversions in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime); /************* Walk through data sets and merge them *************/ progress.progress(); @@ -595,7 +595,7 @@ public class IndexMerger String metric = entry.getKey(); String typeName = entry.getValue(); if ("float".equals(typeName)) { - metWriters.add(new FloatMetricColumnSerializer(metric, outDir, ioPeon)); + metWriters.add(new FloatMetricColumnSerializer(metric, v8OutDir, ioPeon)); } else { ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); @@ -603,7 +603,7 @@ public class IndexMerger throw new ISE("Unknown type[%s]", typeName); } - metWriters.add(new ComplexMetricColumnSerializer(metric, outDir, ioPeon, serde)); + metWriters.add(new ComplexMetricColumnSerializer(metric, v8OutDir, ioPeon, serde)); } } for (MetricColumnSerializer metWriter : metWriters) { @@ -650,7 +650,7 @@ public class IndexMerger if ((++rowCount % 500000) == 0) { log.info( - "outDir[%s] walked 500,000/%,d rows in %,d millis.", outDir, rowCount, System.currentTimeMillis() - time + "outDir[%s] walked 500,000/%,d rows in %,d millis.", v8OutDir, rowCount, System.currentTimeMillis() - time ); time = System.currentTimeMillis(); } @@ -660,13 +660,13 @@ public class IndexMerger rowNumConversion.rewind(); } - final File littleEndianFile = IndexIO.makeTimeFile(outDir, ByteOrder.LITTLE_ENDIAN); + final File littleEndianFile = IndexIO.makeTimeFile(v8OutDir, ByteOrder.LITTLE_ENDIAN); littleEndianFile.delete(); OutputSupplier out = Files.newOutputStreamSupplier(littleEndianFile, true); littleEndianTimeWriter.closeAndConsolidate(out); IndexIO.checkFileSize(littleEndianFile); - final File bigEndianFile = IndexIO.makeTimeFile(outDir, ByteOrder.BIG_ENDIAN); + final File bigEndianFile = IndexIO.makeTimeFile(v8OutDir, ByteOrder.BIG_ENDIAN); bigEndianFile.delete(); out = Files.newOutputStreamSupplier(bigEndianFile, true); bigEndianTimeWriter.closeAndConsolidate(out); @@ -684,7 +684,7 @@ public class IndexMerger ioPeon.cleanup(); log.info( "outDir[%s] completed walk through of %,d rows in %,d millis.", - outDir, + v8OutDir, rowCount, System.currentTimeMillis() - startTime ); @@ -692,7 +692,7 @@ public class IndexMerger /************ Create Inverted Indexes *************/ startTime = System.currentTimeMillis(); - final File invertedFile = new File(outDir, "inverted.drd"); + final File invertedFile = new File(v8OutDir, "inverted.drd"); Files.touch(invertedFile); out = Files.newOutputStreamSupplier(invertedFile, true); for (int i = 0; i < mergedDimensions.size(); ++i) { @@ -725,10 +725,7 @@ public class IndexMerger } ConciseSet bitset = new ConciseSet(); - for (Integer row : CombiningIterable.createSplatted( - convertedInverteds, - Ordering.natural().nullsFirst() - )) { + for (Integer row : CombiningIterable.createSplatted(convertedInverteds, Ordering.natural().nullsFirst())) { if (row != INVALID_ROW) { bitset.add(row); } @@ -744,7 +741,7 @@ public class IndexMerger log.info("Completed dimension[%s] in %,d millis.", dimension, System.currentTimeMillis() - dimStartTime); } - log.info("outDir[%s] completed inverted.drd in %,d millis.", outDir, System.currentTimeMillis() - startTime); + log.info("outDir[%s] completed inverted.drd in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime); final ArrayList expectedFiles = Lists.newArrayList( Iterables.concat( @@ -759,18 +756,18 @@ public class IndexMerger Map files = Maps.newLinkedHashMap(); for (String fileName : expectedFiles) { - files.put(fileName, new File(outDir, fileName)); + files.put(fileName, new File(v8OutDir, fileName)); } - File smooshDir = new File(outDir, "smoosher"); + File smooshDir = new File(v8OutDir, "smoosher"); smooshDir.mkdir(); - for (Map.Entry entry : Smoosh.smoosh(outDir, smooshDir, files).entrySet()) { + for (Map.Entry entry : Smoosh.smoosh(v8OutDir, smooshDir, files).entrySet()) { entry.getValue().delete(); } for (File file : smooshDir.listFiles()) { - Files.move(file, new File(outDir, file.getName())); + Files.move(file, new File(v8OutDir, file.getName())); } if (!smooshDir.delete()) { @@ -780,12 +777,15 @@ public class IndexMerger createIndexDrdFile( IndexIO.CURRENT_VERSION_ID, - outDir, + v8OutDir, GenericIndexed.fromIterable(mergedDimensions, GenericIndexed.stringStrategy), GenericIndexed.fromIterable(mergedMetrics, GenericIndexed.stringStrategy), dataInterval ); + IndexIO.DefaultIndexIOHandler.convertV8toV9(v8OutDir, outDir); + FileUtils.deleteDirectory(v8OutDir); + return outDir; } From 7b42ee6a6e8fd2c60aa44dc51dd44937230a785a Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 15 Jan 2013 22:23:21 -0800 Subject: [PATCH 25/92] Rework DeterminePartitionsJob in the hadoop indexer - Can handle non-rolled-up input (by grouping input rows using an additional MR stage) - Can select its own partitioning dimension, if none is supplied - Can detect and avoid oversized shards due to bad dimension value distribution - Shares input parsing code with IndexGeneratorJob --- .../druid/indexer/DeterminePartitionsJob.java | 688 ++++++++++++------ .../indexer/HadoopDruidIndexerConfig.java | 67 +- .../indexer/HadoopDruidIndexerMapper.java | 69 ++ .../druid/indexer/IndexGeneratorJob.java | 131 +--- .../metamx/druid/indexer/SortableBytes.java | 2 +- 5 files changed, 617 insertions(+), 340 deletions(-) create mode 100644 indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index a40642ebec5..e257d4ed6b7 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -23,21 +23,22 @@ import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; -import com.google.common.base.Preconditions; import com.google.common.base.Splitter; +import com.google.common.base.Throwables; +import com.google.common.collect.ComparisonChain; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSortedSet; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.PeekingIterator; import com.google.common.io.Closeables; -import com.metamx.common.IAE; -import com.metamx.common.Pair; +import com.metamx.common.ISE; import com.metamx.common.guava.nary.BinaryFn; import com.metamx.common.logger.Logger; -import com.metamx.common.parsers.Parser; -import com.metamx.common.parsers.ParserUtils; import com.metamx.druid.CombiningIterable; +import com.metamx.druid.input.InputRow; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.ShardSpec; import com.metamx.druid.shard.SingleDimensionShardSpec; @@ -45,7 +46,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapred.InvalidJobConfException; @@ -56,8 +57,11 @@ import org.apache.hadoop.mapreduce.RecordWriter; import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TaskInputOutputContext; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; import org.codehaus.jackson.type.TypeReference; import org.joda.time.DateTime; import org.joda.time.DateTimeComparator; @@ -65,20 +69,26 @@ import org.joda.time.Interval; import java.io.IOException; import java.io.OutputStream; -import java.util.ArrayList; import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; /** + * Determines appropriate ShardSpecs for a job by determining whether or not partitioning is necessary, and if so, + * choosing the highest cardinality dimension that satisfies the criteria: + * + *
    + *
  • Must have exactly one value per row.
  • + *
  • Must not generate oversized partitions. A dimension with N rows having the same value will necessarily + * put all those rows in the same partition, and that partition may be much larger than the target size.
  • + *
*/ public class DeterminePartitionsJob implements Jobby { private static final Logger log = new Logger(DeterminePartitionsJob.class); - private static final Joiner keyJoiner = Joiner.on(","); - private static final Splitter keySplitter = Splitter.on(","); private static final Joiner tabJoiner = HadoopDruidIndexerConfig.tabJoiner; private static final Splitter tabSplitter = HadoopDruidIndexerConfig.tabSplitter; @@ -91,96 +101,171 @@ public class DeterminePartitionsJob implements Jobby this.config = config; } - public boolean run() + public static void injectSystemProperties(Job job) { - try { - Job job = new Job( - new Configuration(), - String.format("%s-determine_partitions-%s", config.getDataSource(), config.getIntervals()) - ); - - job.getConfiguration().set("io.sort.record.percent", "0.19"); - for (String propName : System.getProperties().stringPropertyNames()) { - Configuration conf = job.getConfiguration(); - if (propName.startsWith("hadoop.")) { - conf.set(propName.substring("hadoop.".length()), System.getProperty(propName)); - } + for (String propName : System.getProperties().stringPropertyNames()) { + Configuration conf = job.getConfiguration(); + if (propName.startsWith("hadoop.")) { + conf.set(propName.substring("hadoop.".length()), System.getProperty(propName)); } - - job.setInputFormatClass(TextInputFormat.class); - - job.setMapperClass(DeterminePartitionsMapper.class); - job.setMapOutputValueClass(Text.class); - - SortableBytes.useSortableBytesAsKey(job); - - job.setCombinerClass(DeterminePartitionsCombiner.class); - job.setReducerClass(DeterminePartitionsReducer.class); - job.setOutputKeyClass(BytesWritable.class); - job.setOutputValueClass(Text.class); - job.setOutputFormatClass(DeterminePartitionsJob.DeterminePartitionsOutputFormat.class); - FileOutputFormat.setOutputPath(job, config.makeIntermediatePath()); - - config.addInputPaths(job); - config.intoConfiguration(job); - - job.setJarByClass(DeterminePartitionsJob.class); - - job.submit(); - log.info("Job submitted, status available at %s", job.getTrackingURL()); - - final boolean retVal = job.waitForCompletion(true); - - if (retVal) { - log.info("Job completed, loading up partitions for intervals[%s].", config.getSegmentGranularIntervals()); - FileSystem fileSystem = null; - Map> shardSpecs = Maps.newTreeMap(DateTimeComparator.getInstance()); - int shardCount = 0; - for (Interval segmentGranularity : config.getSegmentGranularIntervals()) { - DateTime bucket = segmentGranularity.getStart(); - - final Path partitionInfoPath = config.makeSegmentPartitionInfoPath(new Bucket(0, bucket, 0)); - if (fileSystem == null) { - fileSystem = partitionInfoPath.getFileSystem(job.getConfiguration()); - } - if (fileSystem.exists(partitionInfoPath)) { - List specs = config.jsonMapper.readValue( - Utils.openInputStream(job, partitionInfoPath), new TypeReference>() - { - } - ); - - List actualSpecs = Lists.newArrayListWithExpectedSize(specs.size()); - for (int i = 0; i < specs.size(); ++i) { - actualSpecs.add(new HadoopyShardSpec(specs.get(i), shardCount++)); - log.info("DateTime[%s], partition[%d], spec[%s]", bucket, i, actualSpecs.get(i)); - } - - shardSpecs.put(bucket, actualSpecs); - } - else { - log.info("Path[%s] didn't exist!?", partitionInfoPath); - } - } - config.setShardSpecs(shardSpecs); - } - else { - log.info("Job completed unsuccessfully."); - } - - return retVal; - } - catch (Exception e) { - throw new RuntimeException(e); } } - public static class DeterminePartitionsMapper extends Mapper + public boolean run() + { + try { + /* + * Group by (timestamp, dimensions) so we can correctly count dimension values as they would appear + * in the final segment. + */ + final Job groupByJob = new Job( + new Configuration(), + String.format("%s-determine_partitions_groupby-%s", config.getDataSource(), config.getIntervals()) + ); + + injectSystemProperties(groupByJob); + groupByJob.setInputFormatClass(TextInputFormat.class); + groupByJob.setMapperClass(DeterminePartitionsGroupByMapper.class); + groupByJob.setMapOutputKeyClass(Text.class); + groupByJob.setMapOutputValueClass(NullWritable.class); + groupByJob.setCombinerClass(DeterminePartitionsGroupByReducer.class); + groupByJob.setReducerClass(DeterminePartitionsGroupByReducer.class); + groupByJob.setOutputKeyClass(Text.class); + groupByJob.setOutputValueClass(NullWritable.class); + groupByJob.setOutputFormatClass(SequenceFileOutputFormat.class); + groupByJob.setJarByClass(DeterminePartitionsJob.class); + + config.addInputPaths(groupByJob); + config.intoConfiguration(groupByJob); + FileOutputFormat.setOutputPath(groupByJob, config.makeGroupedDataDir()); + + groupByJob.submit(); + log.info("Job submitted, status available at: %s", groupByJob.getTrackingURL()); + + if(!groupByJob.waitForCompletion(true)) { + log.error("Job failed: %s", groupByJob.getJobID().toString()); + return false; + } + + /* + * Read grouped data from previous job and determine appropriate partitions. + */ + final Job dimSelectionJob = new Job( + new Configuration(), + String.format("%s-determine_partitions_dimselection-%s", config.getDataSource(), config.getIntervals()) + ); + + dimSelectionJob.getConfiguration().set("io.sort.record.percent", "0.19"); + + injectSystemProperties(dimSelectionJob); + dimSelectionJob.setInputFormatClass(SequenceFileInputFormat.class); + dimSelectionJob.setMapperClass(DeterminePartitionsDimSelectionMapper.class); + SortableBytes.useSortableBytesAsMapOutputKey(dimSelectionJob); + dimSelectionJob.setMapOutputValueClass(Text.class); + dimSelectionJob.setCombinerClass(DeterminePartitionsDimSelectionCombiner.class); + dimSelectionJob.setReducerClass(DeterminePartitionsDimSelectionReducer.class); + dimSelectionJob.setOutputKeyClass(BytesWritable.class); + dimSelectionJob.setOutputValueClass(Text.class); + dimSelectionJob.setOutputFormatClass(DeterminePartitionsDimSelectionOutputFormat.class); + dimSelectionJob.setJarByClass(DeterminePartitionsJob.class); + + config.intoConfiguration(dimSelectionJob); + FileInputFormat.addInputPath(dimSelectionJob, config.makeGroupedDataDir()); + FileOutputFormat.setOutputPath(dimSelectionJob, config.makeIntermediatePath()); + + dimSelectionJob.submit(); + log.info("Job submitted, status available at: %s", dimSelectionJob.getTrackingURL()); + + if(!dimSelectionJob.waitForCompletion(true)) { + log.error("Job failed: %s", dimSelectionJob.getJobID().toString()); + return false; + } + + /* + * Load partitions determined by the previous job. + */ + + log.info("Job completed, loading up partitions for intervals[%s].", config.getSegmentGranularIntervals()); + FileSystem fileSystem = null; + Map> shardSpecs = Maps.newTreeMap(DateTimeComparator.getInstance()); + int shardCount = 0; + for (Interval segmentGranularity : config.getSegmentGranularIntervals()) { + DateTime bucket = segmentGranularity.getStart(); + + final Path partitionInfoPath = config.makeSegmentPartitionInfoPath(new Bucket(0, bucket, 0)); + if (fileSystem == null) { + fileSystem = partitionInfoPath.getFileSystem(dimSelectionJob.getConfiguration()); + } + if (fileSystem.exists(partitionInfoPath)) { + List specs = config.jsonMapper.readValue( + Utils.openInputStream(dimSelectionJob, partitionInfoPath), new TypeReference>() + { + } + ); + + List actualSpecs = Lists.newArrayListWithExpectedSize(specs.size()); + for (int i = 0; i < specs.size(); ++i) { + actualSpecs.add(new HadoopyShardSpec(specs.get(i), shardCount++)); + log.info("DateTime[%s], partition[%d], spec[%s]", bucket, i, actualSpecs.get(i)); + } + + shardSpecs.put(bucket, actualSpecs); + } + else { + log.info("Path[%s] didn't exist!?", partitionInfoPath); + } + } + config.setShardSpecs(shardSpecs); + + return true; + } catch(Exception e) { + throw Throwables.propagate(e); + } + } + + public static class DeterminePartitionsGroupByMapper extends HadoopDruidIndexerMapper + { + @Override + protected void innerMap( + InputRow inputRow, + Text text, + Context context + ) throws IOException, InterruptedException + { + // Create group key + // TODO -- There are more efficient ways to do this + final Map> dims = Maps.newTreeMap(); + for(final String dim : inputRow.getDimensions()) { + final Set dimValues = ImmutableSortedSet.copyOf(inputRow.getDimension(dim)); + if(dimValues.size() > 0) { + dims.put(dim, dimValues); + } + } + final List groupKey = ImmutableList.of( + getConfig().getRollupSpec().getRollupGranularity().truncate(inputRow.getTimestampFromEpoch()), + dims + ); + context.write(new Text(HadoopDruidIndexerConfig.jsonMapper.writeValueAsBytes(groupKey)), NullWritable.get()); + } + } + + public static class DeterminePartitionsGroupByReducer extends Reducer + { + @Override + protected void reduce( + Text key, + Iterable values, + Context context + ) throws IOException, InterruptedException + { + context.write(key, NullWritable.get()); + } + } + + public static class DeterminePartitionsDimSelectionMapper extends Mapper { private HadoopDruidIndexerConfig config; private String partitionDimension; - private Parser parser; - private Function timestampConverter; @Override protected void setup(Context context) @@ -188,49 +273,49 @@ public class DeterminePartitionsJob implements Jobby { config = HadoopDruidIndexerConfig.fromConfiguration(context.getConfiguration()); partitionDimension = config.getPartitionDimension(); - parser = config.getDataSpec().getParser(); - timestampConverter = ParserUtils.createTimestampParser(config.getTimestampFormat()); } @Override protected void map( - LongWritable key, Text value, Context context + Text key, NullWritable value, Context context ) throws IOException, InterruptedException { - Map values = parser.parse(value.toString()); - final DateTime timestamp; - final String tsStr = (String) values.get(config.getTimestampColumnName()); - try { - timestamp = timestampConverter.apply(tsStr); - } - catch(IllegalArgumentException e) { - if(config.isIgnoreInvalidRows()) { - context.getCounter(HadoopDruidIndexerConfig.IndexJobCounters.INVALID_ROW_COUNTER).increment(1); - return; // we're ignoring this invalid row - } - else { - throw e; - } - } + final List timeAndDims = + HadoopDruidIndexerConfig.jsonMapper.readValue( + key.getBytes(), new TypeReference>() {} + ); + final DateTime timestamp = new DateTime(timeAndDims.get(0)); + final Map dims = (Map) timeAndDims.get(1); final Optional maybeInterval = config.getGranularitySpec().bucketInterval(timestamp); - if(maybeInterval.isPresent()) { - final DateTime bucket = maybeInterval.get().getStart(); - final String outKey = keyJoiner.join(bucket.toString(), partitionDimension); - final Object dimValue = values.get(partitionDimension); - if (! (dimValue instanceof String)) { - throw new IAE("Cannot partition on a tag-style dimension[%s], line was[%s]", partitionDimension, value); + if(!maybeInterval.isPresent()) { + throw new ISE("WTF?! No bucket found for timestamp: %s", timestamp); + } + + final Interval interval = maybeInterval.get(); + final byte[] groupKey = interval.getStart().toString().getBytes(Charsets.UTF_8); + + for(final Map.Entry dimAndValues : dims.entrySet()) { + final String dim = dimAndValues.getKey(); + + if(partitionDimension == null || partitionDimension.equals(dim)) { + final List dimValues = (List) dimAndValues.getValue(); + + if(dimValues.size() == 1) { + // Emit this value. + write(context, groupKey, new DimValueCount(dim, dimValues.get(0), 1)); + } else { + // This dimension is unsuitable for partitioning. Poison it by emitting a negative value. + write(context, groupKey, new DimValueCount(dim, "", -1)); + } } - - final byte[] groupKey = outKey.getBytes(Charsets.UTF_8); - write(context, groupKey, "", 1); - write(context, groupKey, (String) dimValue, 1); } } } - private static abstract class DeterminePartitionsBaseReducer extends Reducer + private static abstract class DeterminePartitionsDimSelectionBaseReducer + extends Reducer { protected static volatile HadoopDruidIndexerConfig config = null; @@ -240,7 +325,7 @@ public class DeterminePartitionsJob implements Jobby throws IOException, InterruptedException { if (config == null) { - synchronized (DeterminePartitionsBaseReducer.class) { + synchronized (DeterminePartitionsDimSelectionBaseReducer.class) { if (config == null) { config = HadoopDruidIndexerConfig.fromConfiguration(context.getConfiguration()); } @@ -255,166 +340,277 @@ public class DeterminePartitionsJob implements Jobby { SortableBytes keyBytes = SortableBytes.fromBytesWritable(key); - final Iterable> combinedIterable = combineRows(values); + final Iterable combinedIterable = combineRows(values); innerReduce(context, keyBytes, combinedIterable); } protected abstract void innerReduce( - Context context, SortableBytes keyBytes, Iterable> combinedIterable + Context context, SortableBytes keyBytes, Iterable combinedIterable ) throws IOException, InterruptedException; - private Iterable> combineRows(Iterable input) + private Iterable combineRows(Iterable input) { - return new CombiningIterable>( + return new CombiningIterable( Iterables.transform( input, - new Function>() + new Function() { @Override - public Pair apply(Text input) + public DimValueCount apply(Text input) { - Iterator splits = tabSplitter.split(input.toString()).iterator(); - return new Pair(splits.next(), Long.parseLong(splits.next())); + return DimValueCount.fromText(input); } } ), - new Comparator>() + new Comparator() { @Override - public int compare(Pair o1, Pair o2) + public int compare(DimValueCount o1, DimValueCount o2) { - return o1.lhs.compareTo(o2.lhs); + return ComparisonChain.start().compare(o1.dim, o2.dim).compare(o1.value, o2.value).result(); } }, - new BinaryFn, Pair, Pair>() + new BinaryFn() { @Override - public Pair apply(Pair arg1, Pair arg2) + public DimValueCount apply(DimValueCount arg1, DimValueCount arg2) { if (arg2 == null) { return arg1; } - return new Pair(arg1.lhs, arg1.rhs + arg2.rhs); + // Respect "poisoning" (negative values mean we can't use this dimension) + final int newNumRows = (arg1.numRows >= 0 && arg2.numRows >= 0 ? arg1.numRows + arg2.numRows : -1); + return new DimValueCount(arg1.dim, arg1.value, newNumRows); } } ); } } - public static class DeterminePartitionsCombiner extends DeterminePartitionsBaseReducer + public static class DeterminePartitionsDimSelectionCombiner extends DeterminePartitionsDimSelectionBaseReducer { @Override protected void innerReduce( - Context context, SortableBytes keyBytes, Iterable> combinedIterable + Context context, SortableBytes keyBytes, Iterable combinedIterable ) throws IOException, InterruptedException { - for (Pair pair : combinedIterable) { - write(context, keyBytes.getGroupKey(), pair.lhs, pair.rhs); + for (DimValueCount dvc : combinedIterable) { + write(context, keyBytes.getGroupKey(), dvc); } } } - public static class DeterminePartitionsReducer extends DeterminePartitionsBaseReducer + public static class DeterminePartitionsDimSelectionReducer extends DeterminePartitionsDimSelectionBaseReducer { - String previousBoundary; - long runningTotal; + private static final double SHARD_COMBINE_THRESHOLD = 0.25; + private static final double SHARD_OVERSIZE_THRESHOLD = 1.5; @Override protected void innerReduce( - Context context, SortableBytes keyBytes, Iterable> combinedIterable + Context context, SortableBytes keyBytes, Iterable combinedIterable ) throws IOException, InterruptedException { - PeekingIterator> iterator = Iterators.peekingIterator(combinedIterable.iterator()); - Pair totalPair = iterator.next(); + PeekingIterator iterator = Iterators.peekingIterator(combinedIterable.iterator()); - Preconditions.checkState(totalPair.lhs.equals(""), "Total pair value was[%s]!?", totalPair.lhs); - long totalRows = totalPair.rhs; + // "iterator" will take us over many candidate dimensions + DimPartitions currentDimPartitions = null; + DimPartition currentDimPartition = null; + String currentDimPartitionStart = null; + boolean currentDimSkip = false; - long numPartitions = Math.max(totalRows / config.getTargetPartitionSize(), 1); - long expectedRowsPerPartition = totalRows / numPartitions; + // We'll store possible partitions in here + final Map dimPartitionss = Maps.newHashMap(); - class PartitionsList extends ArrayList - { - } - List partitions = new PartitionsList(); + while(iterator.hasNext()) { + final DimValueCount dvc = iterator.next(); - runningTotal = 0; - Pair prev = null; - previousBoundary = null; - while (iterator.hasNext()) { - Pair curr = iterator.next(); - - if (runningTotal > expectedRowsPerPartition) { - Preconditions.checkNotNull( - prev, "Prev[null] while runningTotal[%s] was > expectedRows[%s]!?", runningTotal, expectedRowsPerPartition - ); - - addPartition(partitions, curr.lhs); + if(currentDimPartitions == null || !currentDimPartitions.dim.equals(dvc.dim)) { + // Starting a new dimension! Exciting! + currentDimPartitions = new DimPartitions(dvc.dim); + currentDimPartition = new DimPartition(); + currentDimPartitionStart = null; + currentDimSkip = false; } - runningTotal += curr.rhs; - prev = curr; + // Respect poisoning + if(!currentDimSkip && dvc.numRows < 0) { + log.info("Cannot partition on multi-valued dimension: %s", dvc.dim); + currentDimSkip = true; + } + + if(currentDimSkip) { + continue; + } + + // See if we need to cut a new partition ending immediately before this dimension value + if(currentDimPartition.rows > 0 && currentDimPartition.rows + dvc.numRows >= config.getTargetPartitionSize()) { + final ShardSpec shardSpec = new SingleDimensionShardSpec( + currentDimPartitions.dim, + currentDimPartitionStart, + dvc.value, + currentDimPartitions.partitions.size() + ); + + log.info( + "Adding possible shard with %,d rows and %,d unique values: %s", + currentDimPartition.rows, + currentDimPartition.cardinality, + shardSpec + ); + + currentDimPartition.shardSpec = shardSpec; + currentDimPartitions.partitions.add(currentDimPartition); + currentDimPartition = new DimPartition(); + currentDimPartitionStart = dvc.value; + } + + // Update counters + currentDimPartition.cardinality ++; + currentDimPartition.rows += dvc.numRows; + + if(!iterator.hasNext() || !currentDimPartitions.dim.equals(iterator.peek().dim)) { + // Finalize the current dimension + + if(currentDimPartition.rows > 0) { + // One more shard to go + final ShardSpec shardSpec; + + if (currentDimPartitions.partitions.isEmpty()) { + shardSpec = new NoneShardSpec(); + } else { + if(currentDimPartition.rows < config.getTargetPartitionSize() * SHARD_COMBINE_THRESHOLD) { + // Combine with previous shard + final DimPartition previousDimPartition = currentDimPartitions.partitions.remove( + currentDimPartitions.partitions.size() - 1 + ); + + final SingleDimensionShardSpec previousShardSpec = (SingleDimensionShardSpec) previousDimPartition.shardSpec; + + shardSpec = new SingleDimensionShardSpec( + currentDimPartitions.dim, + previousShardSpec.getStart(), + null, + previousShardSpec.getPartitionNum() + ); + + log.info("Removing possible shard: %s", previousShardSpec); + + currentDimPartition.rows += previousDimPartition.rows; + currentDimPartition.cardinality += previousDimPartition.cardinality; + } else { + // Create new shard + shardSpec = new SingleDimensionShardSpec( + currentDimPartitions.dim, + currentDimPartitionStart, + null, + currentDimPartitions.partitions.size() + ); + } + } + + log.info( + "Adding possible shard with %,d rows and %,d unique values: %s", + currentDimPartition.rows, + currentDimPartition.cardinality, + shardSpec + ); + + currentDimPartition.shardSpec = shardSpec; + currentDimPartitions.partitions.add(currentDimPartition); + } + + log.info( + "Completed dimension[%s]: %,d possible shards with %,d unique values", + currentDimPartitions.dim, + currentDimPartitions.partitions.size(), + currentDimPartitions.getCardinality() + ); + + // Add ourselves to the partitions map + dimPartitionss.put(currentDimPartitions.dim, currentDimPartitions); + } } - if (partitions.isEmpty()) { - partitions.add(new NoneShardSpec()); - } else if (((double) runningTotal / (double) expectedRowsPerPartition) < 0.25) { - final SingleDimensionShardSpec lastSpec = (SingleDimensionShardSpec) partitions.remove(partitions.size() - 1); - partitions.add( - new SingleDimensionShardSpec( - config.getPartitionDimension(), - lastSpec.getStart(), - null, - lastSpec.getPartitionNum() - ) - ); - } else { - partitions.add( - new SingleDimensionShardSpec( - config.getPartitionDimension(), - previousBoundary, - null, - partitions.size() - ) - ); + // Choose best dimension + if(dimPartitionss.isEmpty()) { + throw new ISE("No suitable partitioning dimension found!"); } - DateTime bucket = new DateTime( - Iterables.get(keySplitter.split(new String(keyBytes.getGroupKey(), Charsets.UTF_8)), 0) - ); - OutputStream out = Utils.makePathAndOutputStream( + final int totalRows = dimPartitionss.values().iterator().next().getRows(); + + int maxCardinality = -1; + DimPartitions maxCardinalityPartitions = null; + + for(final DimPartitions dimPartitions : dimPartitionss.values()) { + if(dimPartitions.getRows() != totalRows) { + throw new ISE( + "WTF?! Dimension[%s] row count %,d != expected row count %,d", + dimPartitions.dim, + dimPartitions.getRows(), + totalRows + ); + } + + // Make sure none of these shards are oversized + boolean oversized = false; + for(final DimPartition partition : dimPartitions.partitions) { + if(partition.rows > config.getTargetPartitionSize() * SHARD_OVERSIZE_THRESHOLD) { + log.info("Dimension[%s] has an oversized shard: %s", dimPartitions.dim, partition.shardSpec); + oversized = true; + } + } + + if(oversized) { + continue; + } + + if(dimPartitions.getCardinality() > maxCardinality) { + maxCardinality = dimPartitions.getCardinality(); + maxCardinalityPartitions = dimPartitions; + } + } + + if(maxCardinalityPartitions == null) { + throw new ISE("No suitable partitioning dimension found!"); + } + + final DateTime bucket = new DateTime(new String(keyBytes.getGroupKey(), Charsets.UTF_8)); + final OutputStream out = Utils.makePathAndOutputStream( context, config.makeSegmentPartitionInfoPath(new Bucket(0, bucket, 0)), config.isOverwriteFiles() ); - for (ShardSpec partition : partitions) { - log.info("%s", partition); + final List chosenShardSpecs = Lists.transform( + maxCardinalityPartitions.partitions, new Function() + { + @Override + public ShardSpec apply(DimPartition dimPartition) + { + return dimPartition.shardSpec; + } + } + ); + + log.info("Chosen partitions:"); + for (ShardSpec shardSpec : chosenShardSpecs) { + log.info(" %s", shardSpec); } try { - config.jsonMapper.writeValue(out, partitions); + // For some reason this used to work without writerWithType, but now it appears to forget to write "type" + // info for the ShardSpecs (so they cannot be deserialized). + HadoopDruidIndexerConfig.jsonMapper.writerWithType(new TypeReference>() {}).writeValue( + out, + chosenShardSpecs + ); } finally { Closeables.close(out, false); } } - - private void addPartition(List partitions, String boundary) - { - partitions.add( - new SingleDimensionShardSpec( - config.getPartitionDimension(), - previousBoundary, - boundary, - partitions.size() - ) - ); - previousBoundary = boundary; - runningTotal = 0; - } } - public static class DeterminePartitionsOutputFormat extends FileOutputFormat + public static class DeterminePartitionsDimSelectionOutputFormat extends FileOutputFormat { @Override public RecordWriter getRecordWriter(final TaskAttemptContext job) throws IOException, InterruptedException @@ -444,17 +640,81 @@ public class DeterminePartitionsJob implements Jobby } } + private static class DimPartitions + { + public final String dim; + public final List partitions = Lists.newArrayList(); + + private DimPartitions(String dim) + { + this.dim = dim; + } + + public int getCardinality() + { + int sum = 0; + for(final DimPartition dimPartition : partitions) { + sum += dimPartition.cardinality; + } + return sum; + } + + public int getRows() + { + int sum = 0; + for(final DimPartition dimPartition : partitions) { + sum += dimPartition.rows; + } + return sum; + } + } + + private static class DimPartition + { + public ShardSpec shardSpec = null; + public int cardinality = 0; + public int rows = 0; + } + + private static class DimValueCount + { + public final String dim; + public final String value; + public final int numRows; + + private DimValueCount(String dim, String value, int numRows) + { + this.dim = dim; + this.value = value; + this.numRows = numRows; + } + + public Text toText() + { + return new Text(tabJoiner.join(dim, String.valueOf(numRows), value)); + } + + public static DimValueCount fromText(Text text) + { + final Iterator splits = tabSplitter.limit(3).split(text.toString()).iterator(); + final String dim = splits.next(); + final int numRows = Integer.parseInt(splits.next()); + final String value = splits.next(); + + return new DimValueCount(dim, value, numRows); + } + } + private static void write( TaskInputOutputContext context, final byte[] groupKey, - String value, - long numRows + DimValueCount dimValueCount ) throws IOException, InterruptedException { context.write( - new SortableBytes(groupKey, value.getBytes(HadoopDruidIndexerConfig.javaNativeCharset)).toBytesWritable(), - new Text(tabJoiner.join(value, numRows)) + new SortableBytes(groupKey, tabJoiner.join(dimValueCount.dim, dimValueCount.value).getBytes(HadoopDruidIndexerConfig.javaNativeCharset)).toBytesWritable(), + dimValueCount.toText() ); } } diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java index 403484b9c61..d5034ef9966 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java @@ -34,15 +34,19 @@ import com.metamx.common.MapUtils; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; import com.metamx.druid.RegisteringNode; +import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.serde.Registererer; import com.metamx.druid.indexer.data.DataSpec; +import com.metamx.druid.indexer.data.StringInputRowParser; +import com.metamx.druid.indexer.data.TimestampSpec; import com.metamx.druid.indexer.data.ToLowercaseDataSpec; import com.metamx.druid.indexer.granularity.GranularitySpec; import com.metamx.druid.indexer.granularity.UniformGranularitySpec; import com.metamx.druid.indexer.path.PathSpec; import com.metamx.druid.indexer.rollup.DataRollupSpec; import com.metamx.druid.indexer.updater.UpdaterJobSpec; +import com.metamx.druid.input.InputRow; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.ShardSpec; import com.metamx.druid.utils.JodaUtils; @@ -60,8 +64,6 @@ import org.joda.time.format.ISODateTimeFormat; import javax.annotation.Nullable; import java.io.File; import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; import java.nio.charset.Charset; import java.util.Arrays; import java.util.Collections; @@ -237,6 +239,11 @@ public class HadoopDruidIndexerConfig this.timestampFormat = timestampFormat; } + public TimestampSpec getTimestampSpec() + { + return new TimestampSpec(timestampColumnName, timestampFormat); + } + @JsonProperty public DataSpec getDataSpec() { @@ -248,6 +255,32 @@ public class HadoopDruidIndexerConfig this.dataSpec = new ToLowercaseDataSpec(dataSpec); } + public StringInputRowParser getParser() + { + final List dimensionExclusions; + + if(getDataSpec().hasCustomDimensions()) { + dimensionExclusions = null; + } else { + dimensionExclusions = Lists.newArrayList(); + dimensionExclusions.add(getTimestampColumnName()); + dimensionExclusions.addAll( + Lists.transform( + getRollupSpec().getAggs(), new Function() + { + @Override + public String apply(AggregatorFactory aggregatorFactory) + { + return aggregatorFactory.getName(); + } + } + ) + ); + } + + return new StringInputRowParser(getTimestampSpec(), getDataSpec(), dimensionExclusions); + } + @Deprecated @JsonProperty public void setSegmentGranularity(Granularity segmentGranularity) @@ -335,7 +368,7 @@ public class HadoopDruidIndexerConfig public boolean partitionByDimension() { - return partitionDimension != null; + return targetPartitionSize != null; } @JsonProperty @@ -447,21 +480,15 @@ public class HadoopDruidIndexerConfig ********************************************/ /** - * Get the proper bucket for this "row" + * Get the proper bucket for some input row. * - * @param theMap a Map that represents a "row", keys are column names, values are, well, values + * @param inputRow an InputRow * * @return the Bucket that this row belongs to */ - public Optional getBucket(Map theMap) + public Optional getBucket(InputRow inputRow) { - final Optional timeBucket = getGranularitySpec().bucketInterval( - new DateTime( - theMap.get( - getTimestampColumnName() - ) - ) - ); + final Optional timeBucket = getGranularitySpec().bucketInterval(new DateTime(inputRow.getTimestampFromEpoch())); if (!timeBucket.isPresent()) { return Optional.absent(); } @@ -473,7 +500,7 @@ public class HadoopDruidIndexerConfig for (final HadoopyShardSpec hadoopyShardSpec : shards) { final ShardSpec actualSpec = hadoopyShardSpec.getActualSpec(); - if (actualSpec.isInChunk(theMap)) { + if (actualSpec.isInChunk(inputRow)) { return Optional.of( new Bucket( hadoopyShardSpec.getShardNum(), @@ -484,7 +511,7 @@ public class HadoopDruidIndexerConfig } } - throw new ISE("row[%s] doesn't fit in any shard[%s]", theMap, shards); + throw new ISE("row[%s] doesn't fit in any shard[%s]", inputRow, shards); } public Set getSegmentGranularIntervals() @@ -566,6 +593,11 @@ public class HadoopDruidIndexerConfig return new Path(makeIntermediatePath(), "segmentDescriptorInfo"); } + public Path makeGroupedDataDir() + { + return new Path(makeIntermediatePath(), "groupedData"); + } + public Path makeDescriptorInfoPath(DataSegment segment) { return new Path(makeDescriptorInfoDir(), String.format("%s.json", segment.getIdentifier().replace(":", ""))); @@ -626,10 +658,5 @@ public class HadoopDruidIndexerConfig final int nIntervals = getIntervals().size(); Preconditions.checkArgument(nIntervals > 0, "intervals.size()[%s] <= 0", nIntervals); - - if (partitionByDimension()) { - Preconditions.checkNotNull(partitionDimension); - Preconditions.checkNotNull(targetPartitionSize); - } } } diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java new file mode 100644 index 00000000000..f49762cc4c6 --- /dev/null +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java @@ -0,0 +1,69 @@ +package com.metamx.druid.indexer; + +import com.metamx.common.RE; +import com.metamx.druid.indexer.data.StringInputRowParser; +import com.metamx.druid.input.InputRow; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Mapper; +import org.joda.time.DateTime; + +import java.io.IOException; + +public class HadoopDruidIndexerMapper extends Mapper +{ + private HadoopDruidIndexerConfig config; + private StringInputRowParser parser; + + @Override + protected void setup(Context context) + throws IOException, InterruptedException + { + config = HadoopDruidIndexerConfig.fromConfiguration(context.getConfiguration()); + parser = config.getParser(); + } + + public HadoopDruidIndexerConfig getConfig() + { + return config; + } + + public StringInputRowParser getParser() + { + return parser; + } + + @Override + protected void map( + LongWritable key, Text value, Context context + ) throws IOException, InterruptedException + { + try { + final InputRow inputRow; + try { + inputRow = parser.parse(value.toString()); + } + catch (IllegalArgumentException e) { + if (config.isIgnoreInvalidRows()) { + context.getCounter(HadoopDruidIndexerConfig.IndexJobCounters.INVALID_ROW_COUNTER).increment(1); + return; // we're ignoring this invalid row + } else { + throw e; + } + } + + if(config.getGranularitySpec().bucketInterval(new DateTime(inputRow.getTimestampFromEpoch())).isPresent()) { + innerMap(inputRow, value, context); + } + } + catch (RuntimeException e) { + throw new RE(e, "Failure on row[%s]", value); + } + } + + protected void innerMap(InputRow inputRow, Text text, Context context) + throws IOException, InterruptedException + { + // no-op, meant to be overridden + } +} diff --git a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java index 34d743fc9be..721158efb91 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java @@ -19,31 +19,25 @@ package com.metamx.druid.indexer; -import com.google.common.base.Function; import com.google.common.base.Optional; -import com.google.common.base.Predicate; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.common.io.Closeables; import com.google.common.primitives.Longs; import com.metamx.common.ISE; -import com.metamx.common.RE; -import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; -import com.metamx.common.parsers.Parser; -import com.metamx.common.parsers.ParserUtils; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; import com.metamx.druid.index.v1.MMappedIndex; +import com.metamx.druid.indexer.data.StringInputRowParser; import com.metamx.druid.indexer.rollup.DataRollupSpec; -import com.metamx.druid.input.MapBasedInputRow; +import com.metamx.druid.input.InputRow; import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; @@ -53,13 +47,11 @@ import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3native.NativeS3FileSystem; import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.InvalidJobConfException; import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.JobContext; -import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.Partitioner; import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; @@ -68,7 +60,6 @@ import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; import org.joda.time.DateTime; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.io.BufferedOutputStream; import java.io.File; import java.io.FileInputStream; @@ -78,7 +69,6 @@ import java.net.URI; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.zip.ZipEntry; import java.util.zip.ZipOutputStream; @@ -127,7 +117,7 @@ public class IndexGeneratorJob implements Jobby job.setMapperClass(IndexGeneratorMapper.class); job.setMapOutputValueClass(Text.class); - SortableBytes.useSortableBytesAsKey(job); + SortableBytes.useSortableBytesAsMapOutputKey(job); job.setNumReduceTasks(Iterables.size(config.getAllBuckets())); job.setPartitionerClass(IndexGeneratorPartitioner.class); @@ -159,75 +149,29 @@ public class IndexGeneratorJob implements Jobby } } - public static class IndexGeneratorMapper extends Mapper + public static class IndexGeneratorMapper extends HadoopDruidIndexerMapper { - private HadoopDruidIndexerConfig config; - private Parser parser; - private Function timestampConverter; - @Override - protected void setup(Context context) - throws IOException, InterruptedException - { - config = HadoopDruidIndexerConfig.fromConfiguration(context.getConfiguration()); - parser = config.getDataSpec().getParser(); - timestampConverter = ParserUtils.createTimestampParser(config.getTimestampFormat()); - } - - @Override - protected void map( - LongWritable key, Text value, Context context + protected void innerMap( + InputRow inputRow, + Text text, + Context context ) throws IOException, InterruptedException { + // Group by bucket, sort by timestamp + final Optional bucket = getConfig().getBucket(inputRow); - try { - final Map values = parser.parse(value.toString()); - - final String tsStr = (String) values.get(config.getTimestampColumnName()); - final DateTime timestamp; - try { - timestamp = timestampConverter.apply(tsStr); - } - catch (IllegalArgumentException e) { - if (config.isIgnoreInvalidRows()) { - context.getCounter(HadoopDruidIndexerConfig.IndexJobCounters.INVALID_ROW_COUNTER).increment(1); - return; // we're ignoring this invalid row - } else { - throw e; - } - } - - Optional bucket = config.getBucket( - Maps.transformEntries( - values, - new Maps.EntryTransformer() - { - @Override - public String transformEntry(@Nullable String key, @Nullable Object value) - { - if (key.equalsIgnoreCase(config.getTimestampColumnName())) { - return timestamp.toString(); - } - return value.toString(); - } - } - ) - ); - - if (bucket.isPresent()) { - // Group by bucket, sort by timestamp - context.write( - new SortableBytes( - bucket.get().toGroupKey(), - Longs.toByteArray(timestamp.getMillis()) - ).toBytesWritable(), - value - ); - } - } - catch (RuntimeException e) { - throw new RE(e, "Failure on row[%s]", value); + if(!bucket.isPresent()) { + throw new ISE("WTF?! No bucket found for row: %s", inputRow); } + + context.write( + new SortableBytes( + bucket.get().toGroupKey(), + Longs.toByteArray(inputRow.getTimestampFromEpoch()) + ).toBytesWritable(), + text + ); } } @@ -253,8 +197,7 @@ public class IndexGeneratorJob implements Jobby { private HadoopDruidIndexerConfig config; private List metricNames = Lists.newArrayList(); - private Function timestampConverter; - private Parser parser; + private StringInputRowParser parser; @Override protected void setup(Context context) @@ -265,8 +208,8 @@ public class IndexGeneratorJob implements Jobby for (AggregatorFactory factory : config.getRollupSpec().getAggs()) { metricNames.add(factory.getName().toLowerCase()); } - timestampConverter = ParserUtils.createTimestampParser(config.getTimestampFormat()); - parser = config.getDataSpec().getParser(); + + parser = config.getParser(); } @Override @@ -299,32 +242,10 @@ public class IndexGeneratorJob implements Jobby for (final Text value : values) { context.progress(); - Map event = parser.parse(value.toString()); - final long timestamp = timestampConverter.apply((String) event.get(config.getTimestampColumnName())) - .getMillis(); - List dimensionNames = - config.getDataSpec().hasCustomDimensions() ? - config.getDataSpec().getDimensions() : - Lists.newArrayList( - FunctionalIterable.create(event.keySet()) - .filter( - new Predicate() - { - @Override - public boolean apply(@Nullable String input) - { - return !(metricNames.contains(input.toLowerCase()) - || config.getTimestampColumnName() - .equalsIgnoreCase(input)); - } - } - ) - ); - allDimensionNames.addAll(dimensionNames); + final InputRow inputRow = parser.parse(value.toString()); + allDimensionNames.addAll(inputRow.getDimensions()); - int numRows = index.add( - new MapBasedInputRow(timestamp, dimensionNames, event) - ); + int numRows = index.add(inputRow); ++lineCount; if (numRows >= rollupSpec.rowFlushBoundary) { diff --git a/indexer/src/main/java/com/metamx/druid/indexer/SortableBytes.java b/indexer/src/main/java/com/metamx/druid/indexer/SortableBytes.java index 3abaa7951b8..394f9dacffb 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/SortableBytes.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/SortableBytes.java @@ -102,7 +102,7 @@ public class SortableBytes ); } - public static void useSortableBytesAsKey(Job job) + public static void useSortableBytesAsMapOutputKey(Job job) { job.setMapOutputKeyClass(BytesWritable.class); job.setGroupingComparatorClass(SortableBytesGroupingComparator.class); From 6fc350bfbac649060546f1802cd5c2fc541411a9 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 16 Jan 2013 11:30:24 -0800 Subject: [PATCH 26/92] YeOldePlumberSchool: Populate dimension metadata in segment descriptor --- .../druid/merger/common/index/YeOldePlumberSchool.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java index a496f560970..77ce51ccb73 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java @@ -21,6 +21,7 @@ package com.metamx.druid.merger.common.index; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -138,7 +139,13 @@ public class YeOldePlumberSchool implements PlumberSchool IndexMerger.mergeMMapped(indexes, schema.getAggregators(), fileToUpload); } - final DataSegment segmentToUpload = theSink.getSegment().withVersion(version); + // Map merged segment so we can extract dimensions + final MMappedIndex mappedSegment = IndexIO.mapDir(fileToUpload); + + final DataSegment segmentToUpload = theSink.getSegment() + .withDimensions(ImmutableList.copyOf(mappedSegment.getAvailableDimensions())) + .withVersion(version); + segmentPusher.push(fileToUpload, segmentToUpload); log.info( From a2090411a3a821469b06fb6e978ec331b1ef3133 Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 7 Jan 2013 16:54:21 -0800 Subject: [PATCH 27/92] modify cacheBroker interface to implement bulk-get --- .../druid/client/CachingClusteredClient.java | 137 ++++++++++-------- .../com/metamx/druid/client/cache/Cache.java | 31 ---- .../druid/client/cache/CacheBroker.java | 12 +- .../druid/client/cache/MapCacheBroker.java | 43 +++++- .../client/cache/MemcachedCacheBroker.java | 125 +++++++++++----- .../client/cache/MapCacheBrokerTest.java | 47 +++--- .../cache/MemcachedCacheBrokerBenchmark.java | 33 ++++- .../cache/MemcachedCacheBrokerTest.java | 47 +++--- 8 files changed, 288 insertions(+), 187 deletions(-) delete mode 100644 client/src/main/java/com/metamx/druid/client/cache/Cache.java diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index 47376dd2e68..23f5ae6992e 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -20,6 +20,7 @@ package com.metamx.druid.client; import com.google.common.base.Function; +import com.google.common.base.Predicate; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -40,7 +41,6 @@ import com.metamx.druid.Query; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.aggregation.AggregatorFactory; -import com.metamx.druid.client.cache.Cache; import com.metamx.druid.client.cache.CacheBroker; import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.partition.PartitionChunk; @@ -57,6 +57,7 @@ import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -98,7 +99,7 @@ public class CachingClusteredClient implements QueryRunner @Override public ServerView.CallbackAction segmentRemoved(DruidServer server, DataSegment segment) { - CachingClusteredClient.this.cacheBroker.provideCache(segment.getIdentifier()).close(); + CachingClusteredClient.this.cacheBroker.close(segment.getIdentifier()); return ServerView.CallbackAction.CONTINUE; } } @@ -111,7 +112,8 @@ public class CachingClusteredClient implements QueryRunner final QueryToolChest> toolChest = warehouse.getToolChest(query); final CacheStrategy> strategy = toolChest.getCacheStrategy(query); - final Map> segs = Maps.newTreeMap(); + final Map> serverSegments = Maps.newTreeMap(); + final List> cachedResults = Lists.newArrayList(); final Map cachePopulatorMap = Maps.newHashMap(); @@ -131,10 +133,9 @@ public class CachingClusteredClient implements QueryRunner return Sequences.empty(); } - byte[] queryCacheKey = null; - if (strategy != null) { - queryCacheKey = strategy.computeCacheKey(query); - } + Map, Pair> segments = Maps.newLinkedHashMap(); + + final byte[] queryCacheKey = (strategy != null) ? strategy.computeCacheKey(query) : null; for (Interval interval : rewrittenQuery.getIntervals()) { List> serversLookup = timeline.lookup(interval); @@ -146,55 +147,61 @@ public class CachingClusteredClient implements QueryRunner holder.getInterval(), holder.getVersion(), chunk.getChunkNumber() ); - if (queryCacheKey == null) { - final DruidServer server = selector.pick(); - List descriptors = segs.get(server); - - if (descriptors == null) { - descriptors = Lists.newArrayList(); - segs.put(server, descriptors); - } - - descriptors.add(descriptor); - } - else { - final Interval segmentQueryInterval = holder.getInterval(); - final byte[] versionBytes = descriptor.getVersion().getBytes(); - - final byte[] cacheKey = ByteBuffer - .allocate(16 + versionBytes.length + 4 + queryCacheKey.length) - .putLong(segmentQueryInterval.getStartMillis()) - .putLong(segmentQueryInterval.getEndMillis()) - .put(versionBytes) - .putInt(descriptor.getPartitionNumber()) - .put(queryCacheKey) - .array(); - final String segmentIdentifier = selector.getSegment().getIdentifier(); - final Cache cache = cacheBroker.provideCache(segmentIdentifier); - final byte[] cachedValue = cache.get(cacheKey); - - if (useCache && cachedValue != null) { - cachedResults.add(Pair.of(segmentQueryInterval.getStart(), cachedValue)); - } else { - final DruidServer server = selector.pick(); - List descriptors = segs.get(server); - - if (descriptors == null) { - descriptors = Lists.newArrayList(); - segs.put(server, descriptors); - } - - descriptors.add(descriptor); - cachePopulatorMap.put( - String.format("%s_%s", segmentIdentifier, segmentQueryInterval), - new CachePopulator(cache, objectMapper, cacheKey) - ); - } - } + segments.put( + Pair.of(selector, descriptor), + queryCacheKey == null ? null : + computeSegmentCacheKey(selector.getSegment().getIdentifier(), descriptor, queryCacheKey) + ); } } } + Map, byte[]> cachedValues = cacheBroker.getBulk( + Iterables.filter(segments.values(), new Predicate>() + { + @Override + public boolean apply( + @Nullable Pair input + ) + { + return input != null; + } + }) + ); + + for(Pair segment : segments.keySet()) { + Pair segmentCacheKey = segments.get(segment); + + final ServerSelector selector = segment.lhs; + final SegmentDescriptor descriptor = segment.rhs; + final Interval segmentQueryInterval = descriptor.getInterval(); + + final byte[] cachedValue = segmentCacheKey == null ? null : cachedValues.get(segmentCacheKey); + + if (useCache && cachedValue != null) { + cachedResults.add(Pair.of(segmentQueryInterval.getStart(), cachedValue)); + } else { + final DruidServer server = selector.pick(); + List descriptors = serverSegments.get(server); + + if (descriptors == null) { + descriptors = Lists.newArrayList(); + serverSegments.put(server, descriptors); + } + + descriptors.add(descriptor); + + if(segmentCacheKey != null) { + final String segmentIdentifier = selector.getSegment().getIdentifier(); + cachePopulatorMap.put( + String.format("%s_%s", segmentIdentifier, segmentQueryInterval), + new CachePopulator(cacheBroker, objectMapper, segmentCacheKey) + ); + } + } + } + + return new LazySequence( new Supplier>() { @@ -264,7 +271,7 @@ public class CachingClusteredClient implements QueryRunner @SuppressWarnings("unchecked") private void addSequencesFromServer(ArrayList>> listOfSequences) { - for (Map.Entry> entry : segs.entrySet()) { + for (Map.Entry> entry : serverSegments.entrySet()) { final DruidServer server = entry.getKey(); final List descriptors = entry.getValue(); @@ -328,13 +335,29 @@ public class CachingClusteredClient implements QueryRunner ); } + private Pair computeSegmentCacheKey(String segmentIdentifier, SegmentDescriptor descriptor, byte[] queryCacheKey) + { + final Interval segmentQueryInterval = descriptor.getInterval(); + final byte[] versionBytes = descriptor.getVersion().getBytes(); + + return Pair.of( + segmentIdentifier, ByteBuffer + .allocate(16 + versionBytes.length + 4 + queryCacheKey.length) + .putLong(segmentQueryInterval.getStartMillis()) + .putLong(segmentQueryInterval.getEndMillis()) + .put(versionBytes) + .putInt(descriptor.getPartitionNumber()) + .put(queryCacheKey) + ); + } + private static class CachePopulator { - private final Cache cache; + private final CacheBroker cache; private final ObjectMapper mapper; - private final byte[] key; + private final Pair key; - public CachePopulator(Cache cache, ObjectMapper mapper, byte[] key) + public CachePopulator(CacheBroker cache, ObjectMapper mapper, Pair key) { this.cache = cache; this.mapper = mapper; @@ -359,7 +382,7 @@ public class CachingClusteredClient implements QueryRunner offset += array.length; } - cache.put(key, valueBytes); + cache.put(key.lhs, key.rhs.array(), valueBytes); } catch (IOException e) { throw Throwables.propagate(e); diff --git a/client/src/main/java/com/metamx/druid/client/cache/Cache.java b/client/src/main/java/com/metamx/druid/client/cache/Cache.java deleted file mode 100644 index e7907c9548f..00000000000 --- a/client/src/main/java/com/metamx/druid/client/cache/Cache.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.client.cache; - -/** - * An interface to limit the operations that can be done on a Cache so that it is easier to reason about what - * is actually going to be done. - */ -public interface Cache -{ - public byte[] get(byte[] key); - public void put(byte[] key, byte[] value); - public void close(); -} diff --git a/client/src/main/java/com/metamx/druid/client/cache/CacheBroker.java b/client/src/main/java/com/metamx/druid/client/cache/CacheBroker.java index 5f028c06e72..b1dc548a962 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/CacheBroker.java +++ b/client/src/main/java/com/metamx/druid/client/cache/CacheBroker.java @@ -19,10 +19,20 @@ package com.metamx.druid.client.cache; +import com.metamx.common.Pair; + +import java.nio.ByteBuffer; +import java.util.Map; + /** */ public interface CacheBroker { + public byte[] get(String identifier, byte[] key); + public void put(String identifier, byte[] key, byte[] value); + public Map, byte[]> getBulk(Iterable> identifierKeyPairs); + + public void close(String identifier); + public CacheStats getStats(); - public Cache provideCache(String identifier); } diff --git a/client/src/main/java/com/metamx/druid/client/cache/MapCacheBroker.java b/client/src/main/java/com/metamx/druid/client/cache/MapCacheBroker.java index d8ec202021a..85b9156c3fd 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MapCacheBroker.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MapCacheBroker.java @@ -22,6 +22,7 @@ package com.metamx.druid.client.cache; import com.google.common.collect.Maps; import com.google.common.primitives.Ints; import com.metamx.common.ISE; +import com.metamx.common.Pair; import java.nio.ByteBuffer; import java.util.Collections; @@ -34,6 +35,17 @@ import java.util.concurrent.atomic.AtomicLong; */ public class MapCacheBroker implements CacheBroker { + /** + * An interface to limit the operations that can be done on a Cache so that it is easier to reason about what + * is actually going to be done. + */ + public interface Cache + { + public byte[] get(byte[] key); + public void put(byte[] key, byte[] value); + public void close(); + } + private final Map baseMap; private final ByteCountingLRUMap byteCountingLRUMap; @@ -68,7 +80,6 @@ public class MapCacheBroker implements CacheBroker ids = new AtomicInteger(); } - @Override public CacheStats getStats() { @@ -83,7 +94,35 @@ public class MapCacheBroker implements CacheBroker } @Override - public Cache provideCache(final String identifier) + public byte[] get(String identifier, byte[] key) + { + return provideCache(identifier).get(key); + } + + @Override + public void put(String identifier, byte[] key, byte[] value) + { + provideCache(identifier).put(key, value); + } + + @Override + public Map, byte[]> getBulk(Iterable> identifierKeyPairs) + { + Map, byte[]> retVal = Maps.newHashMap(); + + for(Pair e : identifierKeyPairs) { + retVal.put(e, provideCache(e.lhs).get(e.rhs.array())); + } + return retVal; + } + + @Override + public void close(String identifier) + { + provideCache(identifier).close(); + } + + private Cache provideCache(final String identifier) { synchronized (cacheCache) { final Cache cachedCache = cacheCache.get(identifier); diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBroker.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBroker.java index 2f1af877d8c..f2b709de8e5 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBroker.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBroker.java @@ -19,7 +19,11 @@ package com.metamx.druid.client.cache; +import com.google.common.base.Function; import com.google.common.base.Throwables; +import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; +import com.metamx.common.Pair; import net.iharder.base64.Base64; import net.spy.memcached.AddrUtil; import net.spy.memcached.ConnectionFactoryBuilder; @@ -27,9 +31,14 @@ import net.spy.memcached.DefaultHashAlgorithm; import net.spy.memcached.FailureMode; import net.spy.memcached.MemcachedClient; import net.spy.memcached.MemcachedClientIF; +import net.spy.memcached.internal.BulkFuture; import net.spy.memcached.transcoders.SerializingTranscoder; +import javax.annotation.Nullable; import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -94,52 +103,92 @@ public class MemcachedCacheBroker implements CacheBroker } @Override - public Cache provideCache(final String identifier) + public byte[] get(String identifier, byte[] key) { - return new Cache() - { - @Override - public byte[] get(byte[] key) - { - Future future = client.asyncGet(computeKey(identifier, key)); - try { - byte[] bytes = (byte[]) future.get(timeout, TimeUnit.MILLISECONDS); - if(bytes != null) { - hitCount.incrementAndGet(); + Future future = client.asyncGet(computeKey(identifier, key)); + try { + byte[] bytes = (byte[]) future.get(timeout, TimeUnit.MILLISECONDS); + if(bytes != null) { + hitCount.incrementAndGet(); + } + else { + missCount.incrementAndGet(); + } + return bytes; + } + catch(TimeoutException e) { + timeoutCount.incrementAndGet(); + future.cancel(false); + return null; + } + catch(InterruptedException e) { + throw Throwables.propagate(e); + } + catch(ExecutionException e) { + throw Throwables.propagate(e); + } + } + + @Override + public void put(String identifier, byte[] key, byte[] value) + { + client.set(computeKey(identifier, key), expiration, value); + } + + @Override + public Map, byte[]> getBulk(Iterable> identifierKeyPairs) + { + Map> keyLookup = Maps.uniqueIndex( + identifierKeyPairs, + new Function, String>() + { + @Override + public String apply( + @Nullable Pair input + ) + { + return computeKey(input.lhs, input.rhs.array()); } - else { - missCount.incrementAndGet(); - } - return bytes; - } - catch(TimeoutException e) { - timeoutCount.incrementAndGet(); - future.cancel(false); - return null; - } - catch(InterruptedException e) { - throw Throwables.propagate(e); - } - catch(ExecutionException e) { - throw Throwables.propagate(e); } + ); + + BulkFuture> future = client.asyncGetBulk(keyLookup.keySet()); + + try { + Map some = future.getSome(timeout, TimeUnit.MILLISECONDS); + + if(future.isTimeout()) { + future.cancel(false); + timeoutCount.incrementAndGet(); + } + missCount.addAndGet(keyLookup.size() - some.size()); + hitCount.addAndGet(some.size()); + + Map, byte[]> results = Maps.newHashMap(); + for(Map.Entry entry : some.entrySet()) { + results.put( + keyLookup.get(entry.getKey()), + (byte[])entry.getValue() + ); } - @Override - public void put(byte[] key, byte[] value) - { - client.set(computeKey(identifier, key), expiration, value); - } + return results; + } + catch(InterruptedException e) { + throw Throwables.propagate(e); + } + catch(ExecutionException e) { + throw Throwables.propagate(e); + } + } - @Override - public void close() - { - // no resources to cleanup - } - }; + @Override + public void close(String identifier) + { + // no resources to cleanup } private String computeKey(String identifier, byte[] key) { - return identifier + Base64.encodeBytes(key, Base64.DONT_BREAK_LINES); + return identifier + ":" + Base64.encodeBytes(key, Base64.DONT_BREAK_LINES); } } diff --git a/client/src/test/java/com/metamx/druid/client/cache/MapCacheBrokerTest.java b/client/src/test/java/com/metamx/druid/client/cache/MapCacheBrokerTest.java index 4338a23a10d..2041e2dca05 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MapCacheBrokerTest.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MapCacheBrokerTest.java @@ -31,56 +31,53 @@ public class MapCacheBrokerTest private static final byte[] HI = "hi".getBytes(); private static final byte[] HO = "ho".getBytes(); private ByteCountingLRUMap baseMap; - private MapCacheBroker broker; + private MapCacheBroker cache; @Before public void setUp() throws Exception { baseMap = new ByteCountingLRUMap(1024 * 1024); - broker = new MapCacheBroker(baseMap); + cache = new MapCacheBroker(baseMap); } @Test public void testSanity() throws Exception { - Cache aCache = broker.provideCache("a"); - Cache theCache = broker.provideCache("the"); - - Assert.assertNull(aCache.get(HI)); + Assert.assertNull(cache.get("a", HI)); Assert.assertEquals(0, baseMap.size()); - put(aCache, HI, 1); + put(cache, "a", HI, 1); Assert.assertEquals(1, baseMap.size()); - Assert.assertEquals(1, get(aCache, HI)); - Assert.assertNull(theCache.get(HI)); + Assert.assertEquals(1, get(cache, "a", HI)); + Assert.assertNull(cache.get("the", HI)); - put(theCache, HI, 2); + put(cache, "the", HI, 2); Assert.assertEquals(2, baseMap.size()); - Assert.assertEquals(1, get(aCache, HI)); - Assert.assertEquals(2, get(theCache, HI)); + Assert.assertEquals(1, get(cache, "a", HI)); + Assert.assertEquals(2, get(cache, "the", HI)); - put(theCache, HO, 10); + put(cache, "the", HO, 10); Assert.assertEquals(3, baseMap.size()); - Assert.assertEquals(1, get(aCache, HI)); - Assert.assertNull(aCache.get(HO)); - Assert.assertEquals(2, get(theCache, HI)); - Assert.assertEquals(10, get(theCache, HO)); + Assert.assertEquals(1, get(cache, "a", HI)); + Assert.assertNull(cache.get("a", HO)); + Assert.assertEquals(2, get(cache, "the", HI)); + Assert.assertEquals(10, get(cache, "the", HO)); - theCache.close(); + cache.close("the"); Assert.assertEquals(1, baseMap.size()); - Assert.assertEquals(1, get(aCache, HI)); - Assert.assertNull(aCache.get(HO)); + Assert.assertEquals(1, get(cache, "a", HI)); + Assert.assertNull(cache.get("a", HO)); - aCache.close(); + cache.close("a"); Assert.assertEquals(0, baseMap.size()); } - public void put(Cache cache, byte[] key, Integer value) + public void put(CacheBroker cache, String identifier, byte[] key, Integer value) { - cache.put(key, Ints.toByteArray(value)); + cache.put(identifier, key, Ints.toByteArray(value)); } - public int get(Cache cache, byte[] key) + public int get(CacheBroker cache, String identifier, byte[] key) { - return Ints.fromByteArray(cache.get(key)); + return Ints.fromByteArray(cache.get(identifier, key)); } } diff --git a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java index d1839098e8b..1aa5ed205e7 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java @@ -3,6 +3,8 @@ package com.metamx.druid.client.cache; import com.google.caliper.Param; import com.google.caliper.Runner; import com.google.caliper.SimpleBenchmark; +import com.google.common.collect.Lists; +import com.metamx.common.Pair; import net.spy.memcached.AddrUtil; import net.spy.memcached.ConnectionFactoryBuilder; import net.spy.memcached.DefaultHashAlgorithm; @@ -11,17 +13,20 @@ import net.spy.memcached.MemcachedClient; import net.spy.memcached.MemcachedClientIF; import net.spy.memcached.transcoders.SerializingTranscoder; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; import java.util.Random; import java.util.concurrent.TimeUnit; public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark { private static final String BASE_KEY = "test_2012-11-26T00:00:00.000Z_2012-11-27T00:00:00.000Z_2012-11-27T04:11:25.979Z_"; + public static final String IDENTIFIER = "default"; - private MemcachedCacheBroker broker; + private MemcachedCacheBroker cache; private MemcachedClientIF client; - private Cache cache; private static byte[] randBytes; @Param({"localhost:11211"}) String hosts; @@ -53,14 +58,12 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark AddrUtil.getAddresses(hosts) ); - broker = new MemcachedCacheBroker( + cache = new MemcachedCacheBroker( client, 500, // 500 milliseconds 3600 // 1 hour ); - cache = broker.provideCache("default"); - randBytes = new byte[objectSize * 1024]; new Random(0).nextBytes(randBytes); @@ -77,7 +80,7 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark for(int i = 0; i < reps; ++i) { for(int k = 0; k < objectCount; ++k) { String key = BASE_KEY + i; - cache.put(key.getBytes(), randBytes); + cache.put(IDENTIFIER, key.getBytes(), randBytes); } // make sure the write queue is empty client.waitForQueues(1, TimeUnit.HOURS); @@ -89,14 +92,28 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark long count = 0; for (int i = 0; i < reps; i++) { for(int k = 0; k < objectCount; ++k) { - String key = BASE_KEY + i; - bytes = cache.get(key.getBytes()); + String key = BASE_KEY + k; + bytes = cache.get(IDENTIFIER, key.getBytes()); count += bytes.length; } } return count; } + public long timeBulkGetObjects(int reps) { + long count = 0; + for (int i = 0; i < reps; i++) { + List> keys = Lists.newArrayList(); + for(int k = 0; k < objectCount; ++k) { + String key = BASE_KEY + k; + keys.add(Pair.of(IDENTIFIER, ByteBuffer.wrap(key.getBytes()))); + } + Map, byte[]> results = cache.getBulk(keys); + for(byte[] bytes : results.values()) count += bytes.length; + } + return count; + } + public static void main(String[] args) throws Exception { Runner.main(MemcachedCacheBrokerBenchmark.class, args); } diff --git a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java index 3cb2dba09b8..9bfb928a949 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java @@ -51,51 +51,48 @@ public class MemcachedCacheBrokerTest { private static final byte[] HI = "hi".getBytes(); private static final byte[] HO = "ho".getBytes(); - private MemcachedCacheBroker broker; + private MemcachedCacheBroker cache; @Before public void setUp() throws Exception { MemcachedClientIF client = new MockMemcachedClient(); - broker = new MemcachedCacheBroker(client, 500, 3600); + cache = new MemcachedCacheBroker(client, 500, 3600); } @Test public void testSanity() throws Exception { - Cache aCache = broker.provideCache("a"); - Cache theCache = broker.provideCache("the"); + Assert.assertNull(cache.get("a", HI)); + put(cache, "a", HI, 1); + Assert.assertEquals(1, get(cache, "a", HI)); + Assert.assertNull(cache.get("the", HI)); - Assert.assertNull(aCache.get(HI)); - put(aCache, HI, 1); - Assert.assertEquals(1, get(aCache, HI)); - Assert.assertNull(theCache.get(HI)); + put(cache, "the", HI, 2); + Assert.assertEquals(1, get(cache, "a", HI)); + Assert.assertEquals(2, get(cache, "the", HI)); - put(theCache, HI, 2); - Assert.assertEquals(1, get(aCache, HI)); - Assert.assertEquals(2, get(theCache, HI)); + put(cache, "the", HO, 10); + Assert.assertEquals(1, get(cache, "a", HI)); + Assert.assertNull(cache.get("a", HO)); + Assert.assertEquals(2, get(cache, "the", HI)); + Assert.assertEquals(10, get(cache, "the", HO)); - put(theCache, HO, 10); - Assert.assertEquals(1, get(aCache, HI)); - Assert.assertNull(aCache.get(HO)); - Assert.assertEquals(2, get(theCache, HI)); - Assert.assertEquals(10, get(theCache, HO)); + cache.close("the"); + Assert.assertEquals(1, get(cache, "a", HI)); + Assert.assertNull(cache.get("a", HO)); - theCache.close(); - Assert.assertEquals(1, get(aCache, HI)); - Assert.assertNull(aCache.get(HO)); - - aCache.close(); + cache.close("a"); } - public void put(Cache cache, byte[] key, Integer value) + public void put(CacheBroker cache, String identifier, byte[] key, Integer value) { - cache.put(key, Ints.toByteArray(value)); + cache.put(identifier, key, Ints.toByteArray(value)); } - public int get(Cache cache, byte[] key) + public int get(CacheBroker cache, String identifier, byte[] key) { - return Ints.fromByteArray(cache.get(key)); + return Ints.fromByteArray(cache.get(identifier, key)); } } From c8cb96b00654d9de38e9bf4e28675984b20ae57f Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Wed, 16 Jan 2013 17:10:33 -0600 Subject: [PATCH 28/92] 1) Remove vast majority of usages of IndexIO.mapDir() and deprecated it. IndexIO.loadIndex() is the new IndexIO.mapDir() 2) Fix bug with IndexMerger and null columns 3) Add QueryableIndexIndexableAdapter so that QueryableIndexes can be merged 4) Adjust twitter example to have multiple values for each hash tag 5) Adjusted GroupByQueryEngine to just drop dimensions that don't exist instead of throwing an NPE --- examples/twitter/group_by_query.body | 16 +- .../TwitterSpritzerFirehoseFactory.java | 47 +-- examples/twitter/twitter_realtime.spec | 4 +- .../metamx/druid/index/QueryableIndex.java | 1 + .../druid/index/SimpleQueryableIndex.java | 6 + .../com/metamx/druid/index/column/Column.java | 1 + .../druid/index/column/ComplexColumn.java | 1 + .../druid/index/column/ComplexColumnImpl.java | 12 +- .../index/column/DictionaryEncodedColumn.java | 1 + .../druid/index/column/FloatColumn.java | 13 +- .../druid/index/column/GenericColumn.java | 2 + .../index/column/IndexedComplexColumn.java | 10 +- .../column/IndexedFloatsGenericColumn.java | 12 + .../column/IndexedLongsGenericColumn.java | 12 + .../metamx/druid/index/column/LongColumn.java | 13 +- .../druid/index/column/SimpleColumn.java | 14 + .../column/SimpleDictionaryEncodedColumn.java | 8 +- .../index/column/StringMultiValueColumn.java | 12 + .../serde/ComplexColumnPartSupplier.java | 5 +- .../v1/CompressedFloatsIndexedSupplier.java | 5 + .../druid/index/v1/IncrementalIndex.java | 2 +- .../com/metamx/druid/index/v1/IndexIO.java | 65 ++++- .../druid/indexer/IndexGeneratorJob.java | 8 +- .../common/index/YeOldePlumberSchool.java | 10 +- .../druid/merger/common/task/AppendTask.java | 6 +- .../merger/common/task/DefaultMergeTask.java | 10 +- .../druid/realtime/RealtimePlumberSchool.java | 23 +- .../v1/ComplexMetricColumnSerializer.java | 16 +- .../index/v1/FloatMetricColumnSerializer.java | 35 +-- .../metamx/druid/index/v1/IndexMerger.java | 61 ++-- .../index/v1/MMappedIndexQueryableIndex.java | 8 +- .../v1/QueryableIndexIndexableAdapter.java | 267 ++++++++++++++++++ .../druid/query/group/GroupByQueryEngine.java | 5 +- .../metamx/druid/index/v1/EmptyIndexTest.java | 12 +- .../druid/index/v1/IndexMergerTest.java | 27 +- .../com/metamx/druid/index/v1/TestIndex.java | 4 +- 36 files changed, 570 insertions(+), 184 deletions(-) create mode 100644 server/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java diff --git a/examples/twitter/group_by_query.body b/examples/twitter/group_by_query.body index 4ea7806e94c..e0607aa1554 100644 --- a/examples/twitter/group_by_query.body +++ b/examples/twitter/group_by_query.body @@ -2,19 +2,11 @@ "queryType": "groupBy", "dataSource": "twitterstream", "granularity": "all", - "dimensions": ["lang"], + "dimensions": ["lang", "utc_offset"], "aggregations":[ - { "type": "count", "name": "rows"}, - { "type": "doubleSum", "fieldName": "tweets", "name": "tweets"}, - - { "type": "max", "fieldName": "max_statuses_count", "name": "theMaxStatusesCount"}, - { "type": "max", "fieldName": "max_retweet_count", "name": "theMaxRetweetCount"}, - - { "type": "max", "fieldName": "max_friends_count", "name": "theMaxFriendsCount"}, - { "type": "max", "fieldName": "max_follower_count", "name": "theMaxFollowerCount"}, - - { "type": "doubleSum", "fieldName": "total_statuses_count", "name": "total_tweets_all_time"} - + { "type": "count", "name": "rows"}, + { "type": "doubleSum", "fieldName": "tweets", "name": "tweets"} ], + "filter": { "type": "selector", "dimension": "lang", "value": "en" }, "intervals":["2012-10-01T00:00/2020-01-01T00"] } diff --git a/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java b/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java index 826218afbd8..992cd239487 100644 --- a/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java +++ b/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java @@ -1,25 +1,34 @@ package druid.examples.twitter; +import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.realtime.Firehose; import com.metamx.druid.realtime.FirehoseFactory; import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonTypeName; import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.map.ObjectMapper; -import twitter4j.*; +import org.codehaus.jackson.annotate.JsonTypeName; +import twitter4j.ConnectionLifeCycleListener; +import twitter4j.HashtagEntity; +import twitter4j.Status; +import twitter4j.StatusDeletionNotice; +import twitter4j.StatusListener; +import twitter4j.TwitterStream; +import twitter4j.TwitterStreamFactory; +import twitter4j.User; import java.io.IOException; +import java.util.Arrays; import java.util.HashMap; import java.util.LinkedList; +import java.util.List; import java.util.Map; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.TimeUnit; -import static java.lang.Thread.*; +import static java.lang.Thread.sleep; /** @@ -241,30 +250,26 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory { } catch (InterruptedException e) { throw new RuntimeException("InterruptedException", e); } - //log.info("twitterStatus: "+ status.getCreatedAt() + " @" + status.getUser().getScreenName() + " - " + status.getText());//DEBUG - - // theMap.put("twid", status.getUser().getScreenName()); - // theMap.put("msg", status.getText()); // ToDo: verify encoding HashtagEntity[] hts = status.getHashtagEntities(); if (hts != null && hts.length > 0) { - // ToDo: get all the hash tags instead of just the first one - theMap.put("htags", hts[0].getText()); - } else { - theMap.put("htags", null); + List hashTags = Lists.newArrayListWithExpectedSize(hts.length); + for (HashtagEntity ht : hts) { + hashTags.add(ht.getText()); + } + + theMap.put("htags", Arrays.asList(hashTags.get(0))); } long retweetCount = status.getRetweetCount(); theMap.put("retweet_count", retweetCount); - User u = status.getUser(); - if (u != null) { - theMap.put("follower_count", u.getFollowersCount()); - theMap.put("friends_count", u.getFriendsCount()); - theMap.put("lang", u.getLang()); - theMap.put("utc_offset", u.getUtcOffset()); // resolution in seconds, -1 if not available? - theMap.put("statuses_count", u.getStatusesCount()); - } else { - log.error("status.getUser() is null"); + User user = status.getUser(); + if (user != null) { + theMap.put("follower_count", user.getFollowersCount()); + theMap.put("friends_count", user.getFriendsCount()); + theMap.put("lang", user.getLang()); + theMap.put("utc_offset", user.getUtcOffset()); // resolution in seconds, -1 if not available? + theMap.put("statuses_count", user.getStatusesCount()); } return new MapBasedInputRow(status.getCreatedAt().getTime(), dimensions, theMap); diff --git a/examples/twitter/twitter_realtime.spec b/examples/twitter/twitter_realtime.spec index 14d34421c6b..00b1707028d 100644 --- a/examples/twitter/twitter_realtime.spec +++ b/examples/twitter/twitter_realtime.spec @@ -31,8 +31,8 @@ "firehose": { "type": "twitzer", - "maxEventCount": 50000, - "maxRunMinutes": 10 + "maxEventCount": 500000, + "maxRunMinutes": 120 }, "plumber": { diff --git a/index-common/src/main/java/com/metamx/druid/index/QueryableIndex.java b/index-common/src/main/java/com/metamx/druid/index/QueryableIndex.java index 0c4f57cf9b9..82cee9e54dd 100644 --- a/index-common/src/main/java/com/metamx/druid/index/QueryableIndex.java +++ b/index-common/src/main/java/com/metamx/druid/index/QueryableIndex.java @@ -28,6 +28,7 @@ import org.joda.time.Interval; public interface QueryableIndex extends ColumnSelector { public Interval getDataInterval(); + public int getNumRows(); public Indexed getColumnNames(); public Indexed getAvailableDimensions(); } diff --git a/index-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java b/index-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java index 7e8a6ba7635..2f60b73adc6 100644 --- a/index-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java +++ b/index-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java @@ -56,6 +56,12 @@ public class SimpleQueryableIndex implements QueryableIndex return dataInterval; } + @Override + public int getNumRows() + { + return timeColumn.getLength(); + } + @Override public Indexed getColumnNames() { diff --git a/index-common/src/main/java/com/metamx/druid/index/column/Column.java b/index-common/src/main/java/com/metamx/druid/index/column/Column.java index 5702b42206e..fa418a3398a 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/Column.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/Column.java @@ -25,6 +25,7 @@ public interface Column { public ColumnCapabilities getCapabilities(); + public int getLength(); public DictionaryEncodedColumn getDictionaryEncoding(); public RunLengthColumn getRunLengthColumn(); public GenericColumn getGenericColumn(); diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ComplexColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/ComplexColumn.java index 7d21987f6f9..f7cfb706e44 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/ComplexColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/ComplexColumn.java @@ -26,5 +26,6 @@ import java.io.Closeable; public interface ComplexColumn extends Closeable { public Class getClazz(); + public String getTypeName(); public Object getRowValue(int rowNum); } diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java b/index-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java index 8f9a302adc5..46f665c57c6 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java @@ -29,10 +29,12 @@ public class ComplexColumnImpl extends AbstractColumn .setType(ValueType.COMPLEX); private final Indexed column; + private final String typeName; - public ComplexColumnImpl(Indexed column) + public ComplexColumnImpl(String typeName, Indexed column) { this.column = column; + this.typeName = typeName; } @Override @@ -41,9 +43,15 @@ public class ComplexColumnImpl extends AbstractColumn return CAPABILITIES; } + @Override + public int getLength() + { + return column.size(); + } + @Override public ComplexColumn getComplexColumn() { - return new IndexedComplexColumn(column); + return new IndexedComplexColumn(typeName, column); } } diff --git a/index-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java index e2bd177a25c..9301734f970 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java @@ -26,6 +26,7 @@ import com.metamx.druid.kv.IndexedInts; public interface DictionaryEncodedColumn { public int size(); + public boolean hasMultipleValues(); public int getSingleValueRow(int rowNum); public IndexedInts getMultiValueRow(int rowNum); public String lookupName(int id); diff --git a/index-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java index 491baa3c41d..44ffd7e970a 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java @@ -19,8 +19,7 @@ package com.metamx.druid.index.column; -import com.google.common.base.Supplier; -import com.metamx.druid.kv.IndexedFloats; +import com.metamx.druid.index.v1.CompressedFloatsIndexedSupplier; /** */ @@ -29,9 +28,9 @@ public class FloatColumn extends AbstractColumn private static final ColumnCapabilitiesImpl CAPABILITIES = new ColumnCapabilitiesImpl() .setType(ValueType.FLOAT); - private final Supplier column; + private final CompressedFloatsIndexedSupplier column; - public FloatColumn(Supplier column) + public FloatColumn(CompressedFloatsIndexedSupplier column) { this.column = column; } @@ -42,6 +41,12 @@ public class FloatColumn extends AbstractColumn return CAPABILITIES; } + @Override + public int getLength() + { + return column.size(); + } + @Override public GenericColumn getGenericColumn() { diff --git a/index-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java index 58ea4b86d98..c41b4906bd1 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java @@ -30,6 +30,8 @@ import java.io.Closeable; public interface GenericColumn extends Closeable { public int size(); + public ValueType getType(); + public boolean hasMultipleValues(); public String getStringSingleValueRow(int rowNum); public Indexed getStringMultiValueRow(int rowNum); diff --git a/index-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java index 192734aeec2..bafb6977dd0 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java @@ -28,12 +28,14 @@ import java.io.IOException; public class IndexedComplexColumn implements ComplexColumn { private final Indexed column; + private final String typeName; public IndexedComplexColumn( - Indexed column + String typeName, Indexed column ) { this.column = column; + this.typeName = typeName; } @Override public Class getClazz() @@ -41,6 +43,12 @@ public class IndexedComplexColumn implements ComplexColumn return column.getClazz(); } + @Override + public String getTypeName() + { + return typeName; + } + @Override public Object getRowValue(int rowNum) { diff --git a/index-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java index 815ff84c583..423b046e106 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java @@ -43,6 +43,18 @@ public class IndexedFloatsGenericColumn implements GenericColumn return column.size(); } + @Override + public ValueType getType() + { + return ValueType.FLOAT; + } + + @Override + public boolean hasMultipleValues() + { + return false; + } + @Override public String getStringSingleValueRow(int rowNum) { diff --git a/index-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java index 01ff6063870..0e96a63924b 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java @@ -43,6 +43,18 @@ public class IndexedLongsGenericColumn implements GenericColumn return column.size(); } + @Override + public ValueType getType() + { + return ValueType.LONG; + } + + @Override + public boolean hasMultipleValues() + { + return false; + } + @Override public String getStringSingleValueRow(int rowNum) { diff --git a/index-common/src/main/java/com/metamx/druid/index/column/LongColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/LongColumn.java index 76d74ba377e..1ec297ea61c 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/LongColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/LongColumn.java @@ -19,8 +19,7 @@ package com.metamx.druid.index.column; -import com.google.common.base.Supplier; -import com.metamx.druid.kv.IndexedLongs; +import com.metamx.druid.index.v1.CompressedLongsIndexedSupplier; /** */ @@ -29,9 +28,9 @@ public class LongColumn extends AbstractColumn private static final ColumnCapabilitiesImpl CAPABILITIES = new ColumnCapabilitiesImpl() .setType(ValueType.LONG); - private final Supplier column; + private final CompressedLongsIndexedSupplier column; - public LongColumn(Supplier column) + public LongColumn(CompressedLongsIndexedSupplier column) { this.column = column; } @@ -42,6 +41,12 @@ public class LongColumn extends AbstractColumn return CAPABILITIES; } + @Override + public int getLength() + { + return column.size(); + } + @Override public GenericColumn getGenericColumn() { diff --git a/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java index 2d3fae51abf..a3884203227 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java @@ -20,6 +20,7 @@ package com.metamx.druid.index.column; import com.google.common.base.Supplier; +import com.google.common.io.Closeables; /** */ @@ -55,6 +56,19 @@ class SimpleColumn implements Column return capabilities; } + @Override + public int getLength() + { + GenericColumn column = null; + try { + column = genericColumn.get(); + return column.size(); + } + finally { + Closeables.closeQuietly(column); + } + } + @Override public DictionaryEncodedColumn getDictionaryEncoding() { diff --git a/index-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java index 87bc5bb6a2d..7a28a53b0af 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java @@ -46,7 +46,13 @@ public class SimpleDictionaryEncodedColumn implements DictionaryEncodedColumn @Override public int size() { - return column == null ? multiValueColumn.size() : column.size(); + return hasMultipleValues() ? multiValueColumn.size() : column.size(); + } + + @Override + public boolean hasMultipleValues() + { + return column == null; } @Override diff --git a/index-common/src/main/java/com/metamx/druid/index/column/StringMultiValueColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/StringMultiValueColumn.java index 7d52e42ba4f..053bcee1956 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/StringMultiValueColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/StringMultiValueColumn.java @@ -55,6 +55,12 @@ public class StringMultiValueColumn extends AbstractColumn return CAPABILITIES; } + @Override + public int getLength() + { + return column.size(); + } + @Override public DictionaryEncodedColumn getDictionaryEncoding() { @@ -66,6 +72,12 @@ public class StringMultiValueColumn extends AbstractColumn return column.size(); } + @Override + public boolean hasMultipleValues() + { + return true; + } + @Override public int getSingleValueRow(int rowNum) { diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java b/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java index cb1a4bea395..4a4cfce4f77 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java @@ -32,8 +32,7 @@ public class ComplexColumnPartSupplier implements Supplier private final String typeName; public ComplexColumnPartSupplier( - final GenericIndexed complexType, - final String typeName + final String typeName, final GenericIndexed complexType ) { this.complexType = complexType; this.typeName = typeName; @@ -42,6 +41,6 @@ public class ComplexColumnPartSupplier implements Supplier @Override public ComplexColumn get() { - return new IndexedComplexColumn(complexType); + return new IndexedComplexColumn(typeName, complexType); } } diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java b/index-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java index 2ff8f747adc..1def2af031f 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java @@ -59,6 +59,11 @@ public class CompressedFloatsIndexedSupplier implements Supplier this.baseFloatBuffers = baseFloatBuffers; } + public int size() + { + return totalSize; + } + @Override public IndexedFloats get() { diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java b/index-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java index 31ed9efbbed..624d6d4b375 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java @@ -524,7 +524,7 @@ public class IncrementalIndex implements Iterable public String get(String value) { - return poorMansInterning.get(value); + return value == null ? null : poorMansInterning.get(value); } public int getId(String value) diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index 6836b9233c5..afedbb6a742 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -21,6 +21,7 @@ package com.metamx.druid.index.v1; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -58,6 +59,7 @@ import com.metamx.druid.kv.IndexedIterable; import com.metamx.druid.kv.VSizeIndexed; import com.metamx.druid.kv.VSizeIndexedInts; import com.metamx.druid.utils.SerializerUtils; +import it.uniroma3.mat.extendedset.intset.ConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.Interval; @@ -70,6 +72,7 @@ import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.AbstractList; +import java.util.Arrays; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; @@ -99,7 +102,7 @@ public class IndexIO private static final Logger log = new Logger(IndexIO.class); private static final SerializerUtils serializerUtils = new SerializerUtils(); - private static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder(); + public static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder(); // This should really be provided by DI, should be changed once we switch around to using a DI framework private static final ObjectMapper mapper = new DefaultObjectMapper(); @@ -120,6 +123,7 @@ public class IndexIO return handler.canBeMapped(inDir); } + @Deprecated public static MMappedIndex mapDir(final File inDir) throws IOException { init(); @@ -383,22 +387,70 @@ public class IndexIO serializerUtils.writeString(nameBAOS, dimension); outParts.add(ByteBuffer.wrap(nameBAOS.toByteArray())); - final GenericIndexed dictionary = GenericIndexed.read( + GenericIndexed dictionary = GenericIndexed.read( dimBuffer, GenericIndexed.stringStrategy ); + VSizeIndexedInts singleValCol = null; VSizeIndexed multiValCol = VSizeIndexed.readFromByteBuffer(dimBuffer.asReadOnlyBuffer()); + GenericIndexed bitmaps = bitmapIndexes.get(dimension); + boolean onlyOneValue = true; - for (VSizeIndexedInts rowValue : multiValCol) { + ConciseSet nullsSet = null; + for (int i = 0; i < multiValCol.size(); ++i) { + VSizeIndexedInts rowValue = multiValCol.get(i); if (!onlyOneValue) { break; } if (rowValue.size() > 1) { onlyOneValue = false; } + if (rowValue.size() == 0) { + if (nullsSet == null) { + nullsSet = new ConciseSet(); + } + nullsSet.add(i); + } } if (onlyOneValue) { + log.info("Dimension[%s] is single value, converting...", dimension); + final boolean bumpedDictionary; + if (nullsSet != null) { + log.info("Dimension[%s] has null rows.", dimension); + final ImmutableConciseSet theNullSet = ImmutableConciseSet.newImmutableFromMutable(nullsSet); + + if (dictionary.get(0) != null) { + log.info("Dimension[%s] has no null value in the dictionary, expanding...", dimension); + bumpedDictionary = true; + final List nullList = Lists.newArrayList(); + nullList.add(null); + + dictionary = GenericIndexed.fromIterable( + Iterables.concat(nullList, dictionary), + GenericIndexed.stringStrategy + ); + + bitmaps = GenericIndexed.fromIterable( + Iterables.concat(Arrays.asList(theNullSet), bitmaps), + ConciseCompressedIndexedInts.objectStrategy + ); + } + else { + bumpedDictionary = false; + bitmaps = GenericIndexed.fromIterable( + Iterables.concat( + Arrays.asList(ImmutableConciseSet.union(theNullSet, bitmaps.get(0))), + Iterables.skip(bitmaps, 1) + ), + ConciseCompressedIndexedInts.objectStrategy + ); + } + } + else { + bumpedDictionary = false; + } + final VSizeIndexed finalMultiValCol = multiValCol; singleValCol = VSizeIndexedInts.fromList( new AbstractList() @@ -406,7 +458,8 @@ public class IndexIO @Override public Integer get(int index) { - return finalMultiValCol.get(index).get(0); + final VSizeIndexedInts ints = finalMultiValCol.get(index); + return ints.size() == 0 ? 0 : ints.get(0) + (bumpedDictionary ? 1 : 0); } @Override @@ -423,7 +476,7 @@ public class IndexIO } builder.addSerde( - new DictionaryEncodedColumnPartSerde(dictionary, singleValCol, multiValCol, bitmapIndexes.get(dimension)) + new DictionaryEncodedColumnPartSerde(dictionary, singleValCol, multiValCol, bitmaps) ); final ColumnDescriptor serdeficator = builder.build(); @@ -587,7 +640,7 @@ public class IndexIO .setType(ValueType.COMPLEX) .setComplexColumn( new ComplexColumnPartSupplier( - (GenericIndexed) metricHolder.complexType, metricHolder.getTypeName() + metricHolder.getTypeName(), (GenericIndexed) metricHolder.complexType ) ) .build() diff --git a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java index 34d743fc9be..28dacd1ca9a 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java @@ -38,10 +38,10 @@ import com.metamx.common.parsers.Parser; import com.metamx.common.parsers.ParserUtils; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; -import com.metamx.druid.index.v1.MMappedIndex; import com.metamx.druid.indexer.rollup.DataRollupSpec; import com.metamx.druid.input.MapBasedInputRow; import org.apache.commons.io.FileUtils; @@ -359,7 +359,7 @@ public class IndexGeneratorJob implements Jobby log.info("%,d lines completed.", lineCount); - List indexes = Lists.newArrayListWithCapacity(indexCount); + List indexes = Lists.newArrayListWithCapacity(indexCount); final File mergedBase; if (toMerge.size() == 0) { @@ -389,9 +389,9 @@ public class IndexGeneratorJob implements Jobby toMerge.add(finalFile); for (File file : toMerge) { - indexes.add(IndexIO.mapDir(file)); + indexes.add(IndexIO.loadIndex(file)); } - mergedBase = IndexMerger.mergeMMapped( + mergedBase = IndexMerger.mergeQueryableIndex( indexes, aggs, new File(baseFlushFile, "merged"), new IndexMerger.ProgressIndicator() { @Override diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java index a496f560970..345af0207ba 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java @@ -27,16 +27,16 @@ import com.google.common.collect.Sets; import com.metamx.common.logger.Logger; import com.metamx.druid.Query; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; -import com.metamx.druid.index.v1.MMappedIndex; +import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.realtime.FireDepartmentMetrics; import com.metamx.druid.realtime.FireHydrant; import com.metamx.druid.realtime.Plumber; import com.metamx.druid.realtime.PlumberSchool; import com.metamx.druid.realtime.Schema; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.realtime.Sink; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; @@ -129,13 +129,13 @@ public class YeOldePlumberSchool implements PlumberSchool } else if(spilled.size() == 1) { fileToUpload = Iterables.getOnlyElement(spilled); } else { - List indexes = Lists.newArrayList(); + List indexes = Lists.newArrayList(); for (final File oneSpill : spilled) { - indexes.add(IndexIO.mapDir(oneSpill)); + indexes.add(IndexIO.loadIndex(oneSpill)); } fileToUpload = new File(tmpSegmentDir, "merged"); - IndexMerger.mergeMMapped(indexes, schema.getAggregators(), fileToUpload); + IndexMerger.mergeQueryableIndex(indexes, schema.getAggregators(), fileToUpload); } final DataSegment segmentToUpload = theSink.getSegment().withVersion(version); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java index f9898070aa1..f1153e5c43c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java @@ -30,7 +30,7 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; import com.metamx.druid.index.v1.IndexableAdapter; -import com.metamx.druid.index.v1.MMappedIndexAdapter; +import com.metamx.druid.index.v1.QueryableIndexIndexableAdapter; import com.metamx.druid.index.v1.Rowboat; import com.metamx.druid.index.v1.RowboatFilteringIndexAdapter; import org.codehaus.jackson.annotate.JsonCreator; @@ -90,8 +90,8 @@ public class AppendTask extends MergeTask for (final SegmentToMergeHolder holder : segmentsToMerge) { adapters.add( new RowboatFilteringIndexAdapter( - new MMappedIndexAdapter( - IndexIO.mapDir(holder.getFile()) + new QueryableIndexIndexableAdapter( + IndexIO.loadIndex(holder.getFile()) ), new Predicate() { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java index 6dfc95c2271..e17db4b980e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java @@ -25,9 +25,9 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; -import com.metamx.druid.index.v1.MMappedIndex; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; @@ -57,16 +57,16 @@ public class DefaultMergeTask extends MergeTask public File merge(final Map segments, final File outDir) throws Exception { - return IndexMerger.mergeMMapped( + return IndexMerger.mergeQueryableIndex( Lists.transform( ImmutableList.copyOf(segments.values()), - new Function() + new Function() { @Override - public MMappedIndex apply(@Nullable File input) + public QueryableIndex apply(@Nullable File input) { try { - return IndexIO.mapDir(input); + return IndexIO.loadIndex(input); } catch (Exception e) { throw Throwables.propagate(e); diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java index 603b9ee1a36..02bd6acbb4c 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java @@ -308,30 +308,21 @@ public class RealtimePlumberSchool implements PlumberSchool final File mergedFile; try { - List indexes = Lists.newArrayList(); + List indexes = Lists.newArrayList(); for (FireHydrant fireHydrant : sink) { Segment segment = fireHydrant.getSegment(); final QueryableIndex queryableIndex = segment.asQueryableIndex(); - if (queryableIndex instanceof MMappedIndexQueryableIndex) { - log.info("Adding hydrant[%s]", fireHydrant); - indexes.add(((MMappedIndexQueryableIndex) queryableIndex).getIndex()); - } - else { - log.makeAlert("[%s] Failure to merge-n-push", schema.getDataSource()) - .addData("type", "Unknown segment type") - .addData("adapterClass", segment.getClass().toString()) - .emit(); - return; - } + log.info("Adding hydrant[%s]", fireHydrant); + indexes.add(queryableIndex); } - mergedFile = IndexMerger.mergeMMapped( + mergedFile = IndexMerger.mergeQueryableIndex( indexes, schema.getAggregators(), new File(computePersistDir(schema, interval), "merged") ); - MMappedIndex index = IndexIO.mapDir(mergedFile); + QueryableIndex index = IndexIO.loadIndex(mergedFile); DataSegment segment = segmentPusher.push( mergedFile, @@ -503,9 +494,7 @@ public class RealtimePlumberSchool implements PlumberSchool new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())) ); - indexToPersist.swapSegment( - new QueryableIndexSegment(null, new MMappedIndexQueryableIndex(IndexIO.mapDir(persistedFile))) - ); + indexToPersist.swapSegment(new QueryableIndexSegment(null, IndexIO.loadIndex(persistedFile))); return numRows; } diff --git a/server/src/main/java/com/metamx/druid/index/v1/ComplexMetricColumnSerializer.java b/server/src/main/java/com/metamx/druid/index/v1/ComplexMetricColumnSerializer.java index 526877176a9..d09581c1c5b 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/ComplexMetricColumnSerializer.java +++ b/server/src/main/java/com/metamx/druid/index/v1/ComplexMetricColumnSerializer.java @@ -19,7 +19,6 @@ package com.metamx.druid.index.v1; -import com.google.common.io.ByteStreams; import com.google.common.io.Files; import com.metamx.druid.index.v1.serde.ComplexMetricSerde; import com.metamx.druid.kv.FlattenedArrayWriter; @@ -27,7 +26,6 @@ import com.metamx.druid.kv.IOPeon; import java.io.File; import java.io.IOException; -import java.nio.ByteOrder; /** */ @@ -75,18 +73,12 @@ public class ComplexMetricColumnSerializer implements MetricColumnSerializer { writer.close(); - final File littleEndianFile = IndexIO.makeMetricFile(outDir, metricName, ByteOrder.LITTLE_ENDIAN); - littleEndianFile.delete(); + final File outFile = IndexIO.makeMetricFile(outDir, metricName, IndexIO.BYTE_ORDER); + outFile.delete(); MetricHolder.writeComplexMetric( - Files.newOutputStreamSupplier(littleEndianFile, true), metricName, serde.getTypeName(), writer - ); - IndexIO.checkFileSize(littleEndianFile); - - final File bigEndianFile = IndexIO.makeMetricFile(outDir, metricName, ByteOrder.BIG_ENDIAN); - ByteStreams.copy( - Files.newInputStreamSupplier(littleEndianFile), - Files.newOutputStreamSupplier(bigEndianFile, false) + Files.newOutputStreamSupplier(outFile, true), metricName, serde.getTypeName(), writer ); + IndexIO.checkFileSize(outFile); writer = null; } diff --git a/server/src/main/java/com/metamx/druid/index/v1/FloatMetricColumnSerializer.java b/server/src/main/java/com/metamx/druid/index/v1/FloatMetricColumnSerializer.java index 8207897e502..20ec5a4d30d 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/FloatMetricColumnSerializer.java +++ b/server/src/main/java/com/metamx/druid/index/v1/FloatMetricColumnSerializer.java @@ -24,7 +24,6 @@ import com.metamx.druid.kv.IOPeon; import java.io.File; import java.io.IOException; -import java.nio.ByteOrder; /** */ @@ -34,8 +33,7 @@ public class FloatMetricColumnSerializer implements MetricColumnSerializer private final IOPeon ioPeon; private final File outDir; - private CompressedFloatsSupplierSerializer littleMetricsWriter; - private CompressedFloatsSupplierSerializer bigEndianMetricsWriter; + private CompressedFloatsSupplierSerializer writer; public FloatMetricColumnSerializer( String metricName, @@ -51,43 +49,30 @@ public class FloatMetricColumnSerializer implements MetricColumnSerializer @Override public void open() throws IOException { - littleMetricsWriter = CompressedFloatsSupplierSerializer.create( - ioPeon, String.format("%s_little", metricName), ByteOrder.LITTLE_ENDIAN - ); - bigEndianMetricsWriter = CompressedFloatsSupplierSerializer.create( - ioPeon, String.format("%s_big", metricName), ByteOrder.BIG_ENDIAN + writer = CompressedFloatsSupplierSerializer.create( + ioPeon, String.format("%s_little", metricName), IndexIO.BYTE_ORDER ); - littleMetricsWriter.open(); - bigEndianMetricsWriter.open(); + writer.open(); } @Override public void serialize(Object obj) throws IOException { float val = (obj == null) ? 0 : ((Number) obj).floatValue(); - littleMetricsWriter.add(val); - bigEndianMetricsWriter.add(val); + writer.add(val); } @Override public void close() throws IOException { - final File littleEndianFile = IndexIO.makeMetricFile(outDir, metricName, ByteOrder.LITTLE_ENDIAN); - littleEndianFile.delete(); + final File outFile = IndexIO.makeMetricFile(outDir, metricName, IndexIO.BYTE_ORDER); + outFile.delete(); MetricHolder.writeFloatMetric( - Files.newOutputStreamSupplier(littleEndianFile, true), metricName, littleMetricsWriter + Files.newOutputStreamSupplier(outFile, true), metricName, writer ); - IndexIO.checkFileSize(littleEndianFile); + IndexIO.checkFileSize(outFile); - final File bigEndianFile = IndexIO.makeMetricFile(outDir, metricName, ByteOrder.BIG_ENDIAN); - bigEndianFile.delete(); - MetricHolder.writeFloatMetric( - Files.newOutputStreamSupplier(bigEndianFile, true), metricName, bigEndianMetricsWriter - ); - IndexIO.checkFileSize(bigEndianFile); - - littleMetricsWriter = null; - bigEndianMetricsWriter = null; + writer = null; } } diff --git a/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java b/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java index 6f9892ee805..4eb8db5fcc7 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java +++ b/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java @@ -44,6 +44,7 @@ import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.ToLowerCaseAggregatorFactory; import com.metamx.druid.guava.FileOutputSupplier; import com.metamx.druid.guava.GuavaUtils; +import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.v1.serde.ComplexMetricSerde; import com.metamx.druid.index.v1.serde.ComplexMetrics; import com.metamx.druid.kv.ConciseCompressedIndexedInts; @@ -75,6 +76,7 @@ import java.nio.channels.FileChannel; import java.util.ArrayList; import java.util.Arrays; import java.util.Iterator; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.TreeSet; @@ -139,26 +141,26 @@ public class IndexMerger ); } - public static File mergeMMapped( - List indexes, final AggregatorFactory[] metricAggs, File outDir + public static File mergeQueryableIndex( + List indexes, final AggregatorFactory[] metricAggs, File outDir ) throws IOException { - return mergeMMapped(indexes, metricAggs, outDir, new NoopProgressIndicator()); + return mergeQueryableIndex(indexes, metricAggs, outDir, new NoopProgressIndicator()); } - public static File mergeMMapped( - List indexes, final AggregatorFactory[] metricAggs, File outDir, ProgressIndicator progress + public static File mergeQueryableIndex( + List indexes, final AggregatorFactory[] metricAggs, File outDir, ProgressIndicator progress ) throws IOException { return merge( Lists.transform( indexes, - new Function() + new Function() { @Override - public IndexableAdapter apply(@Nullable final MMappedIndex input) + public IndexableAdapter apply(final QueryableIndex input) { - return new MMappedIndexAdapter(input); + return new QueryableIndexIndexableAdapter(input); } } ), @@ -392,6 +394,7 @@ public class IndexMerger } final Interval dataInterval; File v8OutDir = new File(outDir, "v8-tmp"); + v8OutDir.mkdirs(); /************* Main index.drd file **************/ progress.progress(); @@ -573,15 +576,11 @@ public class IndexMerger Iterable theRows = rowMergerFn.apply(boats); - CompressedLongsSupplierSerializer littleEndianTimeWriter = CompressedLongsSupplierSerializer.create( - ioPeon, "little_end_time", ByteOrder.LITTLE_ENDIAN - ); - CompressedLongsSupplierSerializer bigEndianTimeWriter = CompressedLongsSupplierSerializer.create( - ioPeon, "big_end_time", ByteOrder.BIG_ENDIAN + CompressedLongsSupplierSerializer timeWriter = CompressedLongsSupplierSerializer.create( + ioPeon, "little_end_time", IndexIO.BYTE_ORDER ); - littleEndianTimeWriter.open(); - bigEndianTimeWriter.open(); + timeWriter.open(); ArrayList forwardDimWriters = Lists.newArrayListWithCapacity(mergedDimensions.size()); for (String dimension : mergedDimensions) { @@ -621,8 +620,7 @@ public class IndexMerger for (Rowboat theRow : theRows) { progress.progress(); - littleEndianTimeWriter.add(theRow.getTimestamp()); - bigEndianTimeWriter.add(theRow.getTimestamp()); + timeWriter.add(theRow.getTimestamp()); final Object[] metrics = theRow.getMetrics(); for (int i = 0; i < metrics.length; ++i) { @@ -660,17 +658,11 @@ public class IndexMerger rowNumConversion.rewind(); } - final File littleEndianFile = IndexIO.makeTimeFile(v8OutDir, ByteOrder.LITTLE_ENDIAN); - littleEndianFile.delete(); - OutputSupplier out = Files.newOutputStreamSupplier(littleEndianFile, true); - littleEndianTimeWriter.closeAndConsolidate(out); - IndexIO.checkFileSize(littleEndianFile); - - final File bigEndianFile = IndexIO.makeTimeFile(v8OutDir, ByteOrder.BIG_ENDIAN); - bigEndianFile.delete(); - out = Files.newOutputStreamSupplier(bigEndianFile, true); - bigEndianTimeWriter.closeAndConsolidate(out); - IndexIO.checkFileSize(bigEndianFile); + final File timeFile = IndexIO.makeTimeFile(v8OutDir, IndexIO.BYTE_ORDER); + timeFile.delete(); + OutputSupplier out = Files.newOutputStreamSupplier(timeFile, true); + timeWriter.closeAndConsolidate(out); + IndexIO.checkFileSize(timeFile); for (int i = 0; i < mergedDimensions.size(); ++i) { forwardDimWriters.get(i).close(); @@ -746,11 +738,12 @@ public class IndexMerger final ArrayList expectedFiles = Lists.newArrayList( Iterables.concat( Arrays.asList( - "index.drd", "inverted.drd", "time_BIG_ENDIAN.drd", "time_LITTLE_ENDIAN.drd" + "index.drd", "inverted.drd", String.format("time_%s.drd", IndexIO.BYTE_ORDER) ), Iterables.transform(mergedDimensions, GuavaUtils.formatFunction("dim_%s.drd")), - Iterables.transform(mergedMetrics, GuavaUtils.formatFunction("met_%s_LITTLE_ENDIAN.drd")), - Iterables.transform(mergedMetrics, GuavaUtils.formatFunction("met_%s_BIG_ENDIAN.drd")) + Iterables.transform( + mergedMetrics, GuavaUtils.formatFunction(String.format("met_%%s_%s.drd", IndexIO.BYTE_ORDER)) + ) ) ); @@ -791,11 +784,13 @@ public class IndexMerger private static ArrayList mergeIndexed(final List> indexedLists) { - TreeSet retVal = Sets.newTreeSet(Ordering.natural().nullsFirst()); + LinkedHashSet retVal = Sets.newLinkedHashSet(); for (Iterable indexedList : indexedLists) { for (T val : indexedList) { - retVal.add(val); + if (val != null) { + retVal.add(val); + } } } diff --git a/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexQueryableIndex.java b/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexQueryableIndex.java index 8aed193edd7..3c9d62d6776 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexQueryableIndex.java +++ b/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexQueryableIndex.java @@ -53,6 +53,12 @@ public class MMappedIndexQueryableIndex implements QueryableIndex return index.getDataInterval(); } + @Override + public int getNumRows() + { + return index.getTimestamps().size(); + } + @Override public Indexed getColumnNames() { @@ -91,7 +97,7 @@ public class MMappedIndexQueryableIndex implements QueryableIndex return new FloatColumn(metricHolder.floatType); } else { - return new ComplexColumnImpl(metricHolder.getComplexType()); + return new ComplexColumnImpl(metricHolder.getTypeName(), metricHolder.getComplexType()); } } } diff --git a/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java b/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java new file mode 100644 index 00000000000..72eddb9141c --- /dev/null +++ b/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java @@ -0,0 +1,267 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.index.v1; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.google.common.io.Closeables; +import com.metamx.common.ISE; +import com.metamx.druid.index.QueryableIndex; +import com.metamx.druid.index.column.Column; +import com.metamx.druid.index.column.ComplexColumn; +import com.metamx.druid.index.column.DictionaryEncodedColumn; +import com.metamx.druid.index.column.GenericColumn; +import com.metamx.druid.index.column.ValueType; +import com.metamx.druid.kv.ArrayBasedIndexedInts; +import com.metamx.druid.kv.ConciseCompressedIndexedInts; +import com.metamx.druid.kv.Indexed; +import com.metamx.druid.kv.IndexedInts; +import com.metamx.druid.kv.IndexedIterable; +import com.metamx.druid.kv.ListIndexed; +import org.joda.time.Interval; + +import java.io.Closeable; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; + +/** +*/ +public class QueryableIndexIndexableAdapter implements IndexableAdapter +{ + private final int numRows; + private final QueryableIndex input; + + public QueryableIndexIndexableAdapter(QueryableIndex input) + { + this.input = input; + numRows = input.getNumRows(); + } + + @Override + public Interval getDataInterval() + { + return input.getDataInterval(); + } + + @Override + public int getNumRows() + { + return numRows; + } + + @Override + public Indexed getAvailableDimensions() + { + return input.getAvailableDimensions(); + } + + @Override + public Indexed getAvailableMetrics() + { + final Set columns = Sets.newLinkedHashSet(input.getColumnNames()); + final HashSet dimensions = Sets.newHashSet(getAvailableDimensions()); + + return new ListIndexed( + Lists.newArrayList(Sets.difference(columns, dimensions)), + String.class + ); + } + + @Override + public Indexed getDimValueLookup(String dimension) + { + final DictionaryEncodedColumn dict = input.getColumn(dimension).getDictionaryEncoding(); + return new Indexed() + { + @Override + public Class getClazz() + { + return String.class; + } + + @Override + public int size() + { + return dict.getCardinality(); + } + + @Override + public String get(int index) + { + return dict.lookupName(index); + } + + @Override + public int indexOf(String value) + { + return dict.lookupId(value); + } + + @Override + public Iterator iterator() + { + return IndexedIterable.create(this).iterator(); + } + }; + } + + @Override + public Iterable getRows() + { + return new Iterable() + { + @Override + public Iterator iterator() + { + return new Iterator() + { + final GenericColumn timestamps = input.getTimeColumn().getGenericColumn(); + final Object[] metrics; + final Map dimensions; + + final int numMetrics = getAvailableMetrics().size(); + + int currRow = 0; + boolean done = false; + + { + dimensions = Maps.newLinkedHashMap(); + for (String dim : input.getAvailableDimensions()) { + dimensions.put(dim, input.getColumn(dim).getDictionaryEncoding()); + } + + final Indexed availableMetrics = getAvailableMetrics(); + metrics = new Object[availableMetrics.size()]; + for (int i = 0; i < metrics.length; ++i) { + final Column column = input.getColumn(availableMetrics.get(i)); + final ValueType type = column.getCapabilities().getType(); + switch (type) { + case FLOAT: + metrics[i] = column.getGenericColumn(); + break; + case COMPLEX: + metrics[i] = column.getComplexColumn(); + break; + default: + throw new ISE("Cannot handle type[%s]", type); + } + } + } + + @Override + public boolean hasNext() + { + final boolean hasNext = currRow < numRows; + if (!hasNext && !done) { + Closeables.closeQuietly(timestamps); + for (Object metric : metrics) { + if (metric instanceof Closeable) { + Closeables.closeQuietly((Closeable) metric); + } + } + done = true; + } + return hasNext; + } + + @Override + public Rowboat next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + int[][] dims = new int[dimensions.size()][]; + int dimIndex = 0; + for (String dim : dimensions.keySet()) { + final DictionaryEncodedColumn dict = dimensions.get(dim); + final IndexedInts dimVals; + if (dict.hasMultipleValues()) { + dimVals = dict.getMultiValueRow(currRow); + } + else { + dimVals = new ArrayBasedIndexedInts(new int[]{dict.getSingleValueRow(currRow)}); + } + + int[] theVals = new int[dimVals.size()]; + for (int j = 0; j < theVals.length; ++j) { + theVals[j] = dimVals.get(j); + } + + dims[dimIndex++] = theVals; + } + + Object[] metricArray = new Object[numMetrics]; + for (int i = 0; i < metricArray.length; ++i) { + if (metrics[i] instanceof GenericColumn) { + metricArray[i] = ((GenericColumn) metrics[i]).getFloatSingleValueRow(currRow); + } + else if (metrics[i] instanceof ComplexColumn) { + metricArray[i] = ((ComplexColumn) metrics[i]).getRowValue(currRow); + } + } + + final Rowboat retVal = new Rowboat( + timestamps.getLongSingleValueRow(currRow), dims, metricArray, currRow + ); + + ++currRow; + + return retVal; + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + }; + } + }; + } + + @Override + public IndexedInts getInverteds(String dimension, String value) + { + return new ConciseCompressedIndexedInts( + input.getColumn(dimension).getBitmapIndex().getConciseSet(value) + ); + } + + @Override + public String getMetricType(String metric) + { + final Column column = input.getColumn(metric); + + final ValueType type = column.getCapabilities().getType(); + switch (type) { + case FLOAT: + return "float"; + case COMPLEX: + return column.getComplexColumn().getTypeName(); + default: + throw new ISE("Unknown type[%s]", type); + } + } +} diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java index 88cb5840950..788e1adb02a 100644 --- a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java @@ -277,7 +277,10 @@ public class GroupByQueryEngine dimNames = new String[dimensionSpecs.size()]; for (int i = 0; i < dimensionSpecs.size(); ++i) { final DimensionSpec dimSpec = dimensionSpecs.get(i); - dimensions.add(cursor.makeDimensionSelector(dimSpec.getDimension())); + final DimensionSelector selector = cursor.makeDimensionSelector(dimSpec.getDimension()); + if (selector != null) { + dimensions.add(selector); + } dimNames[i] = dimSpec.getOutputName(); } diff --git a/server/src/test/java/com/metamx/druid/index/v1/EmptyIndexTest.java b/server/src/test/java/com/metamx/druid/index/v1/EmptyIndexTest.java index 01dd32a3097..55c4b7be6e5 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/EmptyIndexTest.java +++ b/server/src/test/java/com/metamx/druid/index/v1/EmptyIndexTest.java @@ -23,12 +23,12 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; +import com.metamx.druid.index.QueryableIndex; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; import java.io.File; -import java.util.ArrayList; public class EmptyIndexTest { @@ -48,11 +48,11 @@ public class EmptyIndexTest IncrementalIndexAdapter emptyIndexAdapter = new IncrementalIndexAdapter(new Interval("2012-08-01/P3D"), emptyIndex); IndexMerger.merge(Lists.newArrayList(emptyIndexAdapter), new AggregatorFactory[0], tmpDir); - MMappedIndex emptyIndexMMapped = IndexIO.mapDir(tmpDir); + QueryableIndex emptyQueryableIndex = IndexIO.loadIndex(tmpDir); - Assert.assertEquals("getAvailableDimensions", 0, Iterables.size(emptyIndexMMapped.getAvailableDimensions())); - Assert.assertEquals("getAvailableMetrics", 0, Iterables.size(emptyIndexMMapped.getAvailableMetrics())); - Assert.assertEquals("getDataInterval", new Interval("2012-08-01/P3D"), emptyIndexMMapped.getDataInterval()); - Assert.assertEquals("getReadOnlyTimestamps", 0, emptyIndexMMapped.getReadOnlyTimestamps().size()); + Assert.assertEquals("getAvailableDimensions", 0, Iterables.size(emptyQueryableIndex.getAvailableDimensions())); + Assert.assertEquals("getAvailableMetrics", 0, Iterables.size(emptyQueryableIndex.getColumnNames())); + Assert.assertEquals("getDataInterval", new Interval("2012-08-01/P3D"), emptyQueryableIndex.getDataInterval()); + Assert.assertEquals("getReadOnlyTimestamps", 0, emptyQueryableIndex.getTimeColumn().getLength()); } } diff --git a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java index c3afa5eb94e..097762106a2 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java +++ b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.Lists; import com.google.common.io.Files; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; +import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.input.MapBasedInputRow; import junit.framework.Assert; import org.apache.commons.io.FileUtils; @@ -44,11 +45,11 @@ public class IndexMergerTest final File tempDir = Files.createTempDir(); try { - MMappedIndex index = IndexIO.mapDir(IndexMerger.persist(toPersist, tempDir)); + QueryableIndex index = IndexIO.loadIndex(IndexMerger.persist(toPersist, tempDir)); - Assert.assertEquals(2, index.getTimestamps().size()); + Assert.assertEquals(2, index.getTimeColumn().getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); - Assert.assertEquals(0, index.getAvailableMetrics().size()); + Assert.assertEquals(2, index.getColumnNames().size()); } finally { tempDir.delete(); @@ -84,25 +85,25 @@ public class IndexMergerTest final File tempDir2 = Files.createTempDir(); final File mergedDir = Files.createTempDir(); try { - MMappedIndex index1 = IndexIO.mapDir(IndexMerger.persist(toPersist1, tempDir1)); + QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tempDir1)); - Assert.assertEquals(2, index1.getTimestamps().size()); + Assert.assertEquals(2, index1.getTimeColumn().getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); - Assert.assertEquals(0, index1.getAvailableMetrics().size()); + Assert.assertEquals(2, index1.getColumnNames().size()); - MMappedIndex index2 = IndexIO.mapDir(IndexMerger.persist(toPersist2, tempDir2)); + QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist2, tempDir2)); - Assert.assertEquals(2, index2.getTimestamps().size()); + Assert.assertEquals(2, index2.getTimeColumn().getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index2.getAvailableDimensions())); - Assert.assertEquals(0, index2.getAvailableMetrics().size()); + Assert.assertEquals(2, index2.getColumnNames().size()); - MMappedIndex merged = IndexIO.mapDir( - IndexMerger.mergeMMapped(Arrays.asList(index1, index2), new AggregatorFactory[]{}, mergedDir) + QueryableIndex merged = IndexIO.loadIndex( + IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, mergedDir) ); - Assert.assertEquals(3, merged.getTimestamps().size()); + Assert.assertEquals(3, merged.getTimeColumn().getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); - Assert.assertEquals(0, merged.getAvailableMetrics().size()); + Assert.assertEquals(2, merged.getColumnNames().size()); } finally { FileUtils.deleteQuietly(tempDir1); diff --git a/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java b/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java index 6b07bfa5902..164c18a13fc 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java +++ b/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java @@ -119,8 +119,8 @@ public class TestIndex IndexMerger.persist(bottom, DATA_INTERVAL, bottomFile); mergedRealtime = IndexIO.loadIndex( - IndexMerger.mergeMMapped( - Arrays.asList(IndexIO.mapDir(topFile), IndexIO.mapDir(bottomFile)), + IndexMerger.mergeQueryableIndex( + Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)), METRIC_AGGS, mergedFile ) From e2788187fbbd624fd91bd006e94bcd7b17d6902c Mon Sep 17 00:00:00 2001 From: xvrl Date: Wed, 16 Jan 2013 16:02:51 -0800 Subject: [PATCH 29/92] don't let timeout skew benchmark stats --- .../druid/client/cache/MemcachedCacheBrokerBenchmark.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java index 1aa5ed205e7..d0b34af57df 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java @@ -60,7 +60,7 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark cache = new MemcachedCacheBroker( client, - 500, // 500 milliseconds + 30000, // 30 seconds 3600 // 1 hour ); From 5b1e03530cedc0f3f8ddbbbaf397a90c2aa23594 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Wed, 16 Jan 2013 21:06:57 -0600 Subject: [PATCH 30/92] 1) Fix some bugs found by external test suite --- .../serde/BitmapIndexColumnPartSupplier.java | 7 ++++- .../index/serde/ComplexColumnPartSerde.java | 13 +++++--- .../metamx/druid/index/v1/IndexMerger.java | 7 ++--- .../v1/QueryableIndexIndexableAdapter.java | 31 ++++++++++++++++--- 4 files changed, 44 insertions(+), 14 deletions(-) diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java b/index-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java index b282ab56a5c..6813541ef09 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java @@ -51,7 +51,12 @@ public class BitmapIndexColumnPartSupplier implements Supplier { final int index = dictionary.indexOf(value); - return index >= 0 ? bitmaps.get(index) : EMPTY_SET; + if (index < 0) { + return EMPTY_SET; + } + + final ImmutableConciseSet bitmap = bitmaps.get(index); + return bitmap == null ? EMPTY_SET : bitmap; } }; } diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java index 7e425f9f525..760fcbb3f20 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java @@ -36,25 +36,28 @@ public class ComplexColumnPartSerde implements ColumnPartSerde { @JsonCreator public static ComplexColumnPartSerde createDeserializer( - @JsonProperty("complexType") String complexType + @JsonProperty("typeName") String complexType ) { return new ComplexColumnPartSerde(null, complexType); } private final GenericIndexed column; + private final String typeName; + private final ComplexMetricSerde serde; - public ComplexColumnPartSerde(GenericIndexed column, String complexType) + public ComplexColumnPartSerde(GenericIndexed column, String typeName) { this.column = column; - serde = ComplexMetrics.getSerdeForType(complexType); + this.typeName = typeName; + serde = ComplexMetrics.getSerdeForType(typeName); } @JsonProperty - public GenericIndexed getColumn() + public String getTypeName() { - return column; + return typeName; } @Override diff --git a/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java b/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java index 4eb8db5fcc7..57828855b38 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java +++ b/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java @@ -79,6 +79,7 @@ import java.util.Iterator; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.TreeSet; /** @@ -784,13 +785,11 @@ public class IndexMerger private static ArrayList mergeIndexed(final List> indexedLists) { - LinkedHashSet retVal = Sets.newLinkedHashSet(); + Set retVal = Sets.newTreeSet(Ordering.natural().nullsFirst()); for (Iterable indexedList : indexedLists) { for (T val : indexedList) { - if (val != null) { - retVal.add(val); - } + retVal.add(val); } } diff --git a/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java b/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java index 72eddb9141c..d05864716af 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java +++ b/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java @@ -19,12 +19,14 @@ package com.metamx.druid.index.v1; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.common.io.Closeables; import com.metamx.common.ISE; import com.metamx.druid.index.QueryableIndex; +import com.metamx.druid.index.column.BitmapIndex; import com.metamx.druid.index.column.Column; import com.metamx.druid.index.column.ComplexColumn; import com.metamx.druid.index.column.DictionaryEncodedColumn; @@ -32,6 +34,7 @@ import com.metamx.druid.index.column.GenericColumn; import com.metamx.druid.index.column.ValueType; import com.metamx.druid.kv.ArrayBasedIndexedInts; import com.metamx.druid.kv.ConciseCompressedIndexedInts; +import com.metamx.druid.kv.EmptyIndexedInts; import com.metamx.druid.kv.Indexed; import com.metamx.druid.kv.IndexedInts; import com.metamx.druid.kv.IndexedIterable; @@ -91,7 +94,18 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter @Override public Indexed getDimValueLookup(String dimension) { - final DictionaryEncodedColumn dict = input.getColumn(dimension).getDictionaryEncoding(); + final Column column = input.getColumn(dimension); + + if (column == null) { + return null; + } + + final DictionaryEncodedColumn dict = column.getDictionaryEncoding(); + + if (dict == null) { + return null; + } + return new Indexed() { @Override @@ -244,9 +258,18 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter @Override public IndexedInts getInverteds(String dimension, String value) { - return new ConciseCompressedIndexedInts( - input.getColumn(dimension).getBitmapIndex().getConciseSet(value) - ); + final Column column = input.getColumn(dimension); + + if (column == null) { + return new EmptyIndexedInts(); + } + + final BitmapIndex bitmaps = column.getBitmapIndex(); + if (bitmaps == null) { + return new EmptyIndexedInts(); + } + + return new ConciseCompressedIndexedInts(bitmaps.getConciseSet(value)); } @Override From dcaa77a883bd1690dda721a96621ad15f6e6f656 Mon Sep 17 00:00:00 2001 From: xvrl Date: Wed, 16 Jan 2013 19:15:43 -0800 Subject: [PATCH 31/92] implement bulk get test --- .../cache/MemcachedCacheBrokerTest.java | 93 ++++++++++++++++++- 1 file changed, 89 insertions(+), 4 deletions(-) diff --git a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java index 9bfb928a949..80ddd0072b5 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java @@ -19,7 +19,10 @@ package com.metamx.druid.client.cache; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.primitives.Ints; +import com.metamx.common.Pair; import net.spy.memcached.CASResponse; import net.spy.memcached.CASValue; import net.spy.memcached.CachedData; @@ -27,6 +30,7 @@ import net.spy.memcached.ConnectionObserver; import net.spy.memcached.MemcachedClientIF; import net.spy.memcached.NodeLocator; import net.spy.memcached.internal.BulkFuture; +import net.spy.memcached.ops.OperationStatus; import net.spy.memcached.transcoders.SerializingTranscoder; import net.spy.memcached.transcoders.Transcoder; import org.junit.Assert; @@ -34,6 +38,7 @@ import org.junit.Before; import org.junit.Test; import java.net.SocketAddress; +import java.nio.ByteBuffer; import java.util.Collection; import java.util.Iterator; import java.util.Map; @@ -85,6 +90,28 @@ public class MemcachedCacheBrokerTest cache.close("a"); } + @Test + public void testGetBulk() throws Exception + { + Assert.assertNull(cache.get("the", HI)); + + put(cache, "the", HI, 2); + put(cache, "the", HO, 10); + + Pair key1 = Pair.of("the", ByteBuffer.wrap(HI)); + Pair key2 = Pair.of("the", ByteBuffer.wrap(HO)); + + Map, byte[]> result = cache.getBulk( + Lists.newArrayList( + key1, + key2 + ) + ); + + Assert.assertEquals(2, Ints.fromByteArray(result.get(key1))); + Assert.assertEquals(10, Ints.fromByteArray(result.get(key2))); + } + public void put(CacheBroker cache, String identifier, byte[] key, Integer value) { cache.put(identifier, key, Ints.toByteArray(value)); @@ -362,9 +389,67 @@ class MockMemcachedClient implements MemcachedClientIF } @Override - public BulkFuture> asyncGetBulk(Iterator keys, Transcoder tc) + public BulkFuture> asyncGetBulk(final Iterator keys, final Transcoder tc) { - throw new UnsupportedOperationException("not implemented"); + return new BulkFuture>() + { + @Override + public boolean isTimeout() + { + return false; + } + + @Override + public Map getSome(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException + { + return get(); + } + + @Override + public OperationStatus getStatus() + { + return null; + } + + @Override + public boolean cancel(boolean b) + { + return false; + } + + @Override + public boolean isCancelled() + { + return false; + } + + @Override + public boolean isDone() + { + return true; + } + + @Override + public Map get() throws InterruptedException, ExecutionException + { + Map retVal = Maps.newHashMap(); + + while(keys.hasNext()) { + String key = keys.next(); + CachedData data = theMap.get(key); + retVal.put(key, data != null ? tc.decode(data) : null); + } + + return retVal; + } + + @Override + public Map get(long l, TimeUnit timeUnit) + throws InterruptedException, ExecutionException, TimeoutException + { + return get(); + } + }; } @Override @@ -380,9 +465,9 @@ class MockMemcachedClient implements MemcachedClientIF } @Override - public BulkFuture> asyncGetBulk(Collection keys) + public BulkFuture> asyncGetBulk(final Collection keys) { - throw new UnsupportedOperationException("not implemented"); + return asyncGetBulk(keys.iterator(), transcoder); } @Override From 0bacb85a4a7e62202d1618c548c980ec50ba653b Mon Sep 17 00:00:00 2001 From: xvrl Date: Wed, 16 Jan 2013 19:18:14 -0800 Subject: [PATCH 32/92] fix duplicate keys, shutdown gracefully and make sure we check all multiget keys in memcached benchmark --- .../client/cache/MemcachedCacheBrokerBenchmark.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java index d0b34af57df..14bd9fdc998 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java @@ -44,8 +44,6 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark // disable compression transcoder.setCompressionThreshold(Integer.MAX_VALUE); - System.out.println(String.format("Using memcached hosts [%s]", hosts)); - client = new MemcachedClient( new ConnectionFactoryBuilder().setProtocol(ConnectionFactoryBuilder.Protocol.BINARY) .setHashAlg(DefaultHashAlgorithm.FNV1A_64_HASH) @@ -72,14 +70,13 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark @Override protected void tearDown() throws Exception { - client.flush(); - client.shutdown(); + client.shutdown(1, TimeUnit.MINUTES); } public void timePutObjects(int reps) { for(int i = 0; i < reps; ++i) { for(int k = 0; k < objectCount; ++k) { - String key = BASE_KEY + i; + String key = BASE_KEY + k; cache.put(IDENTIFIER, key.getBytes(), randBytes); } // make sure the write queue is empty @@ -109,7 +106,10 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark keys.add(Pair.of(IDENTIFIER, ByteBuffer.wrap(key.getBytes()))); } Map, byte[]> results = cache.getBulk(keys); - for(byte[] bytes : results.values()) count += bytes.length; + for(Pair key : keys) { + byte[] bytes = results.get(key); + count += bytes.length; + } } return count; } From 689ce4f8e1536d0d27ff250d44fa2a11e713fa1d Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 17 Jan 2013 13:10:11 -0600 Subject: [PATCH 33/92] 1) Upgrade java-util dependency to include "ruby" time --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 20ca0f1bada..c580894b379 100644 --- a/pom.xml +++ b/pom.xml @@ -68,7 +68,7 @@ com.metamx java-util - 0.18.0 + 0.19.1 com.metamx From 38b2041ad9f436913db6352e43983f384a914f39 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Thu, 17 Jan 2013 14:56:48 -0800 Subject: [PATCH 34/92] key/value config table --- .../java/com/metamx/druid/db/DbConnector.java | 8 +-- .../config/WorkerSetupManagerConfig.java | 7 +- .../http/IndexerCoordinatorNode.java | 2 +- .../coordinator/setup/WorkerSetupManager.java | 71 ++++++++++--------- .../merger/worker/config/WorkerConfig.java | 3 +- 5 files changed, 47 insertions(+), 44 deletions(-) diff --git a/common/src/main/java/com/metamx/druid/db/DbConnector.java b/common/src/main/java/com/metamx/druid/db/DbConnector.java index 45a0b937964..f3c4c1f13d3 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnector.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnector.java @@ -59,14 +59,14 @@ public class DbConnector ); } - public static void createWorkerSetupTable(final DBI dbi, final String workerTableName) + public static void createConfigTable(final DBI dbi, final String configTableName) { createTable( dbi, - workerTableName, + configTableName, String.format( - "CREATE table %s (config LONGTEXT NOT NULL)", - workerTableName + "CREATE table %s (name VARCHAR(255) NOT NULL, payload LONGTEXT NOT NULL, INDEX(name), PRIMARY KEY(name))", + configTableName ) ); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java index 97368c9f77e..16eeb1c3439 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java @@ -27,8 +27,11 @@ import org.skife.config.Default; */ public abstract class WorkerSetupManagerConfig { - @Config("druid.indexer.workerSetupTable") - public abstract String getWorkerSetupTable(); + @Config("druid.indexer.configTable") + public abstract String getConfigTable(); + + @Config("druid.indexer.workerSetupConfigName") + public abstract String getWorkerSetupConfigName(); @Config("druid.indexer.poll.duration") @Default("PT1M") diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 15d76cf2fec..3dae4046764 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -469,7 +469,7 @@ public class IndexerCoordinatorNode extends RegisteringNode final DBI dbi = new DbConnector(dbConnectorConfig).getDBI(); final WorkerSetupManagerConfig workerSetupManagerConfig = configFactory.build(WorkerSetupManagerConfig.class); - DbConnector.createWorkerSetupTable(dbi, workerSetupManagerConfig.getWorkerSetupTable()); + DbConnector.createConfigTable(dbi, workerSetupManagerConfig.getConfigTable()); workerSetupManager = new WorkerSetupManager( dbi, Executors.newScheduledThreadPool( 1, diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java index baa56290af7..5e43e68ae66 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java @@ -29,7 +29,6 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig; import org.apache.commons.collections.MapUtils; import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; import org.joda.time.Duration; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.FoldController; @@ -123,37 +122,39 @@ public class WorkerSetupManager { return handle.createQuery( String.format( - "SELECT config FROM %s", - config.getWorkerSetupTable() + "SELECT payload FROM %s WHERE name = :name", + config.getConfigTable() ) - ).fold( - Lists.newArrayList(), - new Folder3, Map>() - { - @Override - public ArrayList fold( - ArrayList workerNodeConfigurations, - Map stringObjectMap, - FoldController foldController, - StatementContext statementContext - ) throws SQLException - { - try { - // stringObjectMap lowercases and jackson may fail serde - workerNodeConfigurations.add( - jsonMapper.readValue( - MapUtils.getString(stringObjectMap, "config"), - WorkerSetupData.class - ) - ); - return workerNodeConfigurations; - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - } - ); + ) + .bind("name", config.getWorkerSetupConfigName()) + .fold( + Lists.newArrayList(), + new Folder3, Map>() + { + @Override + public ArrayList fold( + ArrayList workerNodeConfigurations, + Map stringObjectMap, + FoldController foldController, + StatementContext statementContext + ) throws SQLException + { + try { + // stringObjectMap lowercases and jackson may fail serde + workerNodeConfigurations.add( + jsonMapper.readValue( + MapUtils.getString(stringObjectMap, "payload"), + WorkerSetupData.class + ) + ); + return workerNodeConfigurations; + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + ); } } ); @@ -197,14 +198,14 @@ public class WorkerSetupManager @Override public Void withHandle(Handle handle) throws Exception { - handle.createStatement(String.format("DELETE FROM %s", config.getWorkerSetupTable())).execute(); handle.createStatement( String.format( - "INSERT INTO %s (config) VALUES (:config)", - config.getWorkerSetupTable() + "INSERT INTO %s (name, payload) VALUES (:name, :payload) ON DUPLICATE KEY UPDATE payload = :payload", + config.getConfigTable() ) ) - .bind("config", jsonMapper.writeValueAsString(value)) + .bind("name", config.getWorkerSetupConfigName()) + .bind("payload", jsonMapper.writeValueAsString(value)) .execute(); return null; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java index 4689acef261..5b5f3a0a6e7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java @@ -41,7 +41,6 @@ public abstract class WorkerConfig public int getCapacity() { - return 1; - //return Runtime.getRuntime().availableProcessors() - 1; + return Runtime.getRuntime().availableProcessors() - 1; } } From 71665346667bcd907b1fc349103b2e612c91d347 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 17 Jan 2013 16:03:18 -0800 Subject: [PATCH 35/92] YeOldePlumberSchool: Tweak for IndexIO changes --- .../metamx/druid/merger/common/index/YeOldePlumberSchool.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java index 48d639726ae..c5a2bc11826 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java @@ -140,7 +140,7 @@ public class YeOldePlumberSchool implements PlumberSchool } // Map merged segment so we can extract dimensions - final MMappedIndex mappedSegment = IndexIO.mapDir(fileToUpload); + final QueryableIndex mappedSegment = IndexIO.loadIndex(fileToUpload); final DataSegment segmentToUpload = theSink.getSegment() .withDimensions(ImmutableList.copyOf(mappedSegment.getAvailableDimensions())) From 9032ef521b10f85f3513dda6b2ff66f8668ab4f4 Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 18 Jan 2013 10:18:16 -0800 Subject: [PATCH 36/92] fix interrupted thread --- .../com/metamx/druid/client/cache/MemcachedCacheBroker.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBroker.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBroker.java index f2b709de8e5..07953479337 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBroker.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBroker.java @@ -122,6 +122,7 @@ public class MemcachedCacheBroker implements CacheBroker return null; } catch(InterruptedException e) { + Thread.currentThread().interrupt(); throw Throwables.propagate(e); } catch(ExecutionException e) { @@ -175,6 +176,7 @@ public class MemcachedCacheBroker implements CacheBroker return results; } catch(InterruptedException e) { + Thread.currentThread().interrupt(); throw Throwables.propagate(e); } catch(ExecutionException e) { From a70ae155851f2b5f42d2c5a3c451185274a6c00a Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 18 Jan 2013 10:00:06 -0800 Subject: [PATCH 37/92] replace Pair with NamedKey --- .../druid/client/CachingClusteredClient.java | 33 +++++------ .../druid/client/cache/CacheBroker.java | 55 +++++++++++++++++-- .../druid/client/cache/MapCacheBroker.java | 24 ++++---- .../client/cache/MemcachedCacheBroker.java | 32 +++++------ .../client/cache/MapCacheBrokerTest.java | 16 +++--- .../cache/MemcachedCacheBrokerBenchmark.java | 16 +++--- .../cache/MemcachedCacheBrokerTest.java | 24 ++++---- 7 files changed, 115 insertions(+), 85 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index 23f5ae6992e..625aef4d0e1 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -21,6 +21,7 @@ package com.metamx.druid.client; import com.google.common.base.Function; import com.google.common.base.Predicate; +import com.google.common.base.Predicates; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -133,7 +134,7 @@ public class CachingClusteredClient implements QueryRunner return Sequences.empty(); } - Map, Pair> segments = Maps.newLinkedHashMap(); + Map, CacheBroker.NamedKey> segments = Maps.newLinkedHashMap(); final byte[] queryCacheKey = (strategy != null) ? strategy.computeCacheKey(query) : null; @@ -156,21 +157,13 @@ public class CachingClusteredClient implements QueryRunner } } - Map, byte[]> cachedValues = cacheBroker.getBulk( - Iterables.filter(segments.values(), new Predicate>() - { - @Override - public boolean apply( - @Nullable Pair input - ) - { - return input != null; - } - }) + Map cachedValues = cacheBroker.getBulk( + Iterables.filter(segments.values(), Predicates.notNull()) ); - for(Pair segment : segments.keySet()) { - Pair segmentCacheKey = segments.get(segment); + for(Map.Entry, CacheBroker.NamedKey> entry : segments.entrySet()) { + Pair segment = entry.getKey(); + CacheBroker.NamedKey segmentCacheKey = entry.getValue(); final ServerSelector selector = segment.lhs; final SegmentDescriptor descriptor = segment.rhs; @@ -335,19 +328,19 @@ public class CachingClusteredClient implements QueryRunner ); } - private Pair computeSegmentCacheKey(String segmentIdentifier, SegmentDescriptor descriptor, byte[] queryCacheKey) + private CacheBroker.NamedKey computeSegmentCacheKey(String segmentIdentifier, SegmentDescriptor descriptor, byte[] queryCacheKey) { final Interval segmentQueryInterval = descriptor.getInterval(); final byte[] versionBytes = descriptor.getVersion().getBytes(); - return Pair.of( + return new CacheBroker.NamedKey( segmentIdentifier, ByteBuffer .allocate(16 + versionBytes.length + 4 + queryCacheKey.length) .putLong(segmentQueryInterval.getStartMillis()) .putLong(segmentQueryInterval.getEndMillis()) .put(versionBytes) .putInt(descriptor.getPartitionNumber()) - .put(queryCacheKey) + .put(queryCacheKey).array() ); } @@ -355,9 +348,9 @@ public class CachingClusteredClient implements QueryRunner { private final CacheBroker cache; private final ObjectMapper mapper; - private final Pair key; + private final CacheBroker.NamedKey key; - public CachePopulator(CacheBroker cache, ObjectMapper mapper, Pair key) + public CachePopulator(CacheBroker cache, ObjectMapper mapper, CacheBroker.NamedKey key) { this.cache = cache; this.mapper = mapper; @@ -382,7 +375,7 @@ public class CachingClusteredClient implements QueryRunner offset += array.length; } - cache.put(key.lhs, key.rhs.array(), valueBytes); + cache.put(key, valueBytes); } catch (IOException e) { throw Throwables.propagate(e); diff --git a/client/src/main/java/com/metamx/druid/client/cache/CacheBroker.java b/client/src/main/java/com/metamx/druid/client/cache/CacheBroker.java index b1dc548a962..adcba2e489c 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/CacheBroker.java +++ b/client/src/main/java/com/metamx/druid/client/cache/CacheBroker.java @@ -19,20 +19,63 @@ package com.metamx.druid.client.cache; -import com.metamx.common.Pair; +import com.google.common.base.Preconditions; -import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Map; /** */ public interface CacheBroker { - public byte[] get(String identifier, byte[] key); - public void put(String identifier, byte[] key, byte[] value); - public Map, byte[]> getBulk(Iterable> identifierKeyPairs); + public byte[] get(NamedKey key); + public void put(NamedKey key, byte[] value); + public Map getBulk(Iterable keys); - public void close(String identifier); + public void close(String namespace); public CacheStats getStats(); + + public class NamedKey + { + final public String namespace; + final public byte[] key; + + public NamedKey(String namespace, byte[] key) { + Preconditions.checkArgument(namespace != null, "namespace must not be null"); + Preconditions.checkArgument(key != null, "key must not be null"); + this.namespace = namespace; + this.key = key; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + NamedKey namedKey = (NamedKey) o; + + if (!namespace.equals(namedKey.namespace)) { + return false; + } + if (!Arrays.equals(key, namedKey.key)) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = namespace.hashCode(); + result = 31 * result + Arrays.hashCode(key); + return result; + } + } } diff --git a/client/src/main/java/com/metamx/druid/client/cache/MapCacheBroker.java b/client/src/main/java/com/metamx/druid/client/cache/MapCacheBroker.java index 85b9156c3fd..d541ab40a66 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MapCacheBroker.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MapCacheBroker.java @@ -94,32 +94,32 @@ public class MapCacheBroker implements CacheBroker } @Override - public byte[] get(String identifier, byte[] key) + public byte[] get(NamedKey key) { - return provideCache(identifier).get(key); + return provideCache(key.namespace).get(key.key); } @Override - public void put(String identifier, byte[] key, byte[] value) + public void put(NamedKey key, byte[] value) { - provideCache(identifier).put(key, value); + provideCache(key.namespace).put(key.key, value); } @Override - public Map, byte[]> getBulk(Iterable> identifierKeyPairs) + public Map getBulk(Iterable keys) { - Map, byte[]> retVal = Maps.newHashMap(); + Map retVal = Maps.newHashMap(); - for(Pair e : identifierKeyPairs) { - retVal.put(e, provideCache(e.lhs).get(e.rhs.array())); + for(NamedKey key : keys) { + retVal.put(key, provideCache(key.namespace).get(key.key)); } return retVal; } @Override - public void close(String identifier) + public void close(String namespace) { - provideCache(identifier).close(); + provideCache(namespace).close(); } private Cache provideCache(final String identifier) @@ -148,7 +148,7 @@ public class MapCacheBroker implements CacheBroker } return retVal; } - throw new ISE("Cache for identifier[%s] is closed.", identifier); + throw new ISE("Cache for namespace[%s] is closed.", identifier); } @Override @@ -160,7 +160,7 @@ public class MapCacheBroker implements CacheBroker return; } } - throw new ISE("Cache for identifier[%s] is closed.", identifier); + throw new ISE("Cache for namespace[%s] is closed.", identifier); } @Override diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBroker.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBroker.java index 07953479337..fea391f7912 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBroker.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBroker.java @@ -21,9 +21,7 @@ package com.metamx.druid.client.cache; import com.google.common.base.Function; import com.google.common.base.Throwables; -import com.google.common.collect.Iterables; import com.google.common.collect.Maps; -import com.metamx.common.Pair; import net.iharder.base64.Base64; import net.spy.memcached.AddrUtil; import net.spy.memcached.ConnectionFactoryBuilder; @@ -36,8 +34,6 @@ import net.spy.memcached.transcoders.SerializingTranscoder; import javax.annotation.Nullable; import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Collection; import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; @@ -103,9 +99,9 @@ public class MemcachedCacheBroker implements CacheBroker } @Override - public byte[] get(String identifier, byte[] key) + public byte[] get(NamedKey key) { - Future future = client.asyncGet(computeKey(identifier, key)); + Future future = client.asyncGet(computeKeyString(key)); try { byte[] bytes = (byte[]) future.get(timeout, TimeUnit.MILLISECONDS); if(bytes != null) { @@ -131,24 +127,24 @@ public class MemcachedCacheBroker implements CacheBroker } @Override - public void put(String identifier, byte[] key, byte[] value) + public void put(NamedKey key, byte[] value) { - client.set(computeKey(identifier, key), expiration, value); + client.set(computeKeyString(key), expiration, value); } @Override - public Map, byte[]> getBulk(Iterable> identifierKeyPairs) + public Map getBulk(Iterable keys) { - Map> keyLookup = Maps.uniqueIndex( - identifierKeyPairs, - new Function, String>() + Map keyLookup = Maps.uniqueIndex( + keys, + new Function() { @Override public String apply( - @Nullable Pair input + @Nullable NamedKey input ) { - return computeKey(input.lhs, input.rhs.array()); + return computeKeyString(input); } } ); @@ -165,7 +161,7 @@ public class MemcachedCacheBroker implements CacheBroker missCount.addAndGet(keyLookup.size() - some.size()); hitCount.addAndGet(some.size()); - Map, byte[]> results = Maps.newHashMap(); + Map results = Maps.newHashMap(); for(Map.Entry entry : some.entrySet()) { results.put( keyLookup.get(entry.getKey()), @@ -185,12 +181,12 @@ public class MemcachedCacheBroker implements CacheBroker } @Override - public void close(String identifier) + public void close(String namespace) { // no resources to cleanup } - private String computeKey(String identifier, byte[] key) { - return identifier + ":" + Base64.encodeBytes(key, Base64.DONT_BREAK_LINES); + private static String computeKeyString(NamedKey key) { + return key.namespace + ":" + Base64.encodeBytes(key.key, Base64.DONT_BREAK_LINES); } } diff --git a/client/src/test/java/com/metamx/druid/client/cache/MapCacheBrokerTest.java b/client/src/test/java/com/metamx/druid/client/cache/MapCacheBrokerTest.java index 2041e2dca05..35ea7ac42d9 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MapCacheBrokerTest.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MapCacheBrokerTest.java @@ -43,12 +43,12 @@ public class MapCacheBrokerTest @Test public void testSanity() throws Exception { - Assert.assertNull(cache.get("a", HI)); + Assert.assertNull(cache.get(new CacheBroker.NamedKey("a", HI))); Assert.assertEquals(0, baseMap.size()); put(cache, "a", HI, 1); Assert.assertEquals(1, baseMap.size()); Assert.assertEquals(1, get(cache, "a", HI)); - Assert.assertNull(cache.get("the", HI)); + Assert.assertNull(cache.get(new CacheBroker.NamedKey("the", HI))); put(cache, "the", HI, 2); Assert.assertEquals(2, baseMap.size()); @@ -58,26 +58,26 @@ public class MapCacheBrokerTest put(cache, "the", HO, 10); Assert.assertEquals(3, baseMap.size()); Assert.assertEquals(1, get(cache, "a", HI)); - Assert.assertNull(cache.get("a", HO)); + Assert.assertNull(cache.get(new CacheBroker.NamedKey("a", HO))); Assert.assertEquals(2, get(cache, "the", HI)); Assert.assertEquals(10, get(cache, "the", HO)); cache.close("the"); Assert.assertEquals(1, baseMap.size()); Assert.assertEquals(1, get(cache, "a", HI)); - Assert.assertNull(cache.get("a", HO)); + Assert.assertNull(cache.get(new CacheBroker.NamedKey("a", HO))); cache.close("a"); Assert.assertEquals(0, baseMap.size()); } - public void put(CacheBroker cache, String identifier, byte[] key, Integer value) + public void put(CacheBroker cache, String namespace, byte[] key, Integer value) { - cache.put(identifier, key, Ints.toByteArray(value)); + cache.put(new CacheBroker.NamedKey(namespace, key), Ints.toByteArray(value)); } - public int get(CacheBroker cache, String identifier, byte[] key) + public int get(CacheBroker cache, String namespace, byte[] key) { - return Ints.fromByteArray(cache.get(identifier, key)); + return Ints.fromByteArray(cache.get(new CacheBroker.NamedKey(namespace, key))); } } diff --git a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java index 14bd9fdc998..7c25f3ee954 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java @@ -4,7 +4,6 @@ import com.google.caliper.Param; import com.google.caliper.Runner; import com.google.caliper.SimpleBenchmark; import com.google.common.collect.Lists; -import com.metamx.common.Pair; import net.spy.memcached.AddrUtil; import net.spy.memcached.ConnectionFactoryBuilder; import net.spy.memcached.DefaultHashAlgorithm; @@ -13,7 +12,6 @@ import net.spy.memcached.MemcachedClient; import net.spy.memcached.MemcachedClientIF; import net.spy.memcached.transcoders.SerializingTranscoder; -import java.nio.ByteBuffer; import java.util.List; import java.util.Map; import java.util.Random; @@ -22,7 +20,7 @@ import java.util.concurrent.TimeUnit; public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark { private static final String BASE_KEY = "test_2012-11-26T00:00:00.000Z_2012-11-27T00:00:00.000Z_2012-11-27T04:11:25.979Z_"; - public static final String IDENTIFIER = "default"; + public static final String NAMESPACE = "default"; private MemcachedCacheBroker cache; private MemcachedClientIF client; @@ -77,7 +75,7 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark for(int i = 0; i < reps; ++i) { for(int k = 0; k < objectCount; ++k) { String key = BASE_KEY + k; - cache.put(IDENTIFIER, key.getBytes(), randBytes); + cache.put(new CacheBroker.NamedKey(NAMESPACE, key.getBytes()), randBytes); } // make sure the write queue is empty client.waitForQueues(1, TimeUnit.HOURS); @@ -90,7 +88,7 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark for (int i = 0; i < reps; i++) { for(int k = 0; k < objectCount; ++k) { String key = BASE_KEY + k; - bytes = cache.get(IDENTIFIER, key.getBytes()); + bytes = cache.get(new CacheBroker.NamedKey(NAMESPACE, key.getBytes())); count += bytes.length; } } @@ -100,13 +98,13 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark public long timeBulkGetObjects(int reps) { long count = 0; for (int i = 0; i < reps; i++) { - List> keys = Lists.newArrayList(); + List keys = Lists.newArrayList(); for(int k = 0; k < objectCount; ++k) { String key = BASE_KEY + k; - keys.add(Pair.of(IDENTIFIER, ByteBuffer.wrap(key.getBytes()))); + keys.add(new CacheBroker.NamedKey(NAMESPACE, key.getBytes())); } - Map, byte[]> results = cache.getBulk(keys); - for(Pair key : keys) { + Map results = cache.getBulk(keys); + for(CacheBroker.NamedKey key : keys) { byte[] bytes = results.get(key); count += bytes.length; } diff --git a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java index 80ddd0072b5..31030260df7 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java @@ -68,10 +68,10 @@ public class MemcachedCacheBrokerTest @Test public void testSanity() throws Exception { - Assert.assertNull(cache.get("a", HI)); + Assert.assertNull(cache.get(new CacheBroker.NamedKey("a", HI))); put(cache, "a", HI, 1); Assert.assertEquals(1, get(cache, "a", HI)); - Assert.assertNull(cache.get("the", HI)); + Assert.assertNull(cache.get(new CacheBroker.NamedKey("the", HI))); put(cache, "the", HI, 2); Assert.assertEquals(1, get(cache, "a", HI)); @@ -79,13 +79,13 @@ public class MemcachedCacheBrokerTest put(cache, "the", HO, 10); Assert.assertEquals(1, get(cache, "a", HI)); - Assert.assertNull(cache.get("a", HO)); + Assert.assertNull(cache.get(new CacheBroker.NamedKey("a", HO))); Assert.assertEquals(2, get(cache, "the", HI)); Assert.assertEquals(10, get(cache, "the", HO)); cache.close("the"); Assert.assertEquals(1, get(cache, "a", HI)); - Assert.assertNull(cache.get("a", HO)); + Assert.assertNull(cache.get(new CacheBroker.NamedKey("a", HO))); cache.close("a"); } @@ -93,15 +93,15 @@ public class MemcachedCacheBrokerTest @Test public void testGetBulk() throws Exception { - Assert.assertNull(cache.get("the", HI)); + Assert.assertNull(cache.get(new CacheBroker.NamedKey("the", HI))); put(cache, "the", HI, 2); put(cache, "the", HO, 10); - Pair key1 = Pair.of("the", ByteBuffer.wrap(HI)); - Pair key2 = Pair.of("the", ByteBuffer.wrap(HO)); + CacheBroker.NamedKey key1 = new CacheBroker.NamedKey("the", HI); + CacheBroker.NamedKey key2 = new CacheBroker.NamedKey("the", HO); - Map, byte[]> result = cache.getBulk( + Map result = cache.getBulk( Lists.newArrayList( key1, key2 @@ -112,14 +112,14 @@ public class MemcachedCacheBrokerTest Assert.assertEquals(10, Ints.fromByteArray(result.get(key2))); } - public void put(CacheBroker cache, String identifier, byte[] key, Integer value) + public void put(CacheBroker cache, String namespace, byte[] key, Integer value) { - cache.put(identifier, key, Ints.toByteArray(value)); + cache.put(new CacheBroker.NamedKey(namespace, key), Ints.toByteArray(value)); } - public int get(CacheBroker cache, String identifier, byte[] key) + public int get(CacheBroker cache, String namespace, byte[] key) { - return Ints.fromByteArray(cache.get(identifier, key)); + return Ints.fromByteArray(cache.get(new CacheBroker.NamedKey(namespace, key))); } } From 86ca8967cad017fb059ec618e25f45d967e41b77 Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 18 Jan 2013 15:04:12 -0800 Subject: [PATCH 38/92] rework code pulling from cache to be more readable --- .../druid/client/CachingClusteredClient.java | 77 +++++++++++-------- 1 file changed, 45 insertions(+), 32 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index 625aef4d0e1..d69f91e6e14 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -20,8 +20,6 @@ package com.metamx.druid.client; import com.google.common.base.Function; -import com.google.common.base.Predicate; -import com.google.common.base.Predicates; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -30,6 +28,7 @@ import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.metamx.common.ISE; import com.metamx.common.Pair; @@ -66,6 +65,7 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.Executors; /** @@ -134,9 +134,8 @@ public class CachingClusteredClient implements QueryRunner return Sequences.empty(); } - Map, CacheBroker.NamedKey> segments = Maps.newLinkedHashMap(); - - final byte[] queryCacheKey = (strategy != null) ? strategy.computeCacheKey(query) : null; + // build set of segments to query + Set> segments = Sets.newLinkedHashSet(); for (Interval interval : rewrittenQuery.getIntervals()) { List> serversLookup = timeline.lookup(interval); @@ -148,43 +147,44 @@ public class CachingClusteredClient implements QueryRunner holder.getInterval(), holder.getVersion(), chunk.getChunkNumber() ); - segments.put( - Pair.of(selector, descriptor), - queryCacheKey == null ? null : - computeSegmentCacheKey(selector.getSegment().getIdentifier(), descriptor, queryCacheKey) - ); + segments.add(Pair.of(selector, descriptor)); } } } - Map cachedValues = cacheBroker.getBulk( - Iterables.filter(segments.values(), Predicates.notNull()) - ); + final byte[] queryCacheKey; + if(strategy != null) { + queryCacheKey = strategy.computeCacheKey(query); + } else { + queryCacheKey = null; + } - for(Map.Entry, CacheBroker.NamedKey> entry : segments.entrySet()) { - Pair segment = entry.getKey(); - CacheBroker.NamedKey segmentCacheKey = entry.getValue(); + // Pull cached segments from cache and remove from set of segments to query + if(useCache && queryCacheKey != null) { + Map, CacheBroker.NamedKey> cacheKeys = Maps.newHashMap(); + for(Pair e : segments) { + cacheKeys.put(e, computeSegmentCacheKey(e.lhs.getSegment().getIdentifier(), e.rhs, queryCacheKey)); + } - final ServerSelector selector = segment.lhs; - final SegmentDescriptor descriptor = segment.rhs; - final Interval segmentQueryInterval = descriptor.getInterval(); + Map cachedValues = cacheBroker.getBulk(cacheKeys.values()); - final byte[] cachedValue = segmentCacheKey == null ? null : cachedValues.get(segmentCacheKey); + for(Map.Entry, CacheBroker.NamedKey> entry : cacheKeys.entrySet()) { + Pair segment = entry.getKey(); + CacheBroker.NamedKey segmentCacheKey = entry.getValue(); - if (useCache && cachedValue != null) { - cachedResults.add(Pair.of(segmentQueryInterval.getStart(), cachedValue)); - } else { - final DruidServer server = selector.pick(); - List descriptors = serverSegments.get(server); + final ServerSelector selector = segment.lhs; + final SegmentDescriptor descriptor = segment.rhs; + final Interval segmentQueryInterval = descriptor.getInterval(); - if (descriptors == null) { - descriptors = Lists.newArrayList(); - serverSegments.put(server, descriptors); + final byte[] cachedValue = cachedValues.get(segmentCacheKey); + + if (cachedValue != null) { + cachedResults.add(Pair.of(segmentQueryInterval.getStart(), cachedValue)); + + // remove cached segment from set of segments to query + segments.remove(segment); } - - descriptors.add(descriptor); - - if(segmentCacheKey != null) { + else { final String segmentIdentifier = selector.getSegment().getIdentifier(); cachePopulatorMap.put( String.format("%s_%s", segmentIdentifier, segmentQueryInterval), @@ -194,6 +194,19 @@ public class CachingClusteredClient implements QueryRunner } } + // Compile list of all segments not pulled from cache + for(Pair segment : segments) { + final DruidServer server = segment.lhs.pick(); + List descriptors = serverSegments.get(server); + + if (descriptors == null) { + descriptors = Lists.newArrayList(); + serverSegments.put(server, descriptors); + } + + descriptors.add(segment.rhs); + } + return new LazySequence( new Supplier>() From e0c34c3b972e94b4734216f2cbd28bafa1e02baf Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 18 Jan 2013 15:22:56 -0800 Subject: [PATCH 39/92] rename cacheBroker -> cache --- .../druid/client/CachingClusteredClient.java | 31 ++++++++------- .../cache/{CacheBroker.java => Cache.java} | 2 +- .../druid/client/cache/CacheMonitor.java | 8 ++-- .../{MapCacheBroker.java => MapCache.java} | 9 ++--- ...eBrokerConfig.java => MapCacheConfig.java} | 2 +- ...edCacheBroker.java => MemcachedCache.java} | 8 ++-- ...rConfig.java => MemcachedCacheConfig.java} | 2 +- .../com/metamx/druid/http/BrokerNode.java | 38 +++++++++---------- .../client/cache/MapCacheBrokerTest.java | 20 +++++----- .../cache/MemcachedCacheBrokerBenchmark.java | 16 ++++---- .../cache/MemcachedCacheBrokerTest.java | 30 +++++++-------- 11 files changed, 81 insertions(+), 85 deletions(-) rename client/src/main/java/com/metamx/druid/client/cache/{CacheBroker.java => Cache.java} (98%) rename client/src/main/java/com/metamx/druid/client/cache/{MapCacheBroker.java => MapCache.java} (96%) rename client/src/main/java/com/metamx/druid/client/cache/{MapCacheBrokerConfig.java => MapCacheConfig.java} (96%) rename client/src/main/java/com/metamx/druid/client/cache/{MemcachedCacheBroker.java => MemcachedCache.java} (95%) rename client/src/main/java/com/metamx/druid/client/cache/{MemcachedCacheBrokerConfig.java => MemcachedCacheConfig.java} (89%) diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index d69f91e6e14..5d640dc4e38 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -41,7 +41,7 @@ import com.metamx.druid.Query; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.aggregation.AggregatorFactory; -import com.metamx.druid.client.cache.CacheBroker; +import com.metamx.druid.client.cache.Cache; import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.query.CacheStrategy; @@ -57,7 +57,6 @@ import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.DateTime; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -76,19 +75,19 @@ public class CachingClusteredClient implements QueryRunner private final QueryToolChestWarehouse warehouse; private final ServerView serverView; - private final CacheBroker cacheBroker; + private final Cache cache; private final ObjectMapper objectMapper; public CachingClusteredClient( QueryToolChestWarehouse warehouse, ServerView serverView, - CacheBroker cacheBroker, + Cache cache, ObjectMapper objectMapper ) { this.warehouse = warehouse; this.serverView = serverView; - this.cacheBroker = cacheBroker; + this.cache = cache; this.objectMapper = objectMapper; serverView.registerSegmentCallback( @@ -100,7 +99,7 @@ public class CachingClusteredClient implements QueryRunner @Override public ServerView.CallbackAction segmentRemoved(DruidServer server, DataSegment segment) { - CachingClusteredClient.this.cacheBroker.close(segment.getIdentifier()); + CachingClusteredClient.this.cache.close(segment.getIdentifier()); return ServerView.CallbackAction.CONTINUE; } } @@ -161,16 +160,16 @@ public class CachingClusteredClient implements QueryRunner // Pull cached segments from cache and remove from set of segments to query if(useCache && queryCacheKey != null) { - Map, CacheBroker.NamedKey> cacheKeys = Maps.newHashMap(); + Map, Cache.NamedKey> cacheKeys = Maps.newHashMap(); for(Pair e : segments) { cacheKeys.put(e, computeSegmentCacheKey(e.lhs.getSegment().getIdentifier(), e.rhs, queryCacheKey)); } - Map cachedValues = cacheBroker.getBulk(cacheKeys.values()); + Map cachedValues = cache.getBulk(cacheKeys.values()); - for(Map.Entry, CacheBroker.NamedKey> entry : cacheKeys.entrySet()) { + for(Map.Entry, Cache.NamedKey> entry : cacheKeys.entrySet()) { Pair segment = entry.getKey(); - CacheBroker.NamedKey segmentCacheKey = entry.getValue(); + Cache.NamedKey segmentCacheKey = entry.getValue(); final ServerSelector selector = segment.lhs; final SegmentDescriptor descriptor = segment.rhs; @@ -188,7 +187,7 @@ public class CachingClusteredClient implements QueryRunner final String segmentIdentifier = selector.getSegment().getIdentifier(); cachePopulatorMap.put( String.format("%s_%s", segmentIdentifier, segmentQueryInterval), - new CachePopulator(cacheBroker, objectMapper, segmentCacheKey) + new CachePopulator(cache, objectMapper, segmentCacheKey) ); } } @@ -341,12 +340,12 @@ public class CachingClusteredClient implements QueryRunner ); } - private CacheBroker.NamedKey computeSegmentCacheKey(String segmentIdentifier, SegmentDescriptor descriptor, byte[] queryCacheKey) + private Cache.NamedKey computeSegmentCacheKey(String segmentIdentifier, SegmentDescriptor descriptor, byte[] queryCacheKey) { final Interval segmentQueryInterval = descriptor.getInterval(); final byte[] versionBytes = descriptor.getVersion().getBytes(); - return new CacheBroker.NamedKey( + return new Cache.NamedKey( segmentIdentifier, ByteBuffer .allocate(16 + versionBytes.length + 4 + queryCacheKey.length) .putLong(segmentQueryInterval.getStartMillis()) @@ -359,11 +358,11 @@ public class CachingClusteredClient implements QueryRunner private static class CachePopulator { - private final CacheBroker cache; + private final Cache cache; private final ObjectMapper mapper; - private final CacheBroker.NamedKey key; + private final Cache.NamedKey key; - public CachePopulator(CacheBroker cache, ObjectMapper mapper, CacheBroker.NamedKey key) + public CachePopulator(Cache cache, ObjectMapper mapper, Cache.NamedKey key) { this.cache = cache; this.mapper = mapper; diff --git a/client/src/main/java/com/metamx/druid/client/cache/CacheBroker.java b/client/src/main/java/com/metamx/druid/client/cache/Cache.java similarity index 98% rename from client/src/main/java/com/metamx/druid/client/cache/CacheBroker.java rename to client/src/main/java/com/metamx/druid/client/cache/Cache.java index adcba2e489c..9bf0cde33e7 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/CacheBroker.java +++ b/client/src/main/java/com/metamx/druid/client/cache/Cache.java @@ -26,7 +26,7 @@ import java.util.Map; /** */ -public interface CacheBroker +public interface Cache { public byte[] get(NamedKey key); public void put(NamedKey key, byte[] value); diff --git a/client/src/main/java/com/metamx/druid/client/cache/CacheMonitor.java b/client/src/main/java/com/metamx/druid/client/cache/CacheMonitor.java index d1337163ba6..b0c36629e89 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/CacheMonitor.java +++ b/client/src/main/java/com/metamx/druid/client/cache/CacheMonitor.java @@ -27,21 +27,21 @@ import com.metamx.metrics.AbstractMonitor; */ public class CacheMonitor extends AbstractMonitor { - private final CacheBroker cacheBroker; + private final Cache cache; private volatile CacheStats prevCacheStats = null; public CacheMonitor( - CacheBroker cacheBroker + Cache cache ) { - this.cacheBroker = cacheBroker; + this.cache = cache; } @Override public boolean doMonitor(ServiceEmitter emitter) { - final CacheStats currCacheStats = cacheBroker.getStats(); + final CacheStats currCacheStats = cache.getStats(); final CacheStats deltaCacheStats = currCacheStats.delta(prevCacheStats); final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); diff --git a/client/src/main/java/com/metamx/druid/client/cache/MapCacheBroker.java b/client/src/main/java/com/metamx/druid/client/cache/MapCache.java similarity index 96% rename from client/src/main/java/com/metamx/druid/client/cache/MapCacheBroker.java rename to client/src/main/java/com/metamx/druid/client/cache/MapCache.java index d541ab40a66..2780acd0f9e 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MapCacheBroker.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MapCache.java @@ -22,7 +22,6 @@ package com.metamx.druid.client.cache; import com.google.common.collect.Maps; import com.google.common.primitives.Ints; import com.metamx.common.ISE; -import com.metamx.common.Pair; import java.nio.ByteBuffer; import java.util.Collections; @@ -33,7 +32,7 @@ import java.util.concurrent.atomic.AtomicLong; /** */ -public class MapCacheBroker implements CacheBroker +public class MapCache implements Cache { /** * An interface to limit the operations that can be done on a Cache so that it is easier to reason about what @@ -57,9 +56,9 @@ public class MapCacheBroker implements CacheBroker private final AtomicLong hitCount = new AtomicLong(0); private final AtomicLong missCount = new AtomicLong(0); - public static CacheBroker create(final MapCacheBrokerConfig config) + public static com.metamx.druid.client.cache.Cache create(final MapCacheConfig config) { - return new MapCacheBroker( + return new MapCache( new ByteCountingLRUMap( config.getInitialSize(), config.getLogEvictionCount(), @@ -68,7 +67,7 @@ public class MapCacheBroker implements CacheBroker ); } - MapCacheBroker( + MapCache( ByteCountingLRUMap byteCountingLRUMap ) { diff --git a/client/src/main/java/com/metamx/druid/client/cache/MapCacheBrokerConfig.java b/client/src/main/java/com/metamx/druid/client/cache/MapCacheConfig.java similarity index 96% rename from client/src/main/java/com/metamx/druid/client/cache/MapCacheBrokerConfig.java rename to client/src/main/java/com/metamx/druid/client/cache/MapCacheConfig.java index 6ff7a778236..2bc468bf899 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MapCacheBrokerConfig.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MapCacheConfig.java @@ -24,7 +24,7 @@ import org.skife.config.Default; /** */ -public abstract class MapCacheBrokerConfig +public abstract class MapCacheConfig { @Config("${prefix}.sizeInBytes") @Default("0") diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBroker.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java similarity index 95% rename from client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBroker.java rename to client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java index fea391f7912..86cbf5153e5 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBroker.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java @@ -41,16 +41,16 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; -public class MemcachedCacheBroker implements CacheBroker +public class MemcachedCache implements Cache { - public static MemcachedCacheBroker create(final MemcachedCacheBrokerConfig config) + public static MemcachedCache create(final MemcachedCacheConfig config) { try { SerializingTranscoder transcoder = new SerializingTranscoder(config.getMaxObjectSize()); // disable compression transcoder.setCompressionThreshold(Integer.MAX_VALUE); - return new MemcachedCacheBroker( + return new MemcachedCache( new MemcachedClient( new ConnectionFactoryBuilder().setProtocol(ConnectionFactoryBuilder.Protocol.BINARY) .setHashAlg(DefaultHashAlgorithm.FNV1A_64_HASH) @@ -79,7 +79,7 @@ public class MemcachedCacheBroker implements CacheBroker private final AtomicLong missCount = new AtomicLong(0); private final AtomicLong timeoutCount = new AtomicLong(0); - MemcachedCacheBroker(MemcachedClientIF client, int timeout, int expiration) { + MemcachedCache(MemcachedClientIF client, int timeout, int expiration) { this.timeout = timeout; this.expiration = expiration; this.client = client; diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBrokerConfig.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java similarity index 89% rename from client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBrokerConfig.java rename to client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java index 5799d739bb6..83f626d8641 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheBrokerConfig.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java @@ -3,7 +3,7 @@ package com.metamx.druid.client.cache; import org.skife.config.Config; import org.skife.config.Default; -public abstract class MemcachedCacheBrokerConfig +public abstract class MemcachedCacheConfig { @Config("${prefix}.expiration") @Default("31536000") diff --git a/client/src/main/java/com/metamx/druid/http/BrokerNode.java b/client/src/main/java/com/metamx/druid/http/BrokerNode.java index aa14b440104..2a94a00d76d 100644 --- a/client/src/main/java/com/metamx/druid/http/BrokerNode.java +++ b/client/src/main/java/com/metamx/druid/http/BrokerNode.java @@ -34,13 +34,13 @@ import com.metamx.druid.client.BrokerServerView; import com.metamx.druid.client.CachingClusteredClient; import com.metamx.druid.client.ClientConfig; import com.metamx.druid.client.ClientInventoryManager; -import com.metamx.druid.client.cache.CacheBroker; +import com.metamx.druid.client.cache.Cache; import com.metamx.druid.client.cache.CacheConfig; import com.metamx.druid.client.cache.CacheMonitor; -import com.metamx.druid.client.cache.MapCacheBroker; -import com.metamx.druid.client.cache.MapCacheBrokerConfig; -import com.metamx.druid.client.cache.MemcachedCacheBroker; -import com.metamx.druid.client.cache.MemcachedCacheBrokerConfig; +import com.metamx.druid.client.cache.MapCache; +import com.metamx.druid.client.cache.MapCacheConfig; +import com.metamx.druid.client.cache.MemcachedCache; +import com.metamx.druid.client.cache.MemcachedCacheConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; @@ -78,7 +78,7 @@ public class BrokerNode extends QueryableNode private QueryToolChestWarehouse warehouse = null; private HttpClient brokerHttpClient = null; - private CacheBroker cacheBroker = null; + private Cache cache = null; private boolean useDiscovery = true; @@ -122,15 +122,15 @@ public class BrokerNode extends QueryableNode return this; } - public CacheBroker getCacheBroker() + public Cache getCache() { initializeCacheBroker(); - return cacheBroker; + return cache; } - public BrokerNode setCacheBroker(CacheBroker cacheBroker) + public BrokerNode setCache(Cache cache) { - checkFieldNotSetAndSet("cacheBroker", cacheBroker); + checkFieldNotSetAndSet("cache", cache); return this; } @@ -185,7 +185,7 @@ public class BrokerNode extends QueryableNode final Lifecycle lifecycle = getLifecycle(); final List monitors = getMonitors(); - monitors.add(new CacheMonitor(cacheBroker)); + monitors.add(new CacheMonitor(cache)); startMonitoring(monitors); final BrokerServerView view = new BrokerServerView(warehouse, getSmileMapper(), brokerHttpClient); @@ -194,7 +194,7 @@ public class BrokerNode extends QueryableNode ); lifecycle.addManagedInstance(clientInventoryManager); - final CachingClusteredClient baseClient = new CachingClusteredClient(warehouse, view, cacheBroker, getSmileMapper()); + final CachingClusteredClient baseClient = new CachingClusteredClient(warehouse, view, cache, getSmileMapper()); lifecycle.addManagedInstance(baseClient); @@ -239,25 +239,25 @@ public class BrokerNode extends QueryableNode private void initializeCacheBroker() { - if (cacheBroker == null) { + if (cache == null) { String cacheType = getConfigFactory() .build(CacheConfig.class) .getType(); if (cacheType.equals(CACHE_TYPE_LOCAL)) { - setCacheBroker( - MapCacheBroker.create( + setCache( + MapCache.create( getConfigFactory().buildWithReplacements( - MapCacheBrokerConfig.class, + MapCacheConfig.class, ImmutableMap.of("prefix", CACHE_PROPERTY_PREFIX) ) ) ); } else if (cacheType.equals(CACHE_TYPE_MEMCACHED)) { - setCacheBroker( - MemcachedCacheBroker.create( + setCache( + MemcachedCache.create( getConfigFactory().buildWithReplacements( - MemcachedCacheBrokerConfig.class, + MemcachedCacheConfig.class, ImmutableMap.of("prefix", CACHE_PROPERTY_PREFIX) ) ) diff --git a/client/src/test/java/com/metamx/druid/client/cache/MapCacheBrokerTest.java b/client/src/test/java/com/metamx/druid/client/cache/MapCacheBrokerTest.java index 35ea7ac42d9..78f071ca539 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MapCacheBrokerTest.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MapCacheBrokerTest.java @@ -31,24 +31,24 @@ public class MapCacheBrokerTest private static final byte[] HI = "hi".getBytes(); private static final byte[] HO = "ho".getBytes(); private ByteCountingLRUMap baseMap; - private MapCacheBroker cache; + private MapCache cache; @Before public void setUp() throws Exception { baseMap = new ByteCountingLRUMap(1024 * 1024); - cache = new MapCacheBroker(baseMap); + cache = new MapCache(baseMap); } @Test public void testSanity() throws Exception { - Assert.assertNull(cache.get(new CacheBroker.NamedKey("a", HI))); + Assert.assertNull(cache.get(new Cache.NamedKey("a", HI))); Assert.assertEquals(0, baseMap.size()); put(cache, "a", HI, 1); Assert.assertEquals(1, baseMap.size()); Assert.assertEquals(1, get(cache, "a", HI)); - Assert.assertNull(cache.get(new CacheBroker.NamedKey("the", HI))); + Assert.assertNull(cache.get(new Cache.NamedKey("the", HI))); put(cache, "the", HI, 2); Assert.assertEquals(2, baseMap.size()); @@ -58,26 +58,26 @@ public class MapCacheBrokerTest put(cache, "the", HO, 10); Assert.assertEquals(3, baseMap.size()); Assert.assertEquals(1, get(cache, "a", HI)); - Assert.assertNull(cache.get(new CacheBroker.NamedKey("a", HO))); + Assert.assertNull(cache.get(new Cache.NamedKey("a", HO))); Assert.assertEquals(2, get(cache, "the", HI)); Assert.assertEquals(10, get(cache, "the", HO)); cache.close("the"); Assert.assertEquals(1, baseMap.size()); Assert.assertEquals(1, get(cache, "a", HI)); - Assert.assertNull(cache.get(new CacheBroker.NamedKey("a", HO))); + Assert.assertNull(cache.get(new Cache.NamedKey("a", HO))); cache.close("a"); Assert.assertEquals(0, baseMap.size()); } - public void put(CacheBroker cache, String namespace, byte[] key, Integer value) + public void put(Cache cache, String namespace, byte[] key, Integer value) { - cache.put(new CacheBroker.NamedKey(namespace, key), Ints.toByteArray(value)); + cache.put(new Cache.NamedKey(namespace, key), Ints.toByteArray(value)); } - public int get(CacheBroker cache, String namespace, byte[] key) + public int get(Cache cache, String namespace, byte[] key) { - return Ints.fromByteArray(cache.get(new CacheBroker.NamedKey(namespace, key))); + return Ints.fromByteArray(cache.get(new Cache.NamedKey(namespace, key))); } } diff --git a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java index 7c25f3ee954..d87dfd5f7a1 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java @@ -22,7 +22,7 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark private static final String BASE_KEY = "test_2012-11-26T00:00:00.000Z_2012-11-27T00:00:00.000Z_2012-11-27T04:11:25.979Z_"; public static final String NAMESPACE = "default"; - private MemcachedCacheBroker cache; + private MemcachedCache cache; private MemcachedClientIF client; private static byte[] randBytes; @@ -54,7 +54,7 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark AddrUtil.getAddresses(hosts) ); - cache = new MemcachedCacheBroker( + cache = new MemcachedCache( client, 30000, // 30 seconds 3600 // 1 hour @@ -75,7 +75,7 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark for(int i = 0; i < reps; ++i) { for(int k = 0; k < objectCount; ++k) { String key = BASE_KEY + k; - cache.put(new CacheBroker.NamedKey(NAMESPACE, key.getBytes()), randBytes); + cache.put(new Cache.NamedKey(NAMESPACE, key.getBytes()), randBytes); } // make sure the write queue is empty client.waitForQueues(1, TimeUnit.HOURS); @@ -88,7 +88,7 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark for (int i = 0; i < reps; i++) { for(int k = 0; k < objectCount; ++k) { String key = BASE_KEY + k; - bytes = cache.get(new CacheBroker.NamedKey(NAMESPACE, key.getBytes())); + bytes = cache.get(new Cache.NamedKey(NAMESPACE, key.getBytes())); count += bytes.length; } } @@ -98,13 +98,13 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark public long timeBulkGetObjects(int reps) { long count = 0; for (int i = 0; i < reps; i++) { - List keys = Lists.newArrayList(); + List keys = Lists.newArrayList(); for(int k = 0; k < objectCount; ++k) { String key = BASE_KEY + k; - keys.add(new CacheBroker.NamedKey(NAMESPACE, key.getBytes())); + keys.add(new Cache.NamedKey(NAMESPACE, key.getBytes())); } - Map results = cache.getBulk(keys); - for(CacheBroker.NamedKey key : keys) { + Map results = cache.getBulk(keys); + for(Cache.NamedKey key : keys) { byte[] bytes = results.get(key); count += bytes.length; } diff --git a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java index 31030260df7..87c1dcdd9f4 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java @@ -22,7 +22,6 @@ package com.metamx.druid.client.cache; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.primitives.Ints; -import com.metamx.common.Pair; import net.spy.memcached.CASResponse; import net.spy.memcached.CASValue; import net.spy.memcached.CachedData; @@ -38,7 +37,6 @@ import org.junit.Before; import org.junit.Test; import java.net.SocketAddress; -import java.nio.ByteBuffer; import java.util.Collection; import java.util.Iterator; import java.util.Map; @@ -56,22 +54,22 @@ public class MemcachedCacheBrokerTest { private static final byte[] HI = "hi".getBytes(); private static final byte[] HO = "ho".getBytes(); - private MemcachedCacheBroker cache; + private MemcachedCache cache; @Before public void setUp() throws Exception { MemcachedClientIF client = new MockMemcachedClient(); - cache = new MemcachedCacheBroker(client, 500, 3600); + cache = new MemcachedCache(client, 500, 3600); } @Test public void testSanity() throws Exception { - Assert.assertNull(cache.get(new CacheBroker.NamedKey("a", HI))); + Assert.assertNull(cache.get(new Cache.NamedKey("a", HI))); put(cache, "a", HI, 1); Assert.assertEquals(1, get(cache, "a", HI)); - Assert.assertNull(cache.get(new CacheBroker.NamedKey("the", HI))); + Assert.assertNull(cache.get(new Cache.NamedKey("the", HI))); put(cache, "the", HI, 2); Assert.assertEquals(1, get(cache, "a", HI)); @@ -79,13 +77,13 @@ public class MemcachedCacheBrokerTest put(cache, "the", HO, 10); Assert.assertEquals(1, get(cache, "a", HI)); - Assert.assertNull(cache.get(new CacheBroker.NamedKey("a", HO))); + Assert.assertNull(cache.get(new Cache.NamedKey("a", HO))); Assert.assertEquals(2, get(cache, "the", HI)); Assert.assertEquals(10, get(cache, "the", HO)); cache.close("the"); Assert.assertEquals(1, get(cache, "a", HI)); - Assert.assertNull(cache.get(new CacheBroker.NamedKey("a", HO))); + Assert.assertNull(cache.get(new Cache.NamedKey("a", HO))); cache.close("a"); } @@ -93,15 +91,15 @@ public class MemcachedCacheBrokerTest @Test public void testGetBulk() throws Exception { - Assert.assertNull(cache.get(new CacheBroker.NamedKey("the", HI))); + Assert.assertNull(cache.get(new Cache.NamedKey("the", HI))); put(cache, "the", HI, 2); put(cache, "the", HO, 10); - CacheBroker.NamedKey key1 = new CacheBroker.NamedKey("the", HI); - CacheBroker.NamedKey key2 = new CacheBroker.NamedKey("the", HO); + Cache.NamedKey key1 = new Cache.NamedKey("the", HI); + Cache.NamedKey key2 = new Cache.NamedKey("the", HO); - Map result = cache.getBulk( + Map result = cache.getBulk( Lists.newArrayList( key1, key2 @@ -112,14 +110,14 @@ public class MemcachedCacheBrokerTest Assert.assertEquals(10, Ints.fromByteArray(result.get(key2))); } - public void put(CacheBroker cache, String namespace, byte[] key, Integer value) + public void put(Cache cache, String namespace, byte[] key, Integer value) { - cache.put(new CacheBroker.NamedKey(namespace, key), Ints.toByteArray(value)); + cache.put(new Cache.NamedKey(namespace, key), Ints.toByteArray(value)); } - public int get(CacheBroker cache, String namespace, byte[] key) + public int get(Cache cache, String namespace, byte[] key) { - return Ints.fromByteArray(cache.get(new CacheBroker.NamedKey(namespace, key))); + return Ints.fromByteArray(cache.get(new Cache.NamedKey(namespace, key))); } } From ac31afbce57c2974b044ed8de1a9ed224e2eb9ad Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 18 Jan 2013 16:45:23 -0800 Subject: [PATCH 40/92] remove redundant index for primary key in config table --- common/src/main/java/com/metamx/druid/db/DbConnector.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/src/main/java/com/metamx/druid/db/DbConnector.java b/common/src/main/java/com/metamx/druid/db/DbConnector.java index f3c4c1f13d3..73013ce6aa2 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnector.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnector.java @@ -65,7 +65,7 @@ public class DbConnector dbi, configTableName, String.format( - "CREATE table %s (name VARCHAR(255) NOT NULL, payload LONGTEXT NOT NULL, INDEX(name), PRIMARY KEY(name))", + "CREATE table %s (name VARCHAR(255) NOT NULL, payload LONGTEXT NOT NULL, PRIMARY KEY(name))", configTableName ) ); From 40c0bcad29f5152cff7a880ca124b65aaca41643 Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 18 Jan 2013 18:25:51 -0800 Subject: [PATCH 41/92] simplify MapCache --- .../metamx/druid/client/cache/MapCache.java | 147 ++++++------------ 1 file changed, 51 insertions(+), 96 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/cache/MapCache.java b/client/src/main/java/com/metamx/druid/client/cache/MapCache.java index 2780acd0f9e..53e1e20280a 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MapCache.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MapCache.java @@ -21,7 +21,6 @@ package com.metamx.druid.client.cache; import com.google.common.collect.Maps; import com.google.common.primitives.Ints; -import com.metamx.common.ISE; import java.nio.ByteBuffer; import java.util.Collections; @@ -34,21 +33,10 @@ import java.util.concurrent.atomic.AtomicLong; */ public class MapCache implements Cache { - /** - * An interface to limit the operations that can be done on a Cache so that it is easier to reason about what - * is actually going to be done. - */ - public interface Cache - { - public byte[] get(byte[] key); - public void put(byte[] key, byte[] value); - public void close(); - } - private final Map baseMap; private final ByteCountingLRUMap byteCountingLRUMap; - private final Map cacheCache; + private final Map namespaceId; private final AtomicInteger ids; private final Object clearLock = new Object(); @@ -75,7 +63,7 @@ public class MapCache implements Cache this.baseMap = Collections.synchronizedMap(byteCountingLRUMap); - cacheCache = Maps.newHashMap(); + namespaceId = Maps.newHashMap(); ids = new AtomicInteger(); } @@ -95,22 +83,29 @@ public class MapCache implements Cache @Override public byte[] get(NamedKey key) { - return provideCache(key.namespace).get(key.key); + final byte[] retVal = baseMap.get(computeKey(getNamespaceId(key.namespace), key.key)); + if (retVal == null) { + missCount.incrementAndGet(); + } else { + hitCount.incrementAndGet(); + } + return retVal; } @Override public void put(NamedKey key, byte[] value) { - provideCache(key.namespace).put(key.key, value); + synchronized (clearLock) { + baseMap.put(computeKey(getNamespaceId(key.namespace), key.key), value); + } } @Override public Map getBulk(Iterable keys) { Map retVal = Maps.newHashMap(); - for(NamedKey key : keys) { - retVal.put(key, provideCache(key.namespace).get(key.key)); + retVal.put(key, get(key)); } return retVal; } @@ -118,86 +113,46 @@ public class MapCache implements Cache @Override public void close(String namespace) { - provideCache(namespace).close(); - } + byte[] idBytes; + synchronized (namespaceId) { + idBytes = getNamespaceId(namespace); + if(idBytes == null) return; - private Cache provideCache(final String identifier) - { - synchronized (cacheCache) { - final Cache cachedCache = cacheCache.get(identifier); - if (cachedCache != null) { - return cachedCache; + namespaceId.remove(namespace); + } + synchronized (clearLock) { + Iterator iter = baseMap.keySet().iterator(); + while (iter.hasNext()) { + ByteBuffer next = iter.next(); + + if (next.get(0) == idBytes[0] + && next.get(1) == idBytes[1] + && next.get(2) == idBytes[2] + && next.get(3) == idBytes[3]) { + iter.remove(); + } } - - final byte[] myIdBytes = Ints.toByteArray(ids.getAndIncrement()); - - final Cache theCache = new Cache() - { - volatile boolean open = true; - - @Override - public byte[] get(byte[] key) - { - if (open) { - final byte[] retVal = baseMap.get(computeKey(key)); - if (retVal == null) { - missCount.incrementAndGet(); - } else { - hitCount.incrementAndGet(); - } - return retVal; - } - throw new ISE("Cache for namespace[%s] is closed.", identifier); - } - - @Override - public void put(byte[] key, byte[] value) - { - synchronized (clearLock) { - if (open) { - baseMap.put(computeKey(key), value); - return; - } - } - throw new ISE("Cache for namespace[%s] is closed.", identifier); - } - - @Override - public void close() - { - synchronized (cacheCache) { - cacheCache.remove(identifier); - } - synchronized (clearLock) { - if (open) { - open = false; - - Iterator iter = baseMap.keySet().iterator(); - while (iter.hasNext()) { - ByteBuffer next = iter.next(); - - if (next.get(0) == myIdBytes[0] - && next.get(1) == myIdBytes[1] - && next.get(2) == myIdBytes[2] - && next.get(3) == myIdBytes[3]) { - iter.remove(); - } - } - } - } - } - - private ByteBuffer computeKey(byte[] key) - { - final ByteBuffer retVal = ByteBuffer.allocate(key.length + 4).put(myIdBytes).put(key); - retVal.rewind(); - return retVal; - } - }; - - cacheCache.put(identifier, theCache); - - return theCache; } } + + private byte[] getNamespaceId(final String identifier) + { + synchronized (namespaceId) { + byte[] idBytes = namespaceId.get(identifier); + if (idBytes != null) { + return idBytes; + } + + idBytes = Ints.toByteArray(ids.getAndIncrement()); + namespaceId.put(identifier, idBytes); + return idBytes; + } + } + + private ByteBuffer computeKey(byte[] idBytes, byte[] key) + { + final ByteBuffer retVal = ByteBuffer.allocate(key.length + 4).put(idBytes).put(key); + retVal.rewind(); + return retVal; + } } From 068ca67dbaa44164af5e816ed49a7c1b0764d95c Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 21 Jan 2013 14:21:01 -0800 Subject: [PATCH 42/92] fix cache not preserving timezone information --- .../query/timeseries/TimeseriesQueryQueryToolChest.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 99bf679c006..0e5d96b385c 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -191,7 +191,8 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest retVal = Lists.newArrayListWithCapacity(1 + aggs.size()); - retVal.add(input.getTimestamp().getMillis()); + // make sure to preserve timezone information when caching results + retVal.add(input.getTimestamp()); for (AggregatorFactory agg : aggs) { retVal.add(results.getMetric(agg.getName())); } @@ -215,7 +216,7 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest aggsIter = aggs.iterator(); Iterator resultIter = results.iterator(); - DateTime timestamp = new DateTime(resultIter.next()); + DateTime timestamp = (DateTime)resultIter.next(); while (aggsIter.hasNext() && resultIter.hasNext()) { final AggregatorFactory factory = aggsIter.next(); retVal.put(factory.getName(), factory.deserialize(resultIter.next())); From d9e6f1d95460ba363a9d6c26d5a0452a6464d5d1 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 21 Jan 2013 14:38:35 -0800 Subject: [PATCH 43/92] DeterminePartitions follow-up HadoopDruidIndexerConfig: - Add partitionsSpec (backwards compatible with targetPartitionSize and partitionDimension) - Add assumeGrouped flag to partitionsSpec DeterminePartitionsJob: - Skip group-by job if assumeGrouped is set - Clean up code a bit --- .../druid/indexer/DeterminePartitionsJob.java | 194 +++++++++++++----- .../indexer/HadoopDruidIndexerConfig.java | 142 +++++++++---- .../indexer/HadoopDruidIndexerMapper.java | 9 +- .../druid/indexer/IndexGeneratorJob.java | 2 +- .../indexer/partitions/PartitionsSpec.java | 52 +++++ .../indexer/HadoopDruidIndexerConfigTest.java | 164 ++++++++++++++- 6 files changed, 457 insertions(+), 106 deletions(-) create mode 100644 indexer/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index e257d4ed6b7..0e6a686a0f7 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -38,6 +38,7 @@ import com.metamx.common.ISE; import com.metamx.common.guava.nary.BinaryFn; import com.metamx.common.logger.Logger; import com.metamx.druid.CombiningIterable; +import com.metamx.druid.QueryGranularity; import com.metamx.druid.input.InputRow; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.ShardSpec; @@ -103,8 +104,8 @@ public class DeterminePartitionsJob implements Jobby public static void injectSystemProperties(Job job) { + final Configuration conf = job.getConfiguration(); for (String propName : System.getProperties().stringPropertyNames()) { - Configuration conf = job.getConfiguration(); if (propName.startsWith("hadoop.")) { conf.set(propName.substring("hadoop.".length()), System.getProperty(propName)); } @@ -118,37 +119,42 @@ public class DeterminePartitionsJob implements Jobby * Group by (timestamp, dimensions) so we can correctly count dimension values as they would appear * in the final segment. */ - final Job groupByJob = new Job( - new Configuration(), - String.format("%s-determine_partitions_groupby-%s", config.getDataSource(), config.getIntervals()) - ); - injectSystemProperties(groupByJob); - groupByJob.setInputFormatClass(TextInputFormat.class); - groupByJob.setMapperClass(DeterminePartitionsGroupByMapper.class); - groupByJob.setMapOutputKeyClass(Text.class); - groupByJob.setMapOutputValueClass(NullWritable.class); - groupByJob.setCombinerClass(DeterminePartitionsGroupByReducer.class); - groupByJob.setReducerClass(DeterminePartitionsGroupByReducer.class); - groupByJob.setOutputKeyClass(Text.class); - groupByJob.setOutputValueClass(NullWritable.class); - groupByJob.setOutputFormatClass(SequenceFileOutputFormat.class); - groupByJob.setJarByClass(DeterminePartitionsJob.class); + if(!config.getPartitionsSpec().isAssumeGrouped()) { + final Job groupByJob = new Job( + new Configuration(), + String.format("%s-determine_partitions_groupby-%s", config.getDataSource(), config.getIntervals()) + ); - config.addInputPaths(groupByJob); - config.intoConfiguration(groupByJob); - FileOutputFormat.setOutputPath(groupByJob, config.makeGroupedDataDir()); + injectSystemProperties(groupByJob); + groupByJob.setInputFormatClass(TextInputFormat.class); + groupByJob.setMapperClass(DeterminePartitionsGroupByMapper.class); + groupByJob.setMapOutputKeyClass(BytesWritable.class); + groupByJob.setMapOutputValueClass(NullWritable.class); + groupByJob.setCombinerClass(DeterminePartitionsGroupByReducer.class); + groupByJob.setReducerClass(DeterminePartitionsGroupByReducer.class); + groupByJob.setOutputKeyClass(BytesWritable.class); + groupByJob.setOutputValueClass(NullWritable.class); + groupByJob.setOutputFormatClass(SequenceFileOutputFormat.class); + groupByJob.setJarByClass(DeterminePartitionsJob.class); - groupByJob.submit(); - log.info("Job submitted, status available at: %s", groupByJob.getTrackingURL()); + config.addInputPaths(groupByJob); + config.intoConfiguration(groupByJob); + FileOutputFormat.setOutputPath(groupByJob, config.makeGroupedDataDir()); - if(!groupByJob.waitForCompletion(true)) { - log.error("Job failed: %s", groupByJob.getJobID().toString()); - return false; + groupByJob.submit(); + log.info("Job %s submitted, status available at: %s", groupByJob.getJobName(), groupByJob.getTrackingURL()); + + if(!groupByJob.waitForCompletion(true)) { + log.error("Job failed: %s", groupByJob.getJobID().toString()); + return false; + } + } else { + log.info("Skipping group-by job."); } /* - * Read grouped data from previous job and determine appropriate partitions. + * Read grouped data and determine appropriate partitions. */ final Job dimSelectionJob = new Job( new Configuration(), @@ -158,8 +164,19 @@ public class DeterminePartitionsJob implements Jobby dimSelectionJob.getConfiguration().set("io.sort.record.percent", "0.19"); injectSystemProperties(dimSelectionJob); - dimSelectionJob.setInputFormatClass(SequenceFileInputFormat.class); - dimSelectionJob.setMapperClass(DeterminePartitionsDimSelectionMapper.class); + + if(!config.getPartitionsSpec().isAssumeGrouped()) { + // Read grouped data from the groupByJob. + dimSelectionJob.setMapperClass(DeterminePartitionsDimSelectionPostGroupByMapper.class); + dimSelectionJob.setInputFormatClass(SequenceFileInputFormat.class); + FileInputFormat.addInputPath(dimSelectionJob, config.makeGroupedDataDir()); + } else { + // Directly read the source data, since we assume it's already grouped. + dimSelectionJob.setMapperClass(DeterminePartitionsDimSelectionAssumeGroupedMapper.class); + dimSelectionJob.setInputFormatClass(TextInputFormat.class); + config.addInputPaths(dimSelectionJob); + } + SortableBytes.useSortableBytesAsMapOutputKey(dimSelectionJob); dimSelectionJob.setMapOutputValueClass(Text.class); dimSelectionJob.setCombinerClass(DeterminePartitionsDimSelectionCombiner.class); @@ -170,11 +187,14 @@ public class DeterminePartitionsJob implements Jobby dimSelectionJob.setJarByClass(DeterminePartitionsJob.class); config.intoConfiguration(dimSelectionJob); - FileInputFormat.addInputPath(dimSelectionJob, config.makeGroupedDataDir()); FileOutputFormat.setOutputPath(dimSelectionJob, config.makeIntermediatePath()); dimSelectionJob.submit(); - log.info("Job submitted, status available at: %s", dimSelectionJob.getTrackingURL()); + log.info( + "Job %s submitted, status available at: %s", + dimSelectionJob.getJobName(), + dimSelectionJob.getTrackingURL() + ); if(!dimSelectionJob.waitForCompletion(true)) { log.error("Job failed: %s", dimSelectionJob.getJobID().toString()); @@ -223,8 +243,18 @@ public class DeterminePartitionsJob implements Jobby } } - public static class DeterminePartitionsGroupByMapper extends HadoopDruidIndexerMapper + public static class DeterminePartitionsGroupByMapper extends HadoopDruidIndexerMapper { + private QueryGranularity rollupGranularity = null; + + @Override + protected void setup(Context context) + throws IOException, InterruptedException + { + super.setup(context); + rollupGranularity = getConfig().getRollupSpec().getRollupGranularity(); + } + @Override protected void innerMap( InputRow inputRow, @@ -242,18 +272,22 @@ public class DeterminePartitionsJob implements Jobby } } final List groupKey = ImmutableList.of( - getConfig().getRollupSpec().getRollupGranularity().truncate(inputRow.getTimestampFromEpoch()), + rollupGranularity.truncate(inputRow.getTimestampFromEpoch()), dims ); - context.write(new Text(HadoopDruidIndexerConfig.jsonMapper.writeValueAsBytes(groupKey)), NullWritable.get()); + context.write( + new BytesWritable(HadoopDruidIndexerConfig.jsonMapper.writeValueAsBytes(groupKey)), + NullWritable.get() + ); } } - public static class DeterminePartitionsGroupByReducer extends Reducer + public static class DeterminePartitionsGroupByReducer + extends Reducer { @Override protected void reduce( - Text key, + BytesWritable key, Iterable values, Context context ) throws IOException, InterruptedException @@ -262,31 +296,91 @@ public class DeterminePartitionsJob implements Jobby } } - public static class DeterminePartitionsDimSelectionMapper extends Mapper + /** + * This DimSelection mapper runs on data generated by our GroupBy job. + */ + public static class DeterminePartitionsDimSelectionPostGroupByMapper + extends Mapper { - private HadoopDruidIndexerConfig config; - private String partitionDimension; + private DeterminePartitionsDimSelectionMapperHelper helper; @Override protected void setup(Context context) throws IOException, InterruptedException { - config = HadoopDruidIndexerConfig.fromConfiguration(context.getConfiguration()); - partitionDimension = config.getPartitionDimension(); + final HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromConfiguration(context.getConfiguration()); + final String partitionDimension = config.getPartitionDimension(); + helper = new DeterminePartitionsDimSelectionMapperHelper(config, partitionDimension); } @Override protected void map( - Text key, NullWritable value, Context context + BytesWritable key, NullWritable value, Context context ) throws IOException, InterruptedException { - final List timeAndDims = - HadoopDruidIndexerConfig.jsonMapper.readValue( - key.getBytes(), new TypeReference>() {} - ); + final List timeAndDims = HadoopDruidIndexerConfig.jsonMapper.readValue(key.getBytes(), List.class); final DateTime timestamp = new DateTime(timeAndDims.get(0)); - final Map dims = (Map) timeAndDims.get(1); + final Map> dims = (Map>) timeAndDims.get(1); + + helper.emitDimValueCounts(context, timestamp, dims); + } + } + + /** + * This DimSelection mapper runs on raw input data that we assume has already been grouped. + */ + public static class DeterminePartitionsDimSelectionAssumeGroupedMapper + extends HadoopDruidIndexerMapper + { + private DeterminePartitionsDimSelectionMapperHelper helper; + + @Override + protected void setup(Context context) + throws IOException, InterruptedException + { + super.setup(context); + final HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromConfiguration(context.getConfiguration()); + final String partitionDimension = config.getPartitionDimension(); + helper = new DeterminePartitionsDimSelectionMapperHelper(config, partitionDimension); + } + + @Override + protected void innerMap( + InputRow inputRow, + Text text, + Context context + ) throws IOException, InterruptedException + { + final Map> dims = Maps.newHashMap(); + for(final String dim : inputRow.getDimensions()) { + dims.put(dim, inputRow.getDimension(dim)); + } + helper.emitDimValueCounts(context, new DateTime(inputRow.getTimestampFromEpoch()), dims); + } + } + + /** + * Since we have two slightly different DimSelectionMappers, this class encapsulates the shared logic for + * emitting dimension value counts. + */ + public static class DeterminePartitionsDimSelectionMapperHelper + { + private final HadoopDruidIndexerConfig config; + private final String partitionDimension; + + public DeterminePartitionsDimSelectionMapperHelper(HadoopDruidIndexerConfig config, String partitionDimension) + { + this.config = config; + this.partitionDimension = partitionDimension; + } + + public void emitDimValueCounts( + TaskInputOutputContext context, + DateTime timestamp, + Map> dims + ) throws IOException, InterruptedException + { final Optional maybeInterval = config.getGranularitySpec().bucketInterval(timestamp); if(!maybeInterval.isPresent()) { @@ -296,15 +390,15 @@ public class DeterminePartitionsJob implements Jobby final Interval interval = maybeInterval.get(); final byte[] groupKey = interval.getStart().toString().getBytes(Charsets.UTF_8); - for(final Map.Entry dimAndValues : dims.entrySet()) { + for(final Map.Entry> dimAndValues : dims.entrySet()) { final String dim = dimAndValues.getKey(); if(partitionDimension == null || partitionDimension.equals(dim)) { - final List dimValues = (List) dimAndValues.getValue(); + final Iterable dimValues = dimAndValues.getValue(); - if(dimValues.size() == 1) { + if(Iterables.size(dimValues) == 1) { // Emit this value. - write(context, groupKey, new DimValueCount(dim, dimValues.get(0), 1)); + write(context, groupKey, new DimValueCount(dim, Iterables.getOnlyElement(dimValues), 1)); } else { // This dimension is unsuitable for partitioning. Poison it by emitting a negative value. write(context, groupKey, new DimValueCount(dim, "", -1)); @@ -597,8 +691,6 @@ public class DeterminePartitionsJob implements Jobby } try { - // For some reason this used to work without writerWithType, but now it appears to forget to write "type" - // info for the ShardSpecs (so they cannot be deserialized). HadoopDruidIndexerConfig.jsonMapper.writerWithType(new TypeReference>() {}).writeValue( out, chosenShardSpecs diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java index d5034ef9966..3d682dadce0 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java @@ -43,6 +43,7 @@ import com.metamx.druid.indexer.data.TimestampSpec; import com.metamx.druid.indexer.data.ToLowercaseDataSpec; import com.metamx.druid.indexer.granularity.GranularitySpec; import com.metamx.druid.indexer.granularity.UniformGranularitySpec; +import com.metamx.druid.indexer.partitions.PartitionsSpec; import com.metamx.druid.indexer.path.PathSpec; import com.metamx.druid.indexer.rollup.DataRollupSpec; import com.metamx.druid.indexer.updater.UpdaterJobSpec; @@ -54,6 +55,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.Job; import org.codehaus.jackson.JsonGenerator; +import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; import org.codehaus.jackson.map.ObjectMapper; import org.codehaus.jackson.type.TypeReference; @@ -164,8 +166,6 @@ public class HadoopDruidIndexerConfig private static final String CONFIG_PROPERTY = "druid.indexer.config"; - @Deprecated - private volatile List intervals; private volatile String dataSource; private volatile String timestampColumnName; private volatile String timestampFormat; @@ -177,8 +177,7 @@ public class HadoopDruidIndexerConfig private volatile String jobOutputDir; private volatile String segmentOutputDir; private volatile DateTime version = new DateTime(); - private volatile String partitionDimension; - private volatile Long targetPartitionSize; + private volatile PartitionsSpec partitionsSpec; private volatile boolean leaveIntermediate = false; private volatile boolean cleanupOnFailure = true; private volatile Map> shardSpecs = ImmutableMap.of(); @@ -188,22 +187,97 @@ public class HadoopDruidIndexerConfig private volatile boolean ignoreInvalidRows = false; private volatile List registererers = Lists.newArrayList(); + @JsonCreator + public HadoopDruidIndexerConfig( + final @JsonProperty("intervals") List intervals, + final @JsonProperty("dataSource") String dataSource, + final @JsonProperty("timestampColumnName") String timestampColumnName, + final @JsonProperty("timestampFormat") String timestampFormat, + final @JsonProperty("dataSpec") DataSpec dataSpec, + final @JsonProperty("segmentGranularity") Granularity segmentGranularity, + final @JsonProperty("granularitySpec") GranularitySpec granularitySpec, + final @JsonProperty("pathSpec") PathSpec pathSpec, + final @JsonProperty("jobOutputDir") String jobOutputDir, + final @JsonProperty("segmentOutputDir") String segmentOutputDir, + final @JsonProperty("version") DateTime version, + final @JsonProperty("partitionDimension") String partitionDimension, + final @JsonProperty("targetPartitionSize") Long targetPartitionSize, + final @JsonProperty("partitionsSpec") PartitionsSpec partitionsSpec, + final @JsonProperty("leaveIntermediate") boolean leaveIntermediate, + final @JsonProperty("cleanupOnFailure") boolean cleanupOnFailure, + final @JsonProperty("shardSpecs") Map> shardSpecs, + final @JsonProperty("overwriteFiles") boolean overwriteFiles, + final @JsonProperty("rollupSpec") DataRollupSpec rollupSpec, + final @JsonProperty("updaterJobSpec") UpdaterJobSpec updaterJobSpec, + final @JsonProperty("ignoreInvalidRows") boolean ignoreInvalidRows, + final @JsonProperty("registererers") List registererers + ) + { + this.dataSource = dataSource; + this.timestampColumnName = timestampColumnName; + this.timestampFormat = timestampFormat; + this.dataSpec = dataSpec; + this.granularitySpec = granularitySpec; + this.pathSpec = pathSpec; + this.jobOutputDir = jobOutputDir; + this.segmentOutputDir = segmentOutputDir; + this.version = version; + this.partitionsSpec = partitionsSpec; + this.leaveIntermediate = leaveIntermediate; + this.cleanupOnFailure = cleanupOnFailure; + this.shardSpecs = shardSpecs; + this.overwriteFiles = overwriteFiles; + this.rollupSpec = rollupSpec; + this.updaterJobSpec = updaterJobSpec; + this.ignoreInvalidRows = ignoreInvalidRows; + this.registererers = registererers; + + if(partitionsSpec != null) { + Preconditions.checkArgument( + partitionDimension == null && targetPartitionSize == null, + "Cannot mix partitionsSpec with partitionDimension/targetPartitionSize" + ); + + this.partitionsSpec = partitionsSpec; + } else { + // Backwards compatibility + this.partitionsSpec = new PartitionsSpec(partitionDimension, targetPartitionSize, false); + } + + if(granularitySpec != null) { + Preconditions.checkArgument( + segmentGranularity == null && intervals == null, + "Cannot mix granularitySpec with segmentGranularity/intervals" + ); + } else { + // Backwards compatibility + this.segmentGranularity = segmentGranularity; + if(segmentGranularity != null && intervals != null) { + this.granularitySpec = new UniformGranularitySpec(segmentGranularity, intervals); + } + } + } + + /** + * Default constructor does nothing. The caller is expected to use the various setX methods. + */ + public HadoopDruidIndexerConfig() + { + } + public List getIntervals() { return JodaUtils.condenseIntervals(getGranularitySpec().bucketIntervals()); } @Deprecated - @JsonProperty public void setIntervals(List intervals) { - Preconditions.checkState(this.granularitySpec == null, "Use setGranularitySpec"); + Preconditions.checkState(this.granularitySpec == null, "Cannot mix setIntervals with granularitySpec"); + Preconditions.checkState(this.segmentGranularity != null, "Cannot use setIntervals without segmentGranularity"); // For backwards compatibility - this.intervals = intervals; - if (this.segmentGranularity != null) { - this.granularitySpec = new UniformGranularitySpec(this.segmentGranularity, this.intervals); - } + this.granularitySpec = new UniformGranularitySpec(this.segmentGranularity, intervals); } @JsonProperty @@ -281,19 +355,6 @@ public class HadoopDruidIndexerConfig return new StringInputRowParser(getTimestampSpec(), getDataSpec(), dimensionExclusions); } - @Deprecated - @JsonProperty - public void setSegmentGranularity(Granularity segmentGranularity) - { - Preconditions.checkState(this.granularitySpec == null, "Use setGranularitySpec"); - - // For backwards compatibility - this.segmentGranularity = segmentGranularity; - if (this.intervals != null) { - this.granularitySpec = new UniformGranularitySpec(this.segmentGranularity, this.intervals); - } - } - @JsonProperty public GranularitySpec getGranularitySpec() { @@ -302,15 +363,20 @@ public class HadoopDruidIndexerConfig public void setGranularitySpec(GranularitySpec granularitySpec) { - Preconditions.checkState(this.intervals == null, "Use setGranularitySpec instead of setIntervals"); - Preconditions.checkState( - this.segmentGranularity == null, - "Use setGranularitySpec instead of setSegmentGranularity" - ); - this.granularitySpec = granularitySpec; } + @JsonProperty + public PartitionsSpec getPartitionsSpec() + { + return partitionsSpec; + } + + public void setPartitionsSpec(PartitionsSpec partitionsSpec) + { + this.partitionsSpec = partitionsSpec; + } + @JsonProperty public PathSpec getPathSpec() { @@ -355,31 +421,19 @@ public class HadoopDruidIndexerConfig this.version = version; } - @JsonProperty public String getPartitionDimension() { - return partitionDimension; - } - - public void setPartitionDimension(String partitionDimension) - { - this.partitionDimension = (partitionDimension == null) ? partitionDimension : partitionDimension; + return partitionsSpec.getPartitionDimension(); } public boolean partitionByDimension() { - return targetPartitionSize != null; + return partitionsSpec.isDeterminingPartitions(); } - @JsonProperty public Long getTargetPartitionSize() { - return targetPartitionSize; - } - - public void setTargetPartitionSize(Long targetPartitionSize) - { - this.targetPartitionSize = targetPartitionSize; + return partitionsSpec.getTargetPartitionSize(); } public boolean isUpdaterJobSpecSet() diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java index f49762cc4c6..651cb757023 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java @@ -10,7 +10,7 @@ import org.joda.time.DateTime; import java.io.IOException; -public class HadoopDruidIndexerMapper extends Mapper +public abstract class HadoopDruidIndexerMapper extends Mapper { private HadoopDruidIndexerConfig config; private StringInputRowParser parser; @@ -61,9 +61,6 @@ public class HadoopDruidIndexerMapper extends Mapper 0; + } + + @JsonProperty + @Nullable + public String getPartitionDimension() + { + return partitionDimension; + } + + @JsonProperty + public long getTargetPartitionSize() + { + return targetPartitionSize; + } + + @JsonProperty + public boolean isAssumeGrouped() + { + return assumeGrouped; + } +} diff --git a/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java index 6bb56df31f5..f4db1148327 100644 --- a/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -22,6 +22,7 @@ package com.metamx.druid.indexer; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.metamx.druid.indexer.granularity.UniformGranularitySpec; +import com.metamx.druid.indexer.partitions.PartitionsSpec; import com.metamx.druid.jackson.DefaultObjectMapper; import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.Interval; @@ -67,7 +68,7 @@ public class HadoopDruidIndexerConfigTest } @Test - public void testIntervalsAndSegmentGranularity() { + public void testGranularitySpecLegacy() { // Deprecated and replaced by granularitySpec, but still supported final HadoopDruidIndexerConfig cfg; @@ -98,9 +99,8 @@ public class HadoopDruidIndexerConfigTest ); } - @Test - public void testCmdlineAndSegmentGranularity() { + public void testGranularitySpecPostConstructorIntervals() { // Deprecated and replaced by granularitySpec, but still supported final HadoopDruidIndexerConfig cfg; @@ -133,7 +133,7 @@ public class HadoopDruidIndexerConfigTest } @Test - public void testInvalidCombination() { + public void testInvalidGranularityCombination() { boolean thrown = false; try { final HadoopDruidIndexerConfig cfg = jsonMapper.readValue( @@ -154,4 +154,160 @@ public class HadoopDruidIndexerConfigTest Assert.assertTrue("Exception thrown", thrown); } + + @Test + public void testPartitionsSpecNoPartitioning() { + final HadoopDruidIndexerConfig cfg; + + try { + cfg = jsonMapper.readValue( + "{}", + HadoopDruidIndexerConfig.class + ); + } catch(Exception e) { + throw Throwables.propagate(e); + } + + final PartitionsSpec partitionsSpec = cfg.getPartitionsSpec(); + + Assert.assertEquals( + "isDeterminingPartitions", + partitionsSpec.isDeterminingPartitions(), + false + ); + } + + @Test + public void testPartitionsSpecAutoDimension() { + final HadoopDruidIndexerConfig cfg; + + try { + cfg = jsonMapper.readValue( + "{" + + "\"partitionsSpec\":{" + + " \"targetPartitionSize\":100" + + " }" + + "}", + HadoopDruidIndexerConfig.class + ); + } catch(Exception e) { + throw Throwables.propagate(e); + } + + final PartitionsSpec partitionsSpec = cfg.getPartitionsSpec(); + + Assert.assertEquals( + "isDeterminingPartitions", + partitionsSpec.isDeterminingPartitions(), + true + ); + + Assert.assertEquals( + "getTargetPartitionSize", + partitionsSpec.getTargetPartitionSize(), + 100 + ); + + Assert.assertEquals( + "getPartitionDimension", + partitionsSpec.getPartitionDimension(), + null + ); + } + + @Test + public void testPartitionsSpecSpecificDimension() { + final HadoopDruidIndexerConfig cfg; + + try { + cfg = jsonMapper.readValue( + "{" + + "\"partitionsSpec\":{" + + " \"targetPartitionSize\":100," + + " \"partitionDimension\":\"foo\"" + + " }" + + "}", + HadoopDruidIndexerConfig.class + ); + } catch(Exception e) { + throw Throwables.propagate(e); + } + + final PartitionsSpec partitionsSpec = cfg.getPartitionsSpec(); + + Assert.assertEquals( + "isDeterminingPartitions", + partitionsSpec.isDeterminingPartitions(), + true + ); + + Assert.assertEquals( + "getTargetPartitionSize", + partitionsSpec.getTargetPartitionSize(), + 100 + ); + + Assert.assertEquals( + "getPartitionDimension", + partitionsSpec.getPartitionDimension(), + "foo" + ); + } + + @Test + public void testPartitionsSpecLegacy() { + final HadoopDruidIndexerConfig cfg; + + try { + cfg = jsonMapper.readValue( + "{" + + "\"targetPartitionSize\":100," + + "\"partitionDimension\":\"foo\"" + + "}", + HadoopDruidIndexerConfig.class + ); + } catch(Exception e) { + throw Throwables.propagate(e); + } + + final PartitionsSpec partitionsSpec = cfg.getPartitionsSpec(); + + Assert.assertEquals( + "isDeterminingPartitions", + partitionsSpec.isDeterminingPartitions(), + true + ); + + Assert.assertEquals( + "getTargetPartitionSize", + partitionsSpec.getTargetPartitionSize(), + 100 + ); + + Assert.assertEquals( + "getPartitionDimension", + partitionsSpec.getPartitionDimension(), + "foo" + ); + } + + @Test + public void testInvalidPartitionsCombination() { + boolean thrown = false; + try { + final HadoopDruidIndexerConfig cfg = jsonMapper.readValue( + "{" + + "\"targetPartitionSize\":100," + + "\"partitionsSpec\":{" + + " \"targetPartitionSize\":100" + + " }" + + "}", + HadoopDruidIndexerConfig.class + ); + } catch(Exception e) { + thrown = true; + } + + Assert.assertTrue("Exception thrown", thrown); + } } From f05c050c53d4c7d61d7097d8f225c6028d79fd23 Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 21 Jan 2013 15:49:39 -0800 Subject: [PATCH 44/92] add test for timezone --- .../timeseries/TimeseriesQueryRunnerTest.java | 50 +++++++++++++++++++ 1 file changed, 50 insertions(+) diff --git a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java index b6479983634..550e918eb27 100644 --- a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -40,6 +40,7 @@ import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; import org.joda.time.Interval; import org.joda.time.Period; import org.junit.Assert; @@ -257,6 +258,55 @@ public class TimeseriesQueryRunnerTest TestHelper.assertExpectedResults(expectedResults, results); } + @Test + public void testTimeseriesWithTimeZone() + { + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory( + "idx", + "index" + ) + ) + ) + .granularity(new PeriodGranularity(new Period("P1D"), null, DateTimeZone.forID("America/Los_Angeles"))) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-03-31", DateTimeZone.forID("America/Los_Angeles")), + new TimeseriesResultValue( + ImmutableMap.of("rows", 13L, "idx", 6619L) + ) + ), + new Result( + new DateTime("2011-04-01T", DateTimeZone.forID("America/Los_Angeles")), + new TimeseriesResultValue( + ImmutableMap.of("rows", 13L, "idx", 5827L) + ) + ), + new Result( + new DateTime("2011-04-02", DateTimeZone.forID("America/Los_Angeles")), + new TimeseriesResultValue( + ImmutableMap.of("rows", 0L, "idx", 0L) + ) + ) + ); + + Iterable> results = Sequences.toList( + runner.run(query), + Lists.>newArrayList() + ); + + TestHelper.assertExpectedResults(expectedResults, results); + + + } + @Test public void testTimeseriesWithVaryingGran() From 8f38b775ae1aa7f6d044f7760e6618304b6f5788 Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 21 Jan 2013 16:31:32 -0800 Subject: [PATCH 45/92] fix expected object type --- .../query/timeseries/TimeseriesQueryQueryToolChest.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 0e5d96b385c..108d82e0266 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -49,6 +49,7 @@ import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Minutes; import org.joda.time.Period; +import org.joda.time.format.ISODateTimeFormat; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -216,7 +217,10 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest aggsIter = aggs.iterator(); Iterator resultIter = results.iterator(); - DateTime timestamp = (DateTime)resultIter.next(); + DateTime timestamp = ISODateTimeFormat.dateTimeParser() + .withOffsetParsed() + .parseDateTime(resultIter.next().toString()); + while (aggsIter.hasNext() && resultIter.hasNext()) { final AggregatorFactory factory = aggsIter.next(); retVal.put(factory.getName(), factory.deserialize(resultIter.next())); From d7ea8e9afcc3be41ef826da7743051248ecdf7de Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 21 Jan 2013 17:01:41 -0800 Subject: [PATCH 46/92] compare result timestamp based on millis + utcoffset --- client/src/main/java/com/metamx/druid/result/Result.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/src/main/java/com/metamx/druid/result/Result.java b/client/src/main/java/com/metamx/druid/result/Result.java index 9fc5aebf347..9b055cd4800 100644 --- a/client/src/main/java/com/metamx/druid/result/Result.java +++ b/client/src/main/java/com/metamx/druid/result/Result.java @@ -71,7 +71,7 @@ public class Result implements Comparable> Result result = (Result) o; - if (timestamp != null ? !timestamp.equals(result.timestamp) : result.timestamp != null) { + if (timestamp != null ? !(timestamp.isEqual(result.timestamp) && timestamp.getZone().getOffset(timestamp) == result.timestamp.getZone().getOffset(result.timestamp)) : result.timestamp != null) { return false; } if (value != null ? !value.equals(result.value) : result.value != null) { From 86a6d112e338ca7ac55d724c49b8010a51129274 Mon Sep 17 00:00:00 2001 From: xvrl Date: Tue, 22 Jan 2013 16:50:56 -0800 Subject: [PATCH 47/92] proper groupby tests --- .../java/com/metamx/druid/TestHelper.java | 50 ++++ .../query/group/GroupByQueryRunnerTest.java | 230 ++++++++++++++++++ 2 files changed, 280 insertions(+) create mode 100644 server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java diff --git a/server/src/test/java/com/metamx/druid/TestHelper.java b/server/src/test/java/com/metamx/druid/TestHelper.java index cf6c60a38a1..53d45192f7c 100644 --- a/server/src/test/java/com/metamx/druid/TestHelper.java +++ b/server/src/test/java/com/metamx/druid/TestHelper.java @@ -46,6 +46,16 @@ public class TestHelper assertResults(expectedResults, results, failMsg); } + public static void assertExpectedObjects(Iterable expectedResults, Iterable results, String failMsg) + { + assertObjects(expectedResults, results, failMsg); + } + + public static void assertExpectedObjects(Iterable expectedResults, Sequence results, String failMsg) + { + assertObjects(expectedResults, Sequences.toList(results, Lists.newArrayList()), failMsg); + } + private static void assertResults(Iterable> expectedResults, Iterable> actualResults, String failMsg) { Iterator resultsIter = actualResults.iterator(); @@ -86,6 +96,46 @@ public class TestHelper } } + private static void assertObjects(Iterable expectedResults, Iterable actualResults, String failMsg) + { + Iterator resultsIter = actualResults.iterator(); + Iterator resultsIter2 = actualResults.iterator(); + Iterator expectedResultsIter = expectedResults.iterator(); + + while (resultsIter.hasNext() && resultsIter2.hasNext() && expectedResultsIter.hasNext()) { + Object expectedNext = expectedResultsIter.next(); + final Object next = resultsIter.next(); + final Object next2 = resultsIter2.next(); + + Assert.assertEquals(failMsg, expectedNext, next); + Assert.assertEquals( + String.format("%sSecond iterator bad, multiple calls to iterator() should be safe", failMsg), + expectedNext, + next2 + ); + } + + if (resultsIter.hasNext()) { + Assert.fail( + String.format("%sExpected resultsIter to be exhausted, next element was %s", failMsg, resultsIter.next()) + ); + } + + if (resultsIter2.hasNext()) { + Assert.fail( + String.format("%sExpected resultsIter2 to be exhausted, next element was %s", failMsg, resultsIter.next()) + ); + } + + if (expectedResultsIter.hasNext()) { + Assert.fail( + String.format( + "%sExpected expectedResultsIter to be exhausted, next element was %s", failMsg, expectedResultsIter.next() + ) + ); + } + } + private static void assertResult(String msg, Result expected, Result actual) { Assert.assertEquals(msg, expected, actual); diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java new file mode 100644 index 00000000000..d2576e82162 --- /dev/null +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java @@ -0,0 +1,230 @@ +package com.metamx.druid.query.group; + +import com.google.common.base.Function; +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import com.metamx.druid.PeriodGranularity; +import com.metamx.druid.Query; +import com.metamx.druid.TestHelper; +import com.metamx.druid.aggregation.AggregatorFactory; +import com.metamx.druid.aggregation.LongSumAggregatorFactory; +import com.metamx.druid.collect.StupidPool; +import com.metamx.druid.input.MapBasedRow; +import com.metamx.druid.input.Row; +import com.metamx.druid.query.QueryRunner; +import com.metamx.druid.query.QueryRunnerTestHelper; +import com.metamx.druid.query.dimension.DefaultDimensionSpec; +import com.metamx.druid.query.dimension.DimensionSpec; +import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +@RunWith(Parameterized.class) +public class GroupByQueryRunnerTest +{ + private final QueryRunner runner; + private GroupByQueryRunnerFactory factory; + + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( + new GroupByQueryEngine( + new GroupByQueryEngineConfig() + { + @Override + public int getMaxIntermediateRows() + { + return 10000; + } + }, + new StupidPool( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(1024 * 1024); + } + } + ) + ) + ); + + + return Lists.newArrayList( + Iterables.transform( + QueryRunnerTestHelper.makeQueryRunners(factory), new Function() + { + @Override + public Object apply(@Nullable Object input) + { + return new Object[]{factory, ((Object[]) input)[0]}; + } + } + ) + ); + } + + public GroupByQueryRunnerTest(GroupByQueryRunnerFactory factory, QueryRunner runner) { + this.factory = factory; + this.runner = runner; + } + + @Test + public void testGroupBy() { + GroupByQuery query = GroupByQuery.builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions( + Lists.newArrayList( + (DimensionSpec)new DefaultDimensionSpec( + "quality", + "alias" + ) + ) + ) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory( + "idx", + "index" + ) + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "automotive", "rows", 1L, "idx", 135L)), + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "business", "rows", 1L, "idx", 118L)), + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "entertainment", "rows", 1L, "idx", 158L)), + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "health", "rows", 1L, "idx", 120L)), + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "mezzanine", "rows", 3L, "idx", 2870L)), + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "news", "rows", 1L, "idx", 121L)), + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "premium", "rows", 3L, "idx", 2900L)), + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "technology", "rows", 1L, "idx", 78L)), + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "travel", "rows", 1L, "idx", 119L)), + + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "automotive", "rows", 1L, "idx", 147L)), + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "business", "rows", 1L, "idx", 112L)), + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "entertainment", "rows", 1L, "idx", 166L)), + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "health", "rows", 1L, "idx", 113L)), + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "mezzanine", "rows", 3L, "idx", 2447L)), + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "news", "rows", 1L, "idx", 114L)), + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "premium", "rows", 3L, "idx", 2505L)), + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "technology", "rows", 1L, "idx", 97L)), + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "travel", "rows", 1L, "idx", 126L)) + ); + + Iterable results = Sequences.toList( + runner.run(query), + Lists.newArrayList() + ); + + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testMergeResults() { + GroupByQuery.Builder builder = GroupByQuery.builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval("2011-04-02/2011-04-04") + .setDimensions( + Lists.newArrayList( + (DimensionSpec)new DefaultDimensionSpec( + "quality", + "alias" + ) + ) + ) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory( + "idx", + "index" + ) + ) + ) + .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)); + + final GroupByQuery fullQuery = builder.build(); + + QueryRunner mergedRunner = new GroupByQueryQueryToolChest().mergeResults( + new QueryRunner() + { + @Override + public Sequence run(Query query) + { + // simulate two daily segments + final Query query1 = query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03")))); + final Query query2 = query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))); + return Sequences.concat(runner.run(query1), runner.run(query2)); + } + } + ); + + List expectedResults = Arrays.asList( + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "automotive", "rows", 1L, "idx", 269L) + ), + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "business", "rows", 1L, "idx", 217L) + ), + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "entertainment", "rows", 1L, "idx", 319L) + ), + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "health", "rows", 1L, "idx", 216L) + ), + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "mezzanine", "rows", 3L, "idx", 4420L) + ), + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "news", "rows", 1L, "idx", 221L) + ), + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "premium", "rows", 3L, "idx", 4416L) + ), + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "technology", "rows", 1L, "idx", 177L) + ), + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "travel", "rows", 1L, "idx", 243L) + ) + ); + + Iterable results = Sequences.toList( + mergedRunner.run(fullQuery), + Lists.newArrayList() + ); + + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } +} From 35058786d9eb96a6a443c9182e4845c1bc2b08bd Mon Sep 17 00:00:00 2001 From: xvrl Date: Tue, 22 Jan 2013 19:31:31 -0800 Subject: [PATCH 48/92] match query interval to granularity for this test --- .../query/timeseries/TimeseriesQueryRunnerTest.java | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java index 550e918eb27..3be91c3686f 100644 --- a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -263,7 +263,7 @@ public class TimeseriesQueryRunnerTest { TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .intervals(QueryRunnerTestHelper.firstToThird) + .intervals("2011-03-31T00:00:00-07:00/2011-04-02T00:00:00-07:00") .aggregators( Arrays.asList( QueryRunnerTestHelper.rowsCount, @@ -288,12 +288,6 @@ public class TimeseriesQueryRunnerTest new TimeseriesResultValue( ImmutableMap.of("rows", 13L, "idx", 5827L) ) - ), - new Result( - new DateTime("2011-04-02", DateTimeZone.forID("America/Los_Angeles")), - new TimeseriesResultValue( - ImmutableMap.of("rows", 0L, "idx", 0L) - ) ) ); @@ -303,11 +297,8 @@ public class TimeseriesQueryRunnerTest ); TestHelper.assertExpectedResults(expectedResults, results); - - } - @Test public void testTimeseriesWithVaryingGran() { From 55ae4c87dde33e4dc5127c41da945c63987e1d8c Mon Sep 17 00:00:00 2001 From: xvrl Date: Tue, 22 Jan 2013 10:59:05 -0800 Subject: [PATCH 49/92] timezone support in groupby query --- .../group/GroupByQueryQueryToolChest.java | 15 +++++- .../com/metamx/druid/input/MapBasedRow.java | 46 +++++++++++++++---- .../druid/query/group/GroupByQueryEngine.java | 3 +- .../GroupByTimeseriesQueryRunnerTest.java | 2 +- 4 files changed, 54 insertions(+), 12 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index abe5610732d..913ef5e57ce 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -32,6 +32,7 @@ import com.metamx.druid.Query; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; import com.metamx.druid.input.Rows; import com.metamx.druid.query.CacheStrategy; @@ -119,7 +120,19 @@ public class GroupByQueryQueryToolChest implements QueryToolChest() + { + @Override + public Row apply(@Nullable Row input) + { + final MapBasedRow row = (MapBasedRow) input; + return new MapBasedRow(query.getGranularity().toDateTime(row.getTimestampFromEpoch()), row.getEvent()); + } + } + ); } }; } diff --git a/common/src/main/java/com/metamx/druid/input/MapBasedRow.java b/common/src/main/java/com/metamx/druid/input/MapBasedRow.java index 03c56e2f66d..d823762eddd 100644 --- a/common/src/main/java/com/metamx/druid/input/MapBasedRow.java +++ b/common/src/main/java/com/metamx/druid/input/MapBasedRow.java @@ -37,7 +37,7 @@ import java.util.Map; */ public class MapBasedRow implements Row { - private final long timestamp; + private final DateTime timestamp; private final Map event; @JsonCreator @@ -46,22 +46,21 @@ public class MapBasedRow implements Row @JsonProperty("event") Map event ) { - this(timestamp.getMillis(), event); + this.timestamp = timestamp; + this.event = event; } public MapBasedRow( long timestamp, Map event - ) - { - this.timestamp = timestamp; - this.event = event; + ) { + this(new DateTime(timestamp), event); } @Override public long getTimestampFromEpoch() { - return timestamp; + return timestamp.getMillis(); } @Override @@ -120,7 +119,7 @@ public class MapBasedRow implements Row @JsonProperty public DateTime getTimestamp() { - return new DateTime(timestamp); + return timestamp; } @JsonProperty @@ -133,9 +132,38 @@ public class MapBasedRow implements Row public String toString() { return "MapBasedRow{" + - "timestamp=" + new DateTime(timestamp) + + "timestamp=" + timestamp + ", event=" + event + '}'; } + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + MapBasedRow that = (MapBasedRow) o; + + if (!event.equals(that.event)) { + return false; + } + if (!timestamp.equals(that.timestamp)) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = timestamp.hashCode(); + result = 31 * result + event.hashCode(); + return result; + } } diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java index 788e1adb02a..cb9a1468b80 100644 --- a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java @@ -43,6 +43,7 @@ import com.metamx.druid.index.v1.processing.DimensionSelector; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; import com.metamx.druid.query.dimension.DimensionSpec; +import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -347,7 +348,7 @@ public class GroupByQueryEngine .transform( new Function, Row>() { - private final long timestamp = cursor.getTime().getMillis(); + private final DateTime timestamp = cursor.getTime(); private final int[] increments = positionMaintainer.getIncrements(); @Override diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java index 0860715963b..8a1c7637367 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java @@ -112,7 +112,7 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest MapBasedRow row = (MapBasedRow) input; return new Result( - new DateTime(input.getTimestampFromEpoch()), new TimeseriesResultValue(row.getEvent()) + row.getTimestamp(), new TimeseriesResultValue(row.getEvent()) ); } } From 67feee3cd621cd12ac8257b71a5867a940f0d244 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Wed, 23 Jan 2013 18:36:52 -0600 Subject: [PATCH 50/92] 1) Indexes don't always have an index.drd file anymore --- .../druid/loading/MMappedQueryableIndexFactory.java | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java index 9e71e3aeb04..648813d62ac 100644 --- a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java @@ -36,11 +36,6 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory @Override public QueryableIndex factorize(File parentDir) throws StorageAdapterLoadingException { - File indexFile = new File(parentDir, "index.drd"); - if (!indexFile.exists()) { - throw new StorageAdapterLoadingException("indexFile[%s] does not exist.", indexFile); - } - try { if (! IndexIO.canBeMapped(parentDir)) { File canBeMappedDir = new File(parentDir, "forTheMapping"); @@ -55,7 +50,7 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory } for (File file : canBeMappedDir.listFiles()) { if (! file.renameTo(new File(parentDir, file.getName()))) { - throw new StorageAdapterLoadingException("Couldn't rename[%s] to [%s]", canBeMappedDir, indexFile); + throw new StorageAdapterLoadingException("Couldn't rename[%s] to [%s]", canBeMappedDir, parentDir); } } FileUtils.deleteDirectory(canBeMappedDir); @@ -64,7 +59,7 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory return IndexIO.loadIndex(parentDir); } catch (IOException e) { - log.warn(e, "Got exception, deleting index[%s]", indexFile); + log.warn(e, "Got exception, deleting parentDir[%s]", parentDir); try { FileUtils.deleteDirectory(parentDir); } From ee7337fbb9a88e297f7f585f4632089be6f7da85 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 24 Jan 2013 18:25:21 -0600 Subject: [PATCH 51/92] 1) Adjust the Timeseries caching fixes to still store the long, but do the timezone adjustment on the way out. 2) Store a reference to the granularity object instead of getting it every time --- .../druid/query/group/GroupByQueryQueryToolChest.java | 7 +++++-- .../timeseries/TimeseriesQueryQueryToolChest.java | 10 +++++----- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index 913ef5e57ce..42f1940218c 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -29,6 +29,7 @@ import com.metamx.common.guava.ConcatSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.Query; +import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.initialization.Initialization; @@ -125,11 +126,13 @@ public class GroupByQueryQueryToolChest implements QueryToolChest() { + private final QueryGranularity granularity = query.getGranularity(); + @Override - public Row apply(@Nullable Row input) + public Row apply(Row input) { final MapBasedRow row = (MapBasedRow) input; - return new MapBasedRow(query.getGranularity().toDateTime(row.getTimestampFromEpoch()), row.getEvent()); + return new MapBasedRow(granularity.toDateTime(row.getTimestampFromEpoch()), row.getEvent()); } } ); diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 108d82e0266..df619f340f0 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -28,6 +28,7 @@ import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.nary.BinaryFn; import com.metamx.druid.Query; +import com.metamx.druid.QueryGranularity; import com.metamx.druid.ResultGranularTimestampComparator; import com.metamx.druid.TimeseriesBinaryFn; import com.metamx.druid.aggregation.AggregatorFactory; @@ -192,8 +193,7 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest retVal = Lists.newArrayListWithCapacity(1 + aggs.size()); - // make sure to preserve timezone information when caching results - retVal.add(input.getTimestamp()); + retVal.add(input.getTimestamp().getMillis()); for (AggregatorFactory agg : aggs) { retVal.add(results.getMetric(agg.getName())); } @@ -208,6 +208,8 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest>() { + private final QueryGranularity granularity = query.getGranularity(); + @Override public Result apply(@Nullable Object input) { @@ -217,9 +219,7 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest aggsIter = aggs.iterator(); Iterator resultIter = results.iterator(); - DateTime timestamp = ISODateTimeFormat.dateTimeParser() - .withOffsetParsed() - .parseDateTime(resultIter.next().toString()); + DateTime timestamp = granularity.toDateTime(((Number) resultIter.next()).longValue()); while (aggsIter.hasNext() && resultIter.hasNext()) { final AggregatorFactory factory = aggsIter.next(); From 5e1cac6d9f7db3d1c22f18f98110d9f2c10acaf3 Mon Sep 17 00:00:00 2001 From: xvrl Date: Thu, 24 Jan 2013 17:57:10 -0800 Subject: [PATCH 52/92] timezone test --- .../query/group/GroupByQueryRunnerTest.java | 65 +++++++++++++++++++ 1 file changed, 65 insertions(+) diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java index d2576e82162..f0168729c22 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java @@ -21,6 +21,7 @@ import com.metamx.druid.query.dimension.DefaultDimensionSpec; import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; import org.joda.time.Interval; import org.joda.time.Period; import org.junit.Test; @@ -141,6 +142,70 @@ public class GroupByQueryRunnerTest TestHelper.assertExpectedObjects(expectedResults, results, ""); } + @Test + public void testGroupByWithTimeZone() { + DateTimeZone tz = DateTimeZone.forID("America/Los_Angeles"); + + GroupByQuery query = GroupByQuery.builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval("2011-03-31T00:00:00-07:00/2011-04-02T00:00:00-07:00") + .setDimensions( + Lists.newArrayList( + (DimensionSpec) new DefaultDimensionSpec( + "quality", + "alias" + ) + ) + ) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory( + "idx", + "index" + ) + ) + ) + .setGranularity( + new PeriodGranularity( + new Period("P1D"), + null, + tz + ) + ) + .build(); + + List expectedResults = Arrays.asList( + (Row) new MapBasedRow(new DateTime("2011-03-31", tz),ImmutableMap.of("alias", "automotive", "rows", 1L, "idx", 135L)), + (Row) new MapBasedRow(new DateTime("2011-03-31", tz),ImmutableMap.of("alias", "business", "rows", 1L, "idx", 118L)), + (Row) new MapBasedRow(new DateTime("2011-03-31", tz),ImmutableMap.of("alias", "entertainment", "rows", 1L, "idx", 158L)), + (Row) new MapBasedRow(new DateTime("2011-03-31", tz),ImmutableMap.of("alias", "health", "rows", 1L, "idx", 120L)), + (Row) new MapBasedRow(new DateTime("2011-03-31", tz),ImmutableMap.of("alias", "mezzanine", "rows", 3L, "idx", 2870L)), + (Row) new MapBasedRow(new DateTime("2011-03-31", tz),ImmutableMap.of("alias", "news", "rows", 1L, "idx", 121L)), + (Row) new MapBasedRow(new DateTime("2011-03-31", tz),ImmutableMap.of("alias", "premium", "rows", 3L, "idx", 2900L)), + (Row) new MapBasedRow(new DateTime("2011-03-31", tz),ImmutableMap.of("alias", "technology", "rows", 1L, "idx", 78L)), + (Row) new MapBasedRow(new DateTime("2011-03-31", tz),ImmutableMap.of("alias", "travel", "rows", 1L, "idx", 119L)), + + (Row) new MapBasedRow(new DateTime("2011-04-01", tz),ImmutableMap.of("alias", "automotive", "rows", 1L, "idx", 147L)), + (Row) new MapBasedRow(new DateTime("2011-04-01", tz),ImmutableMap.of("alias", "business", "rows", 1L, "idx", 112L)), + (Row) new MapBasedRow(new DateTime("2011-04-01", tz),ImmutableMap.of("alias", "entertainment", "rows", 1L, "idx", 166L)), + (Row) new MapBasedRow(new DateTime("2011-04-01", tz),ImmutableMap.of("alias", "health", "rows", 1L, "idx", 113L)), + (Row) new MapBasedRow(new DateTime("2011-04-01", tz),ImmutableMap.of("alias", "mezzanine", "rows", 3L, "idx", 2447L)), + (Row) new MapBasedRow(new DateTime("2011-04-01", tz),ImmutableMap.of("alias", "news", "rows", 1L, "idx", 114L)), + (Row) new MapBasedRow(new DateTime("2011-04-01", tz),ImmutableMap.of("alias", "premium", "rows", 3L, "idx", 2505L)), + (Row) new MapBasedRow(new DateTime("2011-04-01", tz),ImmutableMap.of("alias", "technology", "rows", 1L, "idx", 97L)), + (Row) new MapBasedRow(new DateTime("2011-04-01", tz),ImmutableMap.of("alias", "travel", "rows", 1L, "idx", 126L)) + ); + + Iterable results = Sequences.toList( + runner.run(query), + Lists.newArrayList() + ); + + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test public void testMergeResults() { GroupByQuery.Builder builder = GroupByQuery.builder() From 37417cf22f9b4ae0d4797890b3e08ddca497b271 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 25 Jan 2013 11:03:44 -0800 Subject: [PATCH 53/92] QueryServlet: Add "host" (remote address) to alerts --- client/src/main/java/com/metamx/druid/http/QueryServlet.java | 1 + 1 file changed, 1 insertion(+) diff --git a/client/src/main/java/com/metamx/druid/http/QueryServlet.java b/client/src/main/java/com/metamx/druid/http/QueryServlet.java index f68efbc1e89..108863bf7af 100644 --- a/client/src/main/java/com/metamx/druid/http/QueryServlet.java +++ b/client/src/main/java/com/metamx/druid/http/QueryServlet.java @@ -148,6 +148,7 @@ public class QueryServlet extends HttpServlet ImmutableMap.builder() .put("exception", e.toString()) .put("query", queryString) + .put("host", req.getRemoteAddr()) .build() ) ); From 4a1f5315a6e493b6b2d6db3cba0af59f3188b936 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Fri, 25 Jan 2013 18:15:51 -0600 Subject: [PATCH 54/92] 1) Make tests pass --- .../group/GroupByQueryQueryToolChest.java | 6 +- .../com/metamx/druid/input/MapBasedRow.java | 29 +++ .../java/com/metamx/druid/input/Rows.java | 6 + .../query/group/GroupByQueryRunnerTest.java | 209 ++++++++---------- 4 files changed, 136 insertions(+), 114 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index abe5610732d..cda91c5f285 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -29,6 +29,7 @@ import com.metamx.common.guava.ConcatSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.Query; +import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.initialization.Initialization; @@ -99,10 +100,11 @@ public class GroupByQueryQueryToolChest implements QueryToolChest() diff --git a/common/src/main/java/com/metamx/druid/input/MapBasedRow.java b/common/src/main/java/com/metamx/druid/input/MapBasedRow.java index 03c56e2f66d..2c006b65874 100644 --- a/common/src/main/java/com/metamx/druid/input/MapBasedRow.java +++ b/common/src/main/java/com/metamx/druid/input/MapBasedRow.java @@ -138,4 +138,33 @@ public class MapBasedRow implements Row '}'; } + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + MapBasedRow that = (MapBasedRow) o; + + if (timestamp != that.timestamp) { + return false; + } + if (event != null ? !event.equals(that.event) : that.event != null) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = (int) (timestamp ^ (timestamp >>> 32)); + result = 31 * result + (event != null ? event.hashCode() : 0); + return result; + } } diff --git a/common/src/main/java/com/metamx/druid/input/Rows.java b/common/src/main/java/com/metamx/druid/input/Rows.java index b77ab749761..3e70b4c26d5 100644 --- a/common/src/main/java/com/metamx/druid/input/Rows.java +++ b/common/src/main/java/com/metamx/druid/input/Rows.java @@ -52,6 +52,12 @@ public class Rows { return row.getFloatMetric(metric); } + + @Override + public String toString() + { + return row.toString(); + } }; } } diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java index d2576e82162..7f630d32b96 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java @@ -1,10 +1,30 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.query.group; import com.google.common.base.Function; +import com.google.common.base.Preconditions; import com.google.common.base.Supplier; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.PeriodGranularity; @@ -33,6 +53,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collection; import java.util.List; +import java.util.Map; @RunWith(Parameterized.class) public class GroupByQueryRunnerTest @@ -88,82 +109,61 @@ public class GroupByQueryRunnerTest @Test public void testGroupBy() { - GroupByQuery query = GroupByQuery.builder() - .setDataSource(QueryRunnerTestHelper.dataSource) - .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions( - Lists.newArrayList( - (DimensionSpec)new DefaultDimensionSpec( - "quality", - "alias" - ) - ) - ) - .setAggregatorSpecs( - Arrays.asList( - QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory( - "idx", - "index" - ) - ) - ) - .setGranularity(QueryRunnerTestHelper.dayGran) - .build(); + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); - List expectedResults = Arrays.asList( - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "automotive", "rows", 1L, "idx", 135L)), - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "business", "rows", 1L, "idx", 118L)), - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "entertainment", "rows", 1L, "idx", 158L)), - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "health", "rows", 1L, "idx", 120L)), - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "mezzanine", "rows", 3L, "idx", 2870L)), - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "news", "rows", 1L, "idx", 121L)), - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "premium", "rows", 3L, "idx", 2900L)), - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "technology", "rows", 1L, "idx", 78L)), - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "travel", "rows", 1L, "idx", 119L)), + List expectedResults = Arrays.asList( + createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "automotive", "rows", 1L, "idx", 147L)), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "business", "rows", 1L, "idx", 112L)), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "entertainment", "rows", 1L, "idx", 166L)), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "health", "rows", 1L, "idx", 113L)), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "mezzanine", "rows", 3L, "idx", 2447L)), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "news", "rows", 1L, "idx", 114L)), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "premium", "rows", 3L, "idx", 2505L)), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "technology", "rows", 1L, "idx", 97L)), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "travel", "rows", 1L, "idx", 126L)) + createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); - Iterable results = Sequences.toList( - runner.run(query), - Lists.newArrayList() - ); + Iterable results = Sequences.toList(runner.run(query), Lists.newArrayList()); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @Test public void testMergeResults() { - GroupByQuery.Builder builder = GroupByQuery.builder() - .setDataSource(QueryRunnerTestHelper.dataSource) - .setInterval("2011-04-02/2011-04-04") - .setDimensions( - Lists.newArrayList( - (DimensionSpec)new DefaultDimensionSpec( - "quality", - "alias" - ) - ) - ) - .setAggregatorSpecs( - Arrays.asList( - QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory( - "idx", - "index" - ) - ) - ) - .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)); + GroupByQuery.Builder builder = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval("2011-04-02/2011-04-04") + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)); final GroupByQuery fullQuery = builder.build(); @@ -174,57 +174,42 @@ public class GroupByQueryRunnerTest public Sequence run(Query query) { // simulate two daily segments - final Query query1 = query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03")))); - final Query query2 = query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))); - return Sequences.concat(runner.run(query1), runner.run(query2)); + final Query query1 = query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) + ); + final Query query2 = query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) + ); + return Sequences.concat(runner.run(query1), runner.run(query2)); } } ); - List expectedResults = Arrays.asList( - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "automotive", "rows", 1L, "idx", 269L) - ), - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "business", "rows", 1L, "idx", 217L) - ), - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "entertainment", "rows", 1L, "idx", 319L) - ), - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "health", "rows", 1L, "idx", 216L) - ), - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "mezzanine", "rows", 3L, "idx", 4420L) - ), - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "news", "rows", 1L, "idx", 221L) - ), - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "premium", "rows", 3L, "idx", 4416L) - ), - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "technology", "rows", 1L, "idx", 177L) - ), - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "travel", "rows", 1L, "idx", 243L) - ) + List expectedResults = Arrays.asList( + createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), + createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) ); - Iterable results = Sequences.toList( - mergedRunner.run(fullQuery), - Lists.newArrayList() - ); + TestHelper.assertExpectedObjects(expectedResults, runner.run(fullQuery), "direct"); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery), "merged"); + } - TestHelper.assertExpectedObjects(expectedResults, results, ""); + private MapBasedRow createExpectedRow(final String timestamp, Object... vals) + { + Preconditions.checkArgument(vals.length % 2 == 0); + + Map theVals = Maps.newHashMap(); + for (int i = 0; i < vals.length; i+=2) { + theVals.put(vals[i].toString(), vals[i+1]); + } + + return new MapBasedRow(new DateTime(timestamp), theVals); } } From c0822325e7997ca33ce38990f0785ff292a52241 Mon Sep 17 00:00:00 2001 From: xvrl Date: Tue, 22 Jan 2013 16:50:56 -0800 Subject: [PATCH 55/92] proper groupby tests --- .../java/com/metamx/druid/TestHelper.java | 50 ++++ .../query/group/GroupByQueryRunnerTest.java | 230 ++++++++++++++++++ 2 files changed, 280 insertions(+) create mode 100644 server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java diff --git a/server/src/test/java/com/metamx/druid/TestHelper.java b/server/src/test/java/com/metamx/druid/TestHelper.java index cf6c60a38a1..53d45192f7c 100644 --- a/server/src/test/java/com/metamx/druid/TestHelper.java +++ b/server/src/test/java/com/metamx/druid/TestHelper.java @@ -46,6 +46,16 @@ public class TestHelper assertResults(expectedResults, results, failMsg); } + public static void assertExpectedObjects(Iterable expectedResults, Iterable results, String failMsg) + { + assertObjects(expectedResults, results, failMsg); + } + + public static void assertExpectedObjects(Iterable expectedResults, Sequence results, String failMsg) + { + assertObjects(expectedResults, Sequences.toList(results, Lists.newArrayList()), failMsg); + } + private static void assertResults(Iterable> expectedResults, Iterable> actualResults, String failMsg) { Iterator resultsIter = actualResults.iterator(); @@ -86,6 +96,46 @@ public class TestHelper } } + private static void assertObjects(Iterable expectedResults, Iterable actualResults, String failMsg) + { + Iterator resultsIter = actualResults.iterator(); + Iterator resultsIter2 = actualResults.iterator(); + Iterator expectedResultsIter = expectedResults.iterator(); + + while (resultsIter.hasNext() && resultsIter2.hasNext() && expectedResultsIter.hasNext()) { + Object expectedNext = expectedResultsIter.next(); + final Object next = resultsIter.next(); + final Object next2 = resultsIter2.next(); + + Assert.assertEquals(failMsg, expectedNext, next); + Assert.assertEquals( + String.format("%sSecond iterator bad, multiple calls to iterator() should be safe", failMsg), + expectedNext, + next2 + ); + } + + if (resultsIter.hasNext()) { + Assert.fail( + String.format("%sExpected resultsIter to be exhausted, next element was %s", failMsg, resultsIter.next()) + ); + } + + if (resultsIter2.hasNext()) { + Assert.fail( + String.format("%sExpected resultsIter2 to be exhausted, next element was %s", failMsg, resultsIter.next()) + ); + } + + if (expectedResultsIter.hasNext()) { + Assert.fail( + String.format( + "%sExpected expectedResultsIter to be exhausted, next element was %s", failMsg, expectedResultsIter.next() + ) + ); + } + } + private static void assertResult(String msg, Result expected, Result actual) { Assert.assertEquals(msg, expected, actual); diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java new file mode 100644 index 00000000000..d2576e82162 --- /dev/null +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java @@ -0,0 +1,230 @@ +package com.metamx.druid.query.group; + +import com.google.common.base.Function; +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import com.metamx.druid.PeriodGranularity; +import com.metamx.druid.Query; +import com.metamx.druid.TestHelper; +import com.metamx.druid.aggregation.AggregatorFactory; +import com.metamx.druid.aggregation.LongSumAggregatorFactory; +import com.metamx.druid.collect.StupidPool; +import com.metamx.druid.input.MapBasedRow; +import com.metamx.druid.input.Row; +import com.metamx.druid.query.QueryRunner; +import com.metamx.druid.query.QueryRunnerTestHelper; +import com.metamx.druid.query.dimension.DefaultDimensionSpec; +import com.metamx.druid.query.dimension.DimensionSpec; +import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +@RunWith(Parameterized.class) +public class GroupByQueryRunnerTest +{ + private final QueryRunner runner; + private GroupByQueryRunnerFactory factory; + + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( + new GroupByQueryEngine( + new GroupByQueryEngineConfig() + { + @Override + public int getMaxIntermediateRows() + { + return 10000; + } + }, + new StupidPool( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(1024 * 1024); + } + } + ) + ) + ); + + + return Lists.newArrayList( + Iterables.transform( + QueryRunnerTestHelper.makeQueryRunners(factory), new Function() + { + @Override + public Object apply(@Nullable Object input) + { + return new Object[]{factory, ((Object[]) input)[0]}; + } + } + ) + ); + } + + public GroupByQueryRunnerTest(GroupByQueryRunnerFactory factory, QueryRunner runner) { + this.factory = factory; + this.runner = runner; + } + + @Test + public void testGroupBy() { + GroupByQuery query = GroupByQuery.builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions( + Lists.newArrayList( + (DimensionSpec)new DefaultDimensionSpec( + "quality", + "alias" + ) + ) + ) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory( + "idx", + "index" + ) + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "automotive", "rows", 1L, "idx", 135L)), + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "business", "rows", 1L, "idx", 118L)), + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "entertainment", "rows", 1L, "idx", 158L)), + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "health", "rows", 1L, "idx", 120L)), + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "mezzanine", "rows", 3L, "idx", 2870L)), + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "news", "rows", 1L, "idx", 121L)), + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "premium", "rows", 3L, "idx", 2900L)), + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "technology", "rows", 1L, "idx", 78L)), + (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "travel", "rows", 1L, "idx", 119L)), + + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "automotive", "rows", 1L, "idx", 147L)), + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "business", "rows", 1L, "idx", 112L)), + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "entertainment", "rows", 1L, "idx", 166L)), + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "health", "rows", 1L, "idx", 113L)), + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "mezzanine", "rows", 3L, "idx", 2447L)), + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "news", "rows", 1L, "idx", 114L)), + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "premium", "rows", 3L, "idx", 2505L)), + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "technology", "rows", 1L, "idx", 97L)), + (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "travel", "rows", 1L, "idx", 126L)) + ); + + Iterable results = Sequences.toList( + runner.run(query), + Lists.newArrayList() + ); + + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testMergeResults() { + GroupByQuery.Builder builder = GroupByQuery.builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval("2011-04-02/2011-04-04") + .setDimensions( + Lists.newArrayList( + (DimensionSpec)new DefaultDimensionSpec( + "quality", + "alias" + ) + ) + ) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory( + "idx", + "index" + ) + ) + ) + .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)); + + final GroupByQuery fullQuery = builder.build(); + + QueryRunner mergedRunner = new GroupByQueryQueryToolChest().mergeResults( + new QueryRunner() + { + @Override + public Sequence run(Query query) + { + // simulate two daily segments + final Query query1 = query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03")))); + final Query query2 = query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))); + return Sequences.concat(runner.run(query1), runner.run(query2)); + } + } + ); + + List expectedResults = Arrays.asList( + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "automotive", "rows", 1L, "idx", 269L) + ), + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "business", "rows", 1L, "idx", 217L) + ), + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "entertainment", "rows", 1L, "idx", 319L) + ), + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "health", "rows", 1L, "idx", 216L) + ), + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "mezzanine", "rows", 3L, "idx", 4420L) + ), + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "news", "rows", 1L, "idx", 221L) + ), + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "premium", "rows", 3L, "idx", 4416L) + ), + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "technology", "rows", 1L, "idx", 177L) + ), + (Row) new MapBasedRow( + new DateTime("2011-04-01"), + ImmutableMap.of("alias", "travel", "rows", 1L, "idx", 243L) + ) + ); + + Iterable results = Sequences.toList( + mergedRunner.run(fullQuery), + Lists.newArrayList() + ); + + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } +} From 7439a2e820e731efe213f6c7e958bb90e9cb292a Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Fri, 25 Jan 2013 18:15:51 -0600 Subject: [PATCH 56/92] 1) Make tests pass --- .../group/GroupByQueryQueryToolChest.java | 5 +- .../java/com/metamx/druid/input/Rows.java | 6 + .../query/group/GroupByQueryRunnerTest.java | 209 ++++++++---------- 3 files changed, 106 insertions(+), 114 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index 42f1940218c..6ce071f8068 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -101,10 +101,11 @@ public class GroupByQueryQueryToolChest implements QueryToolChest() diff --git a/common/src/main/java/com/metamx/druid/input/Rows.java b/common/src/main/java/com/metamx/druid/input/Rows.java index b77ab749761..3e70b4c26d5 100644 --- a/common/src/main/java/com/metamx/druid/input/Rows.java +++ b/common/src/main/java/com/metamx/druid/input/Rows.java @@ -52,6 +52,12 @@ public class Rows { return row.getFloatMetric(metric); } + + @Override + public String toString() + { + return row.toString(); + } }; } } diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java index d2576e82162..7f630d32b96 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java @@ -1,10 +1,30 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.query.group; import com.google.common.base.Function; +import com.google.common.base.Preconditions; import com.google.common.base.Supplier; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.PeriodGranularity; @@ -33,6 +53,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collection; import java.util.List; +import java.util.Map; @RunWith(Parameterized.class) public class GroupByQueryRunnerTest @@ -88,82 +109,61 @@ public class GroupByQueryRunnerTest @Test public void testGroupBy() { - GroupByQuery query = GroupByQuery.builder() - .setDataSource(QueryRunnerTestHelper.dataSource) - .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) - .setDimensions( - Lists.newArrayList( - (DimensionSpec)new DefaultDimensionSpec( - "quality", - "alias" - ) - ) - ) - .setAggregatorSpecs( - Arrays.asList( - QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory( - "idx", - "index" - ) - ) - ) - .setGranularity(QueryRunnerTestHelper.dayGran) - .build(); + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); - List expectedResults = Arrays.asList( - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "automotive", "rows", 1L, "idx", 135L)), - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "business", "rows", 1L, "idx", 118L)), - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "entertainment", "rows", 1L, "idx", 158L)), - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "health", "rows", 1L, "idx", 120L)), - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "mezzanine", "rows", 3L, "idx", 2870L)), - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "news", "rows", 1L, "idx", 121L)), - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "premium", "rows", 3L, "idx", 2900L)), - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "technology", "rows", 1L, "idx", 78L)), - (Row) new MapBasedRow(new DateTime("2011-04-01"),ImmutableMap.of("alias", "travel", "rows", 1L, "idx", 119L)), + List expectedResults = Arrays.asList( + createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "automotive", "rows", 1L, "idx", 147L)), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "business", "rows", 1L, "idx", 112L)), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "entertainment", "rows", 1L, "idx", 166L)), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "health", "rows", 1L, "idx", 113L)), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "mezzanine", "rows", 3L, "idx", 2447L)), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "news", "rows", 1L, "idx", 114L)), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "premium", "rows", 3L, "idx", 2505L)), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "technology", "rows", 1L, "idx", 97L)), - (Row) new MapBasedRow(new DateTime("2011-04-02"),ImmutableMap.of("alias", "travel", "rows", 1L, "idx", 126L)) + createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); - Iterable results = Sequences.toList( - runner.run(query), - Lists.newArrayList() - ); + Iterable results = Sequences.toList(runner.run(query), Lists.newArrayList()); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @Test public void testMergeResults() { - GroupByQuery.Builder builder = GroupByQuery.builder() - .setDataSource(QueryRunnerTestHelper.dataSource) - .setInterval("2011-04-02/2011-04-04") - .setDimensions( - Lists.newArrayList( - (DimensionSpec)new DefaultDimensionSpec( - "quality", - "alias" - ) - ) - ) - .setAggregatorSpecs( - Arrays.asList( - QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory( - "idx", - "index" - ) - ) - ) - .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)); + GroupByQuery.Builder builder = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval("2011-04-02/2011-04-04") + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)); final GroupByQuery fullQuery = builder.build(); @@ -174,57 +174,42 @@ public class GroupByQueryRunnerTest public Sequence run(Query query) { // simulate two daily segments - final Query query1 = query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03")))); - final Query query2 = query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))); - return Sequences.concat(runner.run(query1), runner.run(query2)); + final Query query1 = query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) + ); + final Query query2 = query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) + ); + return Sequences.concat(runner.run(query1), runner.run(query2)); } } ); - List expectedResults = Arrays.asList( - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "automotive", "rows", 1L, "idx", 269L) - ), - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "business", "rows", 1L, "idx", 217L) - ), - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "entertainment", "rows", 1L, "idx", 319L) - ), - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "health", "rows", 1L, "idx", 216L) - ), - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "mezzanine", "rows", 3L, "idx", 4420L) - ), - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "news", "rows", 1L, "idx", 221L) - ), - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "premium", "rows", 3L, "idx", 4416L) - ), - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "technology", "rows", 1L, "idx", 177L) - ), - (Row) new MapBasedRow( - new DateTime("2011-04-01"), - ImmutableMap.of("alias", "travel", "rows", 1L, "idx", 243L) - ) + List expectedResults = Arrays.asList( + createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), + createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) ); - Iterable results = Sequences.toList( - mergedRunner.run(fullQuery), - Lists.newArrayList() - ); + TestHelper.assertExpectedObjects(expectedResults, runner.run(fullQuery), "direct"); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery), "merged"); + } - TestHelper.assertExpectedObjects(expectedResults, results, ""); + private MapBasedRow createExpectedRow(final String timestamp, Object... vals) + { + Preconditions.checkArgument(vals.length % 2 == 0); + + Map theVals = Maps.newHashMap(); + for (int i = 0; i < vals.length; i+=2) { + theVals.put(vals[i].toString(), vals[i+1]); + } + + return new MapBasedRow(new DateTime(timestamp), theVals); } } From f2cce2829732d40ee786d641ccba93d43a1e154e Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Tue, 29 Jan 2013 11:29:26 -0600 Subject: [PATCH 57/92] 1) Make SegmentMetadataQuery work --- .../main/java/com/metamx/druid/BaseQuery.java | 8 +- .../src/main/java/com/metamx/druid/Query.java | 3 + .../query/metadata/AllColumnIncluderator.java | 31 + .../druid/query/metadata/ColumnAnalysis.java | 119 ++++ .../query/metadata/ColumnIncluderator.java | 36 + .../metadata/ListColumnIncluderator.java | 56 ++ .../metadata/NoneColumnIncluderator.java | 31 + .../metadata/SegmentAnalysis.java} | 77 +- .../query/metadata/SegmentMetadataQuery.java | 44 +- .../SegmentMetadataQueryQueryToolChest.java | 134 +++- .../query/segment/QuerySegmentSpecs.java | 45 ++ .../TimeseriesQueryQueryToolChest.java | 5 +- .../druid/index/column/BitmapIndex.java | 2 + .../index/column/DictionaryEncodedColumn.java | 2 +- .../druid/index/column/GenericColumn.java | 2 +- .../column/IndexedFloatsGenericColumn.java | 2 +- .../column/IndexedLongsGenericColumn.java | 2 +- .../druid/index/column/SimpleColumn.java | 2 +- .../column/SimpleDictionaryEncodedColumn.java | 2 +- .../index/column/StringMultiValueColumn.java | 2 +- .../serde/BitmapIndexColumnPartSupplier.java | 13 + .../index/v1/serde/ComplexMetricSerde.java | 26 +- .../druid/realtime/RealtimePlumberSchool.java | 4 - .../druid/coordination/ServerManager.java | 3 +- .../druid/index/IncrementalIndexSegment.java | 2 +- .../druid/index/QueryableIndexSegment.java | 2 +- .../java/com/metamx/druid/index/Segment.java | 2 +- .../index/v1/MMappedIndexQueryableIndex.java | 103 --- .../index/v1/MMappedIndexStorageAdapter.java | 666 ------------------ .../v1/QueryableIndexStorageAdapter.java | 8 +- .../druid/query/metadata/SegmentAnalyzer.java | 160 +++++ .../metadata/SegmentMetadataQueryEngine.java | 109 --- .../SegmentMetadataQueryRunnerFactory.java | 92 ++- .../druid/coordination/ServerManagerTest.java | 2 +- .../druid/loading/NoopSegmentLoader.java | 2 +- .../druid/query/QueryRunnerTestHelper.java | 10 +- .../query/metadata/SegmentAnalyzerTest.java | 102 +++ 37 files changed, 875 insertions(+), 1036 deletions(-) create mode 100644 client/src/main/java/com/metamx/druid/query/metadata/AllColumnIncluderator.java create mode 100644 client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java create mode 100644 client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java create mode 100644 client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java create mode 100644 client/src/main/java/com/metamx/druid/query/metadata/NoneColumnIncluderator.java rename client/src/main/java/com/metamx/druid/{result/SegmentMetadataResultValue.java => query/metadata/SegmentAnalysis.java} (53%) create mode 100644 client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpecs.java delete mode 100644 server/src/main/java/com/metamx/druid/index/v1/MMappedIndexQueryableIndex.java delete mode 100644 server/src/main/java/com/metamx/druid/index/v1/MMappedIndexStorageAdapter.java create mode 100644 server/src/main/java/com/metamx/druid/query/metadata/SegmentAnalyzer.java delete mode 100644 server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryEngine.java create mode 100644 server/src/test/java/com/metamx/druid/query/metadata/SegmentAnalyzerTest.java diff --git a/client/src/main/java/com/metamx/druid/BaseQuery.java b/client/src/main/java/com/metamx/druid/BaseQuery.java index 4538467c167..76448ed9f17 100644 --- a/client/src/main/java/com/metamx/druid/BaseQuery.java +++ b/client/src/main/java/com/metamx/druid/BaseQuery.java @@ -22,6 +22,7 @@ package com.metamx.druid; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import com.metamx.common.guava.Sequence; +import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.segment.QuerySegmentWalker; import org.codehaus.jackson.annotate.JsonProperty; @@ -72,7 +73,12 @@ public abstract class BaseQuery implements Query @Override public Sequence run(QuerySegmentWalker walker) { - return querySegmentSpec.lookup(this, walker).run(this); + return run(querySegmentSpec.lookup(this, walker)); + } + + public Sequence run(QueryRunner runner) + { + return runner.run(this); } @Override diff --git a/client/src/main/java/com/metamx/druid/Query.java b/client/src/main/java/com/metamx/druid/Query.java index bd1dc49702a..4c4e7f715b4 100644 --- a/client/src/main/java/com/metamx/druid/Query.java +++ b/client/src/main/java/com/metamx/druid/Query.java @@ -20,6 +20,7 @@ package com.metamx.druid; import com.metamx.common.guava.Sequence; +import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.group.GroupByQuery; import com.metamx.druid.query.metadata.SegmentMetadataQuery; import com.metamx.druid.query.search.SearchQuery; @@ -57,6 +58,8 @@ public interface Query public Sequence run(QuerySegmentWalker walker); + public Sequence run(QueryRunner runner); + public List getIntervals(); public Duration getDuration(); diff --git a/client/src/main/java/com/metamx/druid/query/metadata/AllColumnIncluderator.java b/client/src/main/java/com/metamx/druid/query/metadata/AllColumnIncluderator.java new file mode 100644 index 00000000000..8687f213503 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/query/metadata/AllColumnIncluderator.java @@ -0,0 +1,31 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.query.metadata; + +/** + */ +public class AllColumnIncluderator implements ColumnIncluderator +{ + @Override + public boolean include(String columnName) + { + return true; + } +} diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java b/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java new file mode 100644 index 00000000000..33b894ca07a --- /dev/null +++ b/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java @@ -0,0 +1,119 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.query.metadata; + +import com.google.common.base.Preconditions; +import com.metamx.druid.index.column.ValueType; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +/** +*/ +public class ColumnAnalysis +{ + private static final String ERROR_PREFIX = "error:"; + + public static ColumnAnalysis error(String reason) + { + return new ColumnAnalysis(ERROR_PREFIX + reason, -1, null); + } + + private final String type; + private final long size; + private final Integer cardinality; + + @JsonCreator + public ColumnAnalysis( + @JsonProperty("type") ValueType type, + @JsonProperty("size") long size, + @JsonProperty("cardinality") Integer cardinality + ) + { + this(type.name(), size, cardinality); + } + + private ColumnAnalysis( + String type, + long size, + Integer cardinality + ) + { + this.type = type; + this.size = size; + this.cardinality = cardinality; + } + + @JsonProperty + public String getType() + { + return type; + } + + @JsonProperty + public long getSize() + { + return size; + } + + @JsonProperty + public Integer getCardinality() + { + return cardinality; + } + + public boolean isError() + { + return type.startsWith(ERROR_PREFIX); + } + + public ColumnAnalysis fold(ColumnAnalysis rhs) + { + if (rhs == null) { + return this; + } + + if (!type.equals(rhs.getType())) { + return ColumnAnalysis.error("cannot_merge_diff_types"); + } + + Integer cardinality = getCardinality(); + final Integer rhsCardinality = rhs.getCardinality(); + if (cardinality == null) { + cardinality = rhsCardinality; + } + else { + if (rhsCardinality != null) { + cardinality = Math.max(cardinality, rhsCardinality); + } + } + + return new ColumnAnalysis(type, size + rhs.getSize(), cardinality); + } + + @Override + public String toString() + { + return "ColumnAnalysis{" + + "type='" + type + '\'' + + ", size=" + size + + ", cardinality=" + cardinality + + '}'; + } +} diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java b/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java new file mode 100644 index 00000000000..466167d48fd --- /dev/null +++ b/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java @@ -0,0 +1,36 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.query.metadata; + +import org.codehaus.jackson.annotate.JsonSubTypes; +import org.codehaus.jackson.annotate.JsonTypeInfo; + +/** + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "none", value= NoneColumnIncluderator.class), + @JsonSubTypes.Type(name = "all", value= AllColumnIncluderator.class), + @JsonSubTypes.Type(name = "list", value= ListColumnIncluderator.class) +}) +public interface ColumnIncluderator +{ + public boolean include(String columnName); +} diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java b/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java new file mode 100644 index 00000000000..4048dcd420a --- /dev/null +++ b/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java @@ -0,0 +1,56 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.query.metadata; + +import com.google.common.collect.Sets; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +import java.util.Collections; +import java.util.List; +import java.util.Set; + +/** + */ +public class ListColumnIncluderator implements ColumnIncluderator +{ + private final Set columns; + + @JsonCreator + public ListColumnIncluderator( + @JsonProperty("columns") List columns + ) + { + this.columns = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + this.columns.addAll(columns); + } + + @JsonProperty + public Set getColumns() + { + return Collections.unmodifiableSet(columns); + } + + @Override + public boolean include(String columnName) + { + return columns.contains(columnName); + } +} diff --git a/client/src/main/java/com/metamx/druid/query/metadata/NoneColumnIncluderator.java b/client/src/main/java/com/metamx/druid/query/metadata/NoneColumnIncluderator.java new file mode 100644 index 00000000000..56e6842b7ad --- /dev/null +++ b/client/src/main/java/com/metamx/druid/query/metadata/NoneColumnIncluderator.java @@ -0,0 +1,31 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.query.metadata; + +/** + */ +public class NoneColumnIncluderator implements ColumnIncluderator +{ + @Override + public boolean include(String columnName) + { + return false; + } +} diff --git a/client/src/main/java/com/metamx/druid/result/SegmentMetadataResultValue.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java similarity index 53% rename from client/src/main/java/com/metamx/druid/result/SegmentMetadataResultValue.java rename to client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java index 5904264017e..1182bfeb9fa 100644 --- a/client/src/main/java/com/metamx/druid/result/SegmentMetadataResultValue.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java @@ -17,61 +17,34 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.result; +package com.metamx.druid.query.metadata; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; +import org.joda.time.Interval; +import java.util.List; import java.util.Map; -public class SegmentMetadataResultValue +public class SegmentAnalysis { - public static class Dimension { - @JsonProperty public long size; - @JsonProperty public int cardinality; - - @JsonCreator - public Dimension( - @JsonProperty("size") long size, - @JsonProperty("cardinality") int cardinality - ) - { - this.size = size; - this.cardinality = cardinality; - } - } - public static class Metric { - @JsonProperty public String type; - @JsonProperty public long size; - - @JsonCreator - public Metric( - @JsonProperty("type") String type, - @JsonProperty("size") long size - ) - { - this.type = type; - this.size = size; - } - } - private final String id; - private final Map dimensions; - private final Map metrics; + private final List interval; + private final Map columns; private final long size; @JsonCreator - public SegmentMetadataResultValue( + public SegmentAnalysis( @JsonProperty("id") String id, - @JsonProperty("dimensions") Map dimensions, - @JsonProperty("metrics") Map metrics, + @JsonProperty("intervals") List interval, + @JsonProperty("columns") Map columns, @JsonProperty("size") long size ) { this.id = id; - this.dimensions = dimensions; - this.metrics = metrics; + this.interval = interval; + this.columns = columns; this.size = size; } @@ -82,15 +55,15 @@ public class SegmentMetadataResultValue } @JsonProperty - public Map getDimensions() + public List getIntervals() { - return dimensions; + return interval; } @JsonProperty - public Map getMetrics() + public Map getColumns() { - return metrics; + return columns; } @JsonProperty @@ -98,4 +71,24 @@ public class SegmentMetadataResultValue { return size; } + + public String toDetailedString() + { + return "SegmentAnalysis{" + + "id='" + id + '\'' + + ", interval=" + interval + + ", columns=" + columns + + ", size=" + size + + '}'; + } + + @Override + public String toString() + { + return "SegmentAnalysis{" + + "id='" + id + '\'' + + ", interval=" + interval + + ", size=" + size + + '}'; + } } diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java index e72b85a4423..7e0d04c0739 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java @@ -22,26 +22,40 @@ package com.metamx.druid.query.metadata; import com.metamx.druid.BaseQuery; import com.metamx.druid.Query; import com.metamx.druid.query.segment.QuerySegmentSpec; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.SegmentMetadataResultValue; import org.codehaus.jackson.annotate.JsonProperty; import java.util.Map; -public class SegmentMetadataQuery extends BaseQuery> +public class SegmentMetadataQuery extends BaseQuery { + private final ColumnIncluderator toInclude; + private final boolean merge; + public SegmentMetadataQuery( @JsonProperty("dataSource") String dataSource, @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec, + @JsonProperty("toInclude") ColumnIncluderator toInclude, + @JsonProperty("merge") Boolean merge, @JsonProperty("context") Map context ) { - super( - dataSource, - querySegmentSpec, - context - ); + super(dataSource, querySegmentSpec, context); + + this.toInclude = toInclude == null ? new AllColumnIncluderator() : toInclude; + this.merge = merge == null ? false : merge; + } + + @JsonProperty + public ColumnIncluderator getToInclude() + { + return toInclude; + } + + @JsonProperty + public boolean isMerge() + { + return merge; } @Override @@ -57,22 +71,16 @@ public class SegmentMetadataQuery extends BaseQuery> withOverriddenContext(Map contextOverride) + public Query withOverriddenContext(Map contextOverride) { return new SegmentMetadataQuery( - getDataSource(), - getQuerySegmentSpec(), - computeOverridenContext(contextOverride) + getDataSource(), getQuerySegmentSpec(), toInclude, merge, computeOverridenContext(contextOverride) ); } @Override - public Query> withQuerySegmentSpec(QuerySegmentSpec spec) + public Query withQuerySegmentSpec(QuerySegmentSpec spec) { - return new SegmentMetadataQuery( - getDataSource(), - spec, - getContext() - ); + return new SegmentMetadataQuery(getDataSource(), spec, toInclude, merge, getContext()); } } diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 14ef61b8d3c..447bba2e5c6 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -22,32 +22,116 @@ package com.metamx.druid.query.metadata; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Joiner; -import com.google.common.collect.ImmutableList; -import com.metamx.common.guava.ConcatSequence; +import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; +import com.metamx.common.ISE; +import com.metamx.common.guava.Comparators; import com.metamx.common.guava.Sequence; -import com.metamx.common.guava.Sequences; +import com.metamx.common.guava.nary.BinaryFn; +import com.metamx.druid.Query; +import com.metamx.druid.collect.OrderedMergeSequence; import com.metamx.druid.query.CacheStrategy; -import com.metamx.druid.query.ConcatQueryRunner; import com.metamx.druid.query.MetricManipulationFn; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryToolChest; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.SegmentMetadataResultValue; +import com.metamx.druid.query.ResultMergeQueryRunner; +import com.metamx.druid.utils.JodaUtils; import com.metamx.emitter.service.ServiceMetricEvent; import org.codehaus.jackson.type.TypeReference; import org.joda.time.Interval; import org.joda.time.Minutes; +import javax.annotation.Nullable; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; -public class SegmentMetadataQueryQueryToolChest implements QueryToolChest, SegmentMetadataQuery> + +public class SegmentMetadataQueryQueryToolChest implements QueryToolChest { - - private static final TypeReference> TYPE_REFERENCE = new TypeReference>(){}; + private static final TypeReference TYPE_REFERENCE = new TypeReference(){}; @Override - public QueryRunner> mergeResults(final QueryRunner> runner) + public QueryRunner mergeResults(final QueryRunner runner) { - return new ConcatQueryRunner>(Sequences.simple(ImmutableList.of(runner))); + return new ResultMergeQueryRunner(runner) + { + @Override + protected Ordering makeOrdering(Query query) + { + if (((SegmentMetadataQuery) query).isMerge()) { + // Merge everything always + return new Ordering() + { + @Override + public int compare( + @Nullable SegmentAnalysis left, @Nullable SegmentAnalysis right + ) + { + return 0; + } + }; + } + + return getOrdering(); // No two elements should be equal, so it should never merge + } + + @Override + protected BinaryFn createMergeFn(final Query inQ) + { + return new BinaryFn() + { + private final SegmentMetadataQuery query = (SegmentMetadataQuery) inQ; + + @Override + public SegmentAnalysis apply(SegmentAnalysis arg1, SegmentAnalysis arg2) + { + if (arg1 == null) { + return arg2; + } + + if (arg2 == null) { + return arg1; + } + + if (!query.isMerge()) { + throw new ISE("Merging when a merge isn't supposed to happen[%s], [%s]", arg1, arg2); + } + + List newIntervals = JodaUtils.condenseIntervals( + Iterables.concat(arg1.getIntervals(), arg2.getIntervals()) + ); + + final Map leftColumns = arg1.getColumns(); + final Map rightColumns = arg2.getColumns(); + Map columns = Maps.newTreeMap(); + + Set rightColumnNames = Sets.newHashSet(rightColumns.keySet()); + for (Map.Entry entry : leftColumns.entrySet()) { + final String columnName = entry.getKey(); + columns.put(columnName, entry.getValue().fold(rightColumns.get(columnName))); + rightColumnNames.remove(columnName); + } + + for (String columnName : rightColumnNames) { + columns.put(columnName, rightColumns.get(columnName)); + } + + return new SegmentAnalysis("merged", newIntervals, columns, arg1.getSize() + arg2.getSize()); + } + }; + } + }; + } + + @Override + public Sequence mergeSequences(Sequence> seqOfSequences) + { + return new OrderedMergeSequence(getOrdering(), seqOfSequences); } @Override @@ -67,13 +151,7 @@ public class SegmentMetadataQueryQueryToolChest implements QueryToolChest> mergeSequences(Sequence>> seqOfSequences) - { - return new ConcatSequence>(seqOfSequences); - } - - @Override - public Function, Result> makeMetricManipulatorFn( + public Function makeMetricManipulatorFn( SegmentMetadataQuery query, MetricManipulationFn fn ) { @@ -81,26 +159,38 @@ public class SegmentMetadataQueryQueryToolChest implements QueryToolChest> getResultTypeReference() + public TypeReference getResultTypeReference() { return TYPE_REFERENCE; } @Override - public CacheStrategy, SegmentMetadataQuery> getCacheStrategy(SegmentMetadataQuery query) + public CacheStrategy getCacheStrategy(SegmentMetadataQuery query) { return null; } @Override - public QueryRunner> preMergeQueryDecoration(QueryRunner> runner) + public QueryRunner preMergeQueryDecoration(QueryRunner runner) { return runner; } @Override - public QueryRunner> postMergeQueryDecoration(QueryRunner> runner) + public QueryRunner postMergeQueryDecoration(QueryRunner runner) { return runner; } + + private Ordering getOrdering() + { + return new Ordering() + { + @Override + public int compare(SegmentAnalysis left, SegmentAnalysis right) + { + return left.getId().compareTo(right.getId()); + } + }; + } } diff --git a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpecs.java b/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpecs.java new file mode 100644 index 00000000000..9516db4c508 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpecs.java @@ -0,0 +1,45 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.query.segment; + +import org.joda.time.Interval; + +import java.util.Arrays; +import java.util.List; + +/** + */ +public class QuerySegmentSpecs +{ + public static QuerySegmentSpec create(String isoInterval) + { + return new LegacySegmentSpec(isoInterval); + } + + public static QuerySegmentSpec create(Interval interval) + { + return create(Arrays.asList(interval)); + } + + public static QuerySegmentSpec create(List intervals) + { + return new MultipleIntervalSegmentSpec(intervals); + } +} diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index df619f340f0..d8ef9802dad 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -102,10 +102,7 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest> mergeSequences(Sequence>> seqOfSequences) { - return new OrderedMergeSequence>( - getOrdering(), - seqOfSequences - ); + return new OrderedMergeSequence>(getOrdering(), seqOfSequences); } @Override diff --git a/index-common/src/main/java/com/metamx/druid/index/column/BitmapIndex.java b/index-common/src/main/java/com/metamx/druid/index/column/BitmapIndex.java index 884b126d250..6873292fee2 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/BitmapIndex.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/BitmapIndex.java @@ -25,5 +25,7 @@ import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; */ public interface BitmapIndex { + public int getCardinality(); + public String getValue(int index); public ImmutableConciseSet getConciseSet(String value); } diff --git a/index-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java index 9301734f970..62057cb8e10 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java @@ -25,7 +25,7 @@ import com.metamx.druid.kv.IndexedInts; */ public interface DictionaryEncodedColumn { - public int size(); + public int length(); public boolean hasMultipleValues(); public int getSingleValueRow(int rowNum); public IndexedInts getMultiValueRow(int rowNum); diff --git a/index-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java index c41b4906bd1..530eb2fc232 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java @@ -29,7 +29,7 @@ import java.io.Closeable; */ public interface GenericColumn extends Closeable { - public int size(); + public int length(); public ValueType getType(); public boolean hasMultipleValues(); diff --git a/index-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java index 423b046e106..5df228435c1 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java @@ -38,7 +38,7 @@ public class IndexedFloatsGenericColumn implements GenericColumn } @Override - public int size() + public int length() { return column.size(); } diff --git a/index-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java index 0e96a63924b..211dab35a19 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java @@ -38,7 +38,7 @@ public class IndexedLongsGenericColumn implements GenericColumn } @Override - public int size() + public int length() { return column.size(); } diff --git a/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java index a3884203227..93825a8e9f0 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java @@ -62,7 +62,7 @@ class SimpleColumn implements Column GenericColumn column = null; try { column = genericColumn.get(); - return column.size(); + return column.length(); } finally { Closeables.closeQuietly(column); diff --git a/index-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java index 7a28a53b0af..fbc3877c5fe 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java @@ -44,7 +44,7 @@ public class SimpleDictionaryEncodedColumn implements DictionaryEncodedColumn } @Override - public int size() + public int length() { return hasMultipleValues() ? multiValueColumn.size() : column.size(); } diff --git a/index-common/src/main/java/com/metamx/druid/index/column/StringMultiValueColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/StringMultiValueColumn.java index 053bcee1956..79327cb8400 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/StringMultiValueColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/StringMultiValueColumn.java @@ -67,7 +67,7 @@ public class StringMultiValueColumn extends AbstractColumn return new DictionaryEncodedColumn() { @Override - public int size() + public int length() { return column.size(); } diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java b/index-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java index 6813541ef09..2381639a0a8 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java @@ -22,6 +22,7 @@ package com.metamx.druid.index.serde; import com.google.common.base.Supplier; import com.metamx.druid.index.column.BitmapIndex; import com.metamx.druid.kv.GenericIndexed; +import com.metamx.druid.kv.Indexed; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; /** @@ -46,6 +47,18 @@ public class BitmapIndexColumnPartSupplier implements Supplier { return new BitmapIndex() { + @Override + public int getCardinality() + { + return dictionary.size(); + } + + @Override + public String getValue(int index) + { + return dictionary.get(index); + } + @Override public ImmutableConciseSet getConciseSet(String value) { diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricSerde.java b/index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricSerde.java index 7473aaec327..e54fb1cceca 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricSerde.java @@ -19,6 +19,7 @@ package com.metamx.druid.index.v1.serde; +import com.google.common.base.Function; import com.metamx.druid.index.column.ColumnBuilder; import com.metamx.druid.index.serde.ColumnPartSerde; import com.metamx.druid.kv.ObjectStrategy; @@ -27,10 +28,10 @@ import java.nio.ByteBuffer; /** */ -public interface ComplexMetricSerde +public abstract class ComplexMetricSerde { - public String getTypeName(); - public ComplexMetricExtractor getExtractor(); + public abstract String getTypeName(); + public abstract ComplexMetricExtractor getExtractor(); /** * Deserializes a ByteBuffer and adds it to the ColumnBuilder. This method allows for the ComplexMetricSerde @@ -42,7 +43,7 @@ public interface ComplexMetricSerde * @param buffer the buffer to deserialize * @return a ColumnPartSerde that can serialize out the object that was read from the buffer to the builder */ - public ColumnPartSerde deserializeColumn(ByteBuffer buffer, ColumnBuilder builder); + public abstract ColumnPartSerde deserializeColumn(ByteBuffer buffer, ColumnBuilder builder); /** * This is deprecated because its usage is going to be removed from the code. @@ -55,5 +56,20 @@ public interface ComplexMetricSerde * @return an ObjectStrategy as used by GenericIndexed */ @Deprecated - public ObjectStrategy getObjectStrategy(); + public abstract ObjectStrategy getObjectStrategy(); + + + /** + * Returns a function that can convert the Object provided by the ComplexColumn created through deserializeColumn + * into a number of expected input bytes to produce that object. + * + * This is used to approximate the size of the input data via the SegmentMetadataQuery and does not need to be + * overridden if you do not care about the query. + * + * @return A function that can compute the size of the complex object or null if you cannot/do not want to compute it + */ + public Function inputSizeFn() + { + return null; + } } diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java index 02bd6acbb4c..f4df5e054f8 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java @@ -33,7 +33,6 @@ import com.metamx.common.Pair; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.Query; -import com.metamx.druid.StorageAdapter; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; import com.metamx.druid.client.ServerView; @@ -44,9 +43,6 @@ import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.IndexGranularity; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; -import com.metamx.druid.index.v1.MMappedIndex; -import com.metamx.druid.index.v1.MMappedIndexQueryableIndex; -import com.metamx.druid.index.v1.MMappedIndexStorageAdapter; import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.query.MetricsEmittingQueryRunner; import com.metamx.druid.query.QueryRunner; diff --git a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java index ce4e5f4accb..3bde07908c7 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java +++ b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java @@ -29,7 +29,6 @@ import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.collect.CountingMap; import com.metamx.druid.index.Segment; -import com.metamx.druid.index.v1.SegmentIdAttachedStorageAdapter; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.loading.StorageAdapterLoadingException; import com.metamx.druid.partition.PartitionChunk; @@ -330,7 +329,7 @@ public class ServerManager implements QuerySegmentWalker } }, new BySegmentQueryRunner( - adapter.getSegmentIdentifier(), + adapter.getIdentifier(), adapter.getDataInterval().getStart(), factory.createRunner(adapter) ) diff --git a/server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java b/server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java index 19b71c0db6b..79c7dc069b6 100644 --- a/server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java +++ b/server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java @@ -38,7 +38,7 @@ public class IncrementalIndexSegment implements Segment } @Override - public String getSegmentIdentifier() + public String getIdentifier() { throw new UnsupportedOperationException(); } diff --git a/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java b/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java index 8df88fcdad1..770eb783d3b 100644 --- a/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java +++ b/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java @@ -37,7 +37,7 @@ public class QueryableIndexSegment implements Segment } @Override - public String getSegmentIdentifier() + public String getIdentifier() { return identifier; } diff --git a/server/src/main/java/com/metamx/druid/index/Segment.java b/server/src/main/java/com/metamx/druid/index/Segment.java index 7bf0acce96c..b2edda90f8d 100644 --- a/server/src/main/java/com/metamx/druid/index/Segment.java +++ b/server/src/main/java/com/metamx/druid/index/Segment.java @@ -26,7 +26,7 @@ import org.joda.time.Interval; */ public interface Segment { - public String getSegmentIdentifier(); + public String getIdentifier(); public Interval getDataInterval(); public QueryableIndex asQueryableIndex(); public StorageAdapter asStorageAdapter(); diff --git a/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexQueryableIndex.java b/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexQueryableIndex.java deleted file mode 100644 index 3c9d62d6776..00000000000 --- a/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexQueryableIndex.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.index.v1; - -import com.metamx.druid.index.QueryableIndex; -import com.metamx.druid.index.column.Column; -import com.metamx.druid.index.column.ComplexColumnImpl; -import com.metamx.druid.index.column.FloatColumn; -import com.metamx.druid.index.column.LongColumn; -import com.metamx.druid.index.column.StringMultiValueColumn; -import com.metamx.druid.kv.Indexed; -import com.metamx.druid.kv.VSizeIndexed; -import org.joda.time.Interval; - -/** - */ -public class MMappedIndexQueryableIndex implements QueryableIndex -{ - private final MMappedIndex index; - - public MMappedIndexQueryableIndex( - MMappedIndex index - ) - { - this.index = index; - } - - public MMappedIndex getIndex() - { - return index; - } - - @Override - public Interval getDataInterval() - { - return index.getDataInterval(); - } - - @Override - public int getNumRows() - { - return index.getTimestamps().size(); - } - - @Override - public Indexed getColumnNames() - { - return null; - } - - @Override - public Indexed getAvailableDimensions() - { - return index.getAvailableDimensions(); - } - - @Override - public Column getTimeColumn() - { - return new LongColumn(index.timestamps); - } - - @Override - public Column getColumn(String columnName) - { - final MetricHolder metricHolder = index.getMetricHolder(columnName); - if (metricHolder == null) { - final VSizeIndexed dimColumn = index.getDimColumn(columnName); - if (dimColumn == null) { - return null; - } - - return new StringMultiValueColumn( - index.getDimValueLookup(columnName), - dimColumn, - index.getInvertedIndexes().get(columnName) - ); - } - else if (metricHolder.getType() == MetricHolder.MetricType.FLOAT) { - return new FloatColumn(metricHolder.floatType); - } - else { - return new ComplexColumnImpl(metricHolder.getTypeName(), metricHolder.getComplexType()); - } - } -} diff --git a/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexStorageAdapter.java b/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexStorageAdapter.java deleted file mode 100644 index 6cffaf14141..00000000000 --- a/server/src/main/java/com/metamx/druid/index/v1/MMappedIndexStorageAdapter.java +++ /dev/null @@ -1,666 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.index.v1; - -import com.google.common.base.Function; -import com.google.common.base.Functions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Maps; -import com.google.common.io.Closeables; -import com.metamx.common.collect.MoreIterators; -import com.metamx.common.guava.FunctionalIterable; -import com.metamx.common.guava.FunctionalIterator; -import com.metamx.druid.BaseStorageAdapter; -import com.metamx.druid.Capabilities; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.index.brita.BitmapIndexSelector; -import com.metamx.druid.index.brita.Filter; -import com.metamx.druid.index.v1.processing.Cursor; -import com.metamx.druid.index.v1.processing.DimensionSelector; -import com.metamx.druid.index.v1.processing.Offset; -import com.metamx.druid.kv.Indexed; -import com.metamx.druid.kv.IndexedFloats; -import com.metamx.druid.kv.IndexedInts; -import com.metamx.druid.kv.IndexedLongs; -import com.metamx.druid.processing.ComplexMetricSelector; -import com.metamx.druid.processing.FloatMetricSelector; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -import java.io.Closeable; -import java.util.Iterator; -import java.util.Map; - -/** - */ -public class MMappedIndexStorageAdapter extends BaseStorageAdapter -{ - private final MMappedIndex index; - - public MMappedIndexStorageAdapter( - MMappedIndex index - ) - { - this.index = index; - } - - public MMappedIndex getIndex() - { - return index; - } - - @Override - public String getSegmentIdentifier() - { - throw new UnsupportedOperationException(); - } - - @Override - public Interval getInterval() - { - return index.getDataInterval(); - } - - @Override - public int getDimensionCardinality(String dimension) - { - final Indexed dimValueLookup = index.getDimValueLookup(dimension.toLowerCase()); - if (dimValueLookup == null) { - return 0; - } - return dimValueLookup.size(); - } - - @Override - public DateTime getMinTime() - { - final IndexedLongs timestamps = index.getReadOnlyTimestamps(); - final DateTime retVal = new DateTime(timestamps.get(0)); - Closeables.closeQuietly(timestamps); - return retVal; - } - - @Override - public DateTime getMaxTime() - { - final IndexedLongs timestamps = index.getReadOnlyTimestamps(); - final DateTime retVal = new DateTime(timestamps.get(timestamps.size() - 1)); - Closeables.closeQuietly(timestamps); - return retVal; - } - - @Override - public Capabilities getCapabilities() - { - return Capabilities.builder().dimensionValuesSorted(true).build(); - } - - @Override - public Iterable makeCursors(Filter filter, Interval interval, QueryGranularity gran) - { - Interval actualInterval = interval; - if (!actualInterval.overlaps(index.dataInterval)) { - return ImmutableList.of(); - } - - if (actualInterval.getStart().isBefore(index.dataInterval.getStart())) { - actualInterval = actualInterval.withStart(index.dataInterval.getStart()); - } - if (actualInterval.getEnd().isAfter(index.dataInterval.getEnd())) { - actualInterval = actualInterval.withEnd(index.dataInterval.getEnd()); - } - - final Iterable iterable; - if (filter == null) { - iterable = new NoFilterCursorIterable(index, actualInterval, gran); - } else { - Offset offset = new ConciseOffset(filter.goConcise(new MMappedBitmapIndexSelector(index))); - - iterable = new CursorIterable(index, actualInterval, gran, offset); - } - - return FunctionalIterable.create(iterable).keep(Functions.identity()); - } - - @Override - public Indexed getAvailableDimensions() - { - return index.getAvailableDimensions(); - } - - @Override - public Indexed getDimValueLookup(String dimension) - { - return index.getDimValueLookup(dimension.toLowerCase()); - } - - @Override - public ImmutableConciseSet getInvertedIndex(String dimension, String dimVal) - { - return index.getInvertedIndex(dimension.toLowerCase(), dimVal); - } - - @Override - public Offset getFilterOffset(Filter filter) - { - return new ConciseOffset( - filter.goConcise( - new MMappedBitmapIndexSelector(index) - ) - ); - } - - private static class CursorIterable implements Iterable - { - private final MMappedIndex index; - private final Interval interval; - private final QueryGranularity gran; - private final Offset offset; - - public CursorIterable( - MMappedIndex index, - Interval interval, - QueryGranularity gran, - Offset offset - ) - { - this.index = index; - this.interval = interval; - this.gran = gran; - this.offset = offset; - } - - @Override - public Iterator iterator() - { - final Offset baseOffset = offset.clone(); - - final Map metricHolderCache = Maps.newHashMap(); - final IndexedLongs timestamps = index.getReadOnlyTimestamps(); - - final FunctionalIterator retVal = FunctionalIterator - .create(gran.iterable(interval.getStartMillis(), interval.getEndMillis()).iterator()) - .transform( - new Function() - { - - @Override - public Cursor apply(final Long input) - { - final long timeStart = Math.max(interval.getStartMillis(), input); - while (baseOffset.withinBounds() - && timestamps.get(baseOffset.getOffset()) < timeStart) { - baseOffset.increment(); - } - - final Offset offset = new TimestampCheckingOffset( - baseOffset, timestamps, Math.min(interval.getEndMillis(), gran.next(timeStart)) - ); - - return new Cursor() - { - private final Offset initOffset = offset.clone(); - private final DateTime myBucket = gran.toDateTime(input); - private Offset cursorOffset = offset; - - @Override - public DateTime getTime() - { - return myBucket; - } - - @Override - public void advance() - { - cursorOffset.increment(); - } - - @Override - public boolean isDone() - { - return !cursorOffset.withinBounds(); - } - - @Override - public void reset() - { - cursorOffset = initOffset.clone(); - } - - @Override - public DimensionSelector makeDimensionSelector(String dimension) - { - final String dimensionName = dimension.toLowerCase(); - final Indexed rowVals = index.getDimColumn(dimensionName); - final Indexed dimValueLookup = index.getDimValueLookup(dimensionName); - - if (rowVals == null) { - return null; - } - - return new DimensionSelector() - { - @Override - public IndexedInts getRow() - { - return rowVals.get(cursorOffset.getOffset()); - } - - @Override - public int getValueCardinality() - { - return dimValueLookup.size(); - } - - @Override - public String lookupName(int id) - { - final String retVal = dimValueLookup.get(id); - return retVal == null ? "" : retVal; - } - - @Override - public int lookupId(String name) - { - return ("".equals(name)) ? dimValueLookup.indexOf(null) : dimValueLookup.indexOf(name); - } - }; - } - - @Override - public FloatMetricSelector makeFloatMetricSelector(String metric) - { - final String metricName = metric.toLowerCase(); - IndexedFloats cachedMetricVals = (IndexedFloats) metricHolderCache.get(metricName); - - if (cachedMetricVals == null) { - MetricHolder holder = index.getMetricHolder(metricName); - if (holder != null) { - cachedMetricVals = holder.getFloatType(); - metricHolderCache.put(metricName, cachedMetricVals); - } - } - - if (cachedMetricVals == null) { - return new FloatMetricSelector() - { - @Override - public float get() - { - return 0.0f; - } - }; - } - - final IndexedFloats metricVals = cachedMetricVals; - return new FloatMetricSelector() - { - @Override - public float get() - { - return metricVals.get(cursorOffset.getOffset()); - } - }; - } - - @Override - public ComplexMetricSelector makeComplexMetricSelector(String metric) - { - final String metricName = metric.toLowerCase(); - Indexed cachedMetricVals = (Indexed) metricHolderCache.get(metricName); - - if (cachedMetricVals == null) { - MetricHolder holder = index.getMetricHolder(metricName); - if (holder != null) { - cachedMetricVals = holder.getComplexType(); - metricHolderCache.put(metricName, cachedMetricVals); - } - } - - if (cachedMetricVals == null) { - return null; - } - - final Indexed metricVals = cachedMetricVals; - return new ComplexMetricSelector() - { - @Override - public Class classOfObject() - { - return metricVals.getClazz(); - } - - @Override - public Object get() - { - return metricVals.get(cursorOffset.getOffset()); - } - }; - } - }; - } - } - ); - - // This after call is not perfect, if there is an exception during processing, it will never get called, - // but it's better than nothing and doing this properly all the time requires a lot more fixerating - return MoreIterators.after( - retVal, - new Runnable() - { - @Override - public void run() - { - Closeables.closeQuietly(timestamps); - for (Object object : metricHolderCache.values()) { - if (object instanceof Closeable) { - Closeables.closeQuietly((Closeable) object); - } - } - } - } - ); - } - } - - private static class TimestampCheckingOffset implements Offset - { - private final Offset baseOffset; - private final IndexedLongs timestamps; - private final long threshold; - - public TimestampCheckingOffset( - Offset baseOffset, - IndexedLongs timestamps, - long threshold - ) - { - this.baseOffset = baseOffset; - this.timestamps = timestamps; - this.threshold = threshold; - } - - @Override - public int getOffset() - { - return baseOffset.getOffset(); - } - - @Override - public Offset clone() - { - return new TimestampCheckingOffset(baseOffset.clone(), timestamps, threshold); - } - - @Override - public boolean withinBounds() - { - return baseOffset.withinBounds() && timestamps.get(baseOffset.getOffset()) < threshold; - } - - @Override - public void increment() - { - baseOffset.increment(); - } - } - - private static class NoFilterCursorIterable implements Iterable - { - private final MMappedIndex index; - private final Interval interval; - private final QueryGranularity gran; - - public NoFilterCursorIterable( - MMappedIndex index, - Interval interval, - QueryGranularity gran - ) - { - this.index = index; - this.interval = interval; - this.gran = gran; - } - - /** - * This produces iterators of Cursor objects that must be fully processed (until isDone() returns true) before the - * next Cursor is processed. It is *not* safe to pass these cursors off to another thread for parallel processing - * - * @return - */ - @Override - public Iterator iterator() - { - final Map metricCacheMap = Maps.newHashMap(); - final IndexedLongs timestamps = index.getReadOnlyTimestamps(); - - final FunctionalIterator retVal = FunctionalIterator - .create(gran.iterable(interval.getStartMillis(), interval.getEndMillis()).iterator()) - .transform( - new Function() - { - private int currRow = 0; - - @Override - public Cursor apply(final Long input) - { - final long timeStart = Math.max(interval.getStartMillis(), input); - while (currRow < timestamps.size() && timestamps.get(currRow) < timeStart) { - ++currRow; - } - - return new Cursor() - { - private final DateTime myBucket = gran.toDateTime(input); - private final long nextBucket = Math.min(gran.next(myBucket.getMillis()), interval.getEndMillis()); - private final int initRow = currRow; - - @Override - public DateTime getTime() - { - return myBucket; - } - - @Override - public void advance() - { - ++currRow; - } - - @Override - public boolean isDone() - { - return currRow >= timestamps.size() || timestamps.get(currRow) >= nextBucket; - } - - @Override - public void reset() - { - currRow = initRow; - } - - @Override - public DimensionSelector makeDimensionSelector(final String dimension) - { - final String dimensionName = dimension.toLowerCase(); - final Indexed rowVals = index.getDimColumn(dimensionName); - final Indexed dimValueLookup = index.getDimValueLookup(dimensionName); - - if (rowVals == null) { - return null; - } - - return new DimensionSelector() - { - @Override - public IndexedInts getRow() - { - return rowVals.get(currRow); - } - - @Override - public int getValueCardinality() - { - return dimValueLookup.size(); - } - - @Override - public String lookupName(int id) - { - final String retVal = dimValueLookup.get(id); - return retVal == null ? "" : retVal; - } - - @Override - public int lookupId(String name) - { - return ("".equals(name)) ? dimValueLookup.indexOf(null) : dimValueLookup.indexOf(name); - } - }; - } - - @Override - public FloatMetricSelector makeFloatMetricSelector(String metric) - { - final String metricName = metric.toLowerCase(); - IndexedFloats cachedMetricVals = (IndexedFloats) metricCacheMap.get(metricName); - - if (cachedMetricVals == null) { - final MetricHolder metricHolder = index.getMetricHolder(metricName); - if (metricHolder != null) { - cachedMetricVals = metricHolder.getFloatType(); - if (cachedMetricVals != null) { - metricCacheMap.put(metricName, cachedMetricVals); - } - } - } - - if (cachedMetricVals == null) { - return new FloatMetricSelector() - { - @Override - public float get() - { - return 0.0f; - } - }; - } - - final IndexedFloats metricVals = cachedMetricVals; - return new FloatMetricSelector() - { - @Override - public float get() - { - return metricVals.get(currRow); - } - }; - } - - @Override - public ComplexMetricSelector makeComplexMetricSelector(String metric) - { - final String metricName = metric.toLowerCase(); - Indexed cachedMetricVals = (Indexed) metricCacheMap.get(metricName); - - if (cachedMetricVals == null) { - final MetricHolder metricHolder = index.getMetricHolder(metricName); - - if (metricHolder != null) { - cachedMetricVals = metricHolder.getComplexType(); - if (cachedMetricVals != null) { - metricCacheMap.put(metricName, cachedMetricVals); - } - } - } - - if (cachedMetricVals == null) { - return null; - } - - final Indexed metricVals = cachedMetricVals; - return new ComplexMetricSelector() - { - @Override - public Class classOfObject() - { - return metricVals.getClazz(); - } - - @Override - public Object get() - { - return metricVals.get(currRow); - } - }; - } - }; - } - } - ); - - return MoreIterators.after( - retVal, - new Runnable() - { - @Override - public void run() - { - Closeables.closeQuietly(timestamps); - for (Object object : metricCacheMap.values()) { - if (object instanceof Closeable) { - Closeables.closeQuietly((Closeable) object); - } - } - } - } - ); - } - } - - private static class MMappedBitmapIndexSelector implements BitmapIndexSelector - { - private final MMappedIndex index; - - public MMappedBitmapIndexSelector(final MMappedIndex index) - { - this.index = index; - } - - @Override - public Indexed getDimensionValues(String dimension) - { - return index.getDimValueLookup(dimension.toLowerCase()); - } - - @Override - public int getNumRows() - { - return index.getReadOnlyTimestamps().size(); - } - - @Override - public ImmutableConciseSet getConciseInvertedIndex(String dimension, String value) - { - return index.getInvertedIndex(dimension.toLowerCase(), value); - } - } -} diff --git a/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java b/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java index 445dd937ce5..9f159eb7aeb 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java +++ b/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java @@ -113,7 +113,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter GenericColumn column = null; try { column = index.getTimeColumn().getGenericColumn(); - return new DateTime(column.getLongSingleValueRow(column.size() - 1)); + return new DateTime(column.getLongSingleValueRow(column.length() - 1)); } finally { Closeables.closeQuietly(column); @@ -572,7 +572,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter public Cursor apply(final Long input) { final long timeStart = Math.max(interval.getStartMillis(), input); - while (currRow < timestamps.size() && timestamps.getLongSingleValueRow(currRow) < timeStart) { + while (currRow < timestamps.length() && timestamps.getLongSingleValueRow(currRow) < timeStart) { ++currRow; } @@ -597,7 +597,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter @Override public boolean isDone() { - return currRow >= timestamps.size() || timestamps.getLongSingleValueRow(currRow) >= nextBucket; + return currRow >= timestamps.length() || timestamps.getLongSingleValueRow(currRow) >= nextBucket; } @Override @@ -848,7 +848,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter GenericColumn column = null; try { column = index.getTimeColumn().getGenericColumn(); - return column.size(); + return column.length(); } finally { Closeables.closeQuietly(column); diff --git a/server/src/main/java/com/metamx/druid/query/metadata/SegmentAnalyzer.java b/server/src/main/java/com/metamx/druid/query/metadata/SegmentAnalyzer.java new file mode 100644 index 00000000000..a219dbae9b4 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/query/metadata/SegmentAnalyzer.java @@ -0,0 +1,160 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.query.metadata; + +import com.google.common.base.Charsets; +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; +import com.google.common.primitives.Floats; +import com.google.common.primitives.Longs; +import com.metamx.common.logger.Logger; +import com.metamx.druid.index.QueryableIndex; +import com.metamx.druid.index.column.BitmapIndex; +import com.metamx.druid.index.column.Column; +import com.metamx.druid.index.column.ColumnCapabilities; +import com.metamx.druid.index.column.ComplexColumn; +import com.metamx.druid.index.column.ValueType; +import com.metamx.druid.index.v1.serde.ComplexMetricSerde; +import com.metamx.druid.index.v1.serde.ComplexMetrics; + +import java.util.Map; + + +public class SegmentAnalyzer +{ + private static final Logger log = new Logger(SegmentAnalyzer.class); + + /** + * This is based on the minimum size of a timestamp (POSIX seconds). An ISO timestamp will actually be more like 24+ + */ + private static final int NUM_BYTES_IN_TIMESTAMP = 10; + + /** + * This is based on assuming 6 units of precision, one decimal point and a single value left of the decimal + */ + private static final int NUM_BYTES_IN_TEXT_FLOAT = 8; + + public Map analyze(QueryableIndex index) + { + Preconditions.checkNotNull(index, "Index cannot be null"); + + Map columns = Maps.newTreeMap(); + + for (String columnName : index.getColumnNames()) { + final Column column = index.getColumn(columnName); + final ColumnCapabilities capabilities = column.getCapabilities(); + + final ColumnAnalysis analysis; + final ValueType type = capabilities.getType(); + switch(type) { + case LONG: + analysis = analyzeLongColumn(column); + break; + case FLOAT: + analysis = analyzeFloatColumn(column); + break; + case STRING: + analysis = analyzeStringColumn(column); + break; + case COMPLEX: + analysis = analyzeComplexColumn(column); + break; + default: + log.warn("Unknown column type[%s].", type); + analysis = ColumnAnalysis.error(String.format("unknown_type_%s", type)); + } + + columns.put(columnName, analysis); + } + + columns.put("__time", lengthBasedAnalysis(index.getTimeColumn(), NUM_BYTES_IN_TIMESTAMP)); + + return columns; + } + + public ColumnAnalysis analyzeLongColumn(Column column) + { + return lengthBasedAnalysis(column, Longs.BYTES); + } + + public ColumnAnalysis analyzeFloatColumn(Column column) + { + return lengthBasedAnalysis(column, NUM_BYTES_IN_TEXT_FLOAT); + } + + private ColumnAnalysis lengthBasedAnalysis(Column column, final int numBytes) + { + final ColumnCapabilities capabilities = column.getCapabilities(); + if (capabilities.hasMultipleValues()) { + return ColumnAnalysis.error("multi_value"); + } + + return new ColumnAnalysis(capabilities.getType(), column.getLength() * numBytes, null); + } + + public ColumnAnalysis analyzeStringColumn(Column column) + { + final ColumnCapabilities capabilities = column.getCapabilities(); + + if (capabilities.hasBitmapIndexes()) { + final BitmapIndex bitmapIndex = column.getBitmapIndex(); + + int cardinality = bitmapIndex.getCardinality(); + long size = 0; + for (int i = 0; i < cardinality; ++i) { + String value = bitmapIndex.getValue(i); + + if (value != null) { + size += value.getBytes(Charsets.UTF_8).length * bitmapIndex.getConciseSet(value).size(); + } + } + + return new ColumnAnalysis(capabilities.getType(), size, cardinality); + } + + return ColumnAnalysis.error("string_no_bitmap"); + } + + public ColumnAnalysis analyzeComplexColumn(Column column) + { + final ColumnCapabilities capabilities = column.getCapabilities(); + final ComplexColumn complexColumn = column.getComplexColumn(); + + final String typeName = complexColumn.getTypeName(); + final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); + if (serde == null) { + return ColumnAnalysis.error(String.format("unknown_complex_%s", typeName)); + } + + final Function inputSizeFn = serde.inputSizeFn(); + if (inputSizeFn == null) { + return ColumnAnalysis.error("noSizeFn"); + } + + final int length = column.getLength(); + long size = 0; + for (int i = 0; i < length; ++i) { + size += inputSizeFn.apply(complexColumn.getRowValue(i)); + } + + return new ColumnAnalysis(capabilities.getType(), size, null); + } +} \ No newline at end of file diff --git a/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryEngine.java b/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryEngine.java deleted file mode 100644 index 7522b4b4750..00000000000 --- a/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryEngine.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.query.metadata; - -import com.google.common.base.Charsets; -import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Maps; -import com.metamx.common.IAE; -import com.metamx.common.guava.Sequence; -import com.metamx.common.guava.Sequences; -import com.metamx.common.guava.SimpleSequence; -import com.metamx.druid.BaseStorageAdapter; -import com.metamx.druid.StorageAdapter; -import com.metamx.druid.index.v1.SegmentIdAttachedStorageAdapter; -import com.metamx.druid.kv.Indexed; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.SegmentMetadataResultValue; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.util.HashMap; -import java.util.List; - - - -public class SegmentMetadataQueryEngine -{ - public Sequence> process( - final SegmentMetadataQuery query, - StorageAdapter storageAdapter - ) - { - final List intervals = query.getQuerySegmentSpec().getIntervals(); - if (intervals.size() != 1) { - throw new IAE("Should only have one interval, got[%s]", intervals); - } - - if(!(storageAdapter instanceof SegmentIdAttachedStorageAdapter) || - !(((SegmentIdAttachedStorageAdapter)storageAdapter).getDelegate() instanceof BaseStorageAdapter)) { - return Sequences.empty(); - } - - final BaseStorageAdapter adapter = (BaseStorageAdapter) - ((SegmentIdAttachedStorageAdapter) storageAdapter).getDelegate(); - - Function sizeDimension = new Function() - { - @Override - public SegmentMetadataResultValue.Dimension apply(@Nullable String input) - { - long size = 0; - final Indexed lookup = adapter.getDimValueLookup(input); - for (String dimVal : lookup) { - ImmutableConciseSet index = adapter.getInvertedIndex(input, dimVal); - size += (dimVal == null) ? 0 : index.size() * Charsets.UTF_8.encode(dimVal).capacity(); - } - return new SegmentMetadataResultValue.Dimension( - size, - adapter.getDimensionCardinality(input) - ); - } - }; - - // TODO: add metric storage size - - long totalSize = 0; - - HashMap dimensions = Maps.newHashMap(); - for(String input : adapter.getAvailableDimensions()) { - SegmentMetadataResultValue.Dimension d = sizeDimension.apply(input); - dimensions.put(input, d); - totalSize += d.size; - } - - return new SimpleSequence>( - ImmutableList.of( - new Result( - adapter.getMinTime(), - new SegmentMetadataResultValue( - storageAdapter.getSegmentIdentifier(), - dimensions, - ImmutableMap.of(), - totalSize - ) - ) - ) - ); - } -} \ No newline at end of file diff --git a/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java index b616456f4f2..f44110fcf14 100644 --- a/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java @@ -21,83 +21,105 @@ package com.metamx.druid.query.metadata; import com.google.common.base.Function; import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.metamx.common.ISE; +import com.google.common.collect.Maps; import com.metamx.common.guava.ExecutorExecutingSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.Query; -import com.metamx.druid.StorageAdapter; +import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.Segment; import com.metamx.druid.query.ConcatQueryRunner; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.QueryToolChest; -import com.metamx.druid.query.metadata.SegmentMetadataQuery; -import com.metamx.druid.query.metadata.SegmentMetadataQueryEngine; -import com.metamx.druid.query.metadata.SegmentMetadataQueryQueryToolChest; -import com.metamx.druid.result.SegmentMetadataResultValue; -import com.metamx.druid.result.Result; +import java.util.Arrays; +import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; -public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory, SegmentMetadataQuery> +public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory { - private static final SegmentMetadataQueryQueryToolChest toolChest = new SegmentMetadataQueryQueryToolChest() - { - @Override - public QueryRunner> mergeResults(QueryRunner> runner) - { - return new ConcatQueryRunner>(Sequences.simple(ImmutableList.of(runner))); - } - }; + private static final SegmentAnalyzer analyzer = new SegmentAnalyzer(); + private static final SegmentMetadataQueryQueryToolChest toolChest = new SegmentMetadataQueryQueryToolChest(); @Override - public QueryRunner> createRunner(final Segment adapter) + public QueryRunner createRunner(final Segment segment) { - return new QueryRunner>() + return new QueryRunner() { @Override - public Sequence> run(Query> query) + public Sequence run(Query inQ) { - if (!(query instanceof SegmentMetadataQuery)) { - throw new ISE("Got a [%s] which isn't a %s", query.getClass(), SegmentMetadataQuery.class); + SegmentMetadataQuery query = (SegmentMetadataQuery) inQ; + + final QueryableIndex index = segment.asQueryableIndex(); + if (index == null) { + return Sequences.empty(); } - return new SegmentMetadataQueryEngine().process((SegmentMetadataQuery) query, adapter.asStorageAdapter()); + + final Map analyzedColumns = analyzer.analyze(index); + + // Initialize with the size of the whitespace, 1 byte per + long totalSize = analyzedColumns.size() * index.getNumRows(); + + Map columns = Maps.newTreeMap(); + ColumnIncluderator includerator = query.getToInclude(); + for (Map.Entry entry : analyzedColumns.entrySet()) { + final String columnName = entry.getKey(); + final ColumnAnalysis column = entry.getValue(); + + if (!column.isError()) { + totalSize += column.getSize(); + } + if (includerator.include(columnName)) { + columns.put(columnName, column); + } + } + + return Sequences.simple( + Arrays.asList( + new SegmentAnalysis( + segment.getIdentifier(), + Arrays.asList(segment.getDataInterval()), + columns, + totalSize + ) + ) + ); } }; } @Override - public QueryRunner> mergeRunners( - final ExecutorService queryExecutor, Iterable>> queryRunners + public QueryRunner mergeRunners( + final ExecutorService queryExecutor, Iterable> queryRunners ) { - return new ConcatQueryRunner>( + return new ConcatQueryRunner( Sequences.map( Sequences.simple(queryRunners), - new Function>, QueryRunner>>() + new Function, QueryRunner>() { @Override - public QueryRunner> apply(final QueryRunner> input) + public QueryRunner apply(final QueryRunner input) { - return new QueryRunner>() + return new QueryRunner() { @Override - public Sequence> run(final Query> query) + public Sequence run(final Query query) { - Future>> future = queryExecutor.submit( - new Callable>>() + Future> future = queryExecutor.submit( + new Callable>() { @Override - public Sequence> call() throws Exception + public Sequence call() throws Exception { - return new ExecutorExecutingSequence>( + return new ExecutorExecutingSequence( input.run(query), queryExecutor ); @@ -122,7 +144,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory getToolchest() { return toolChest; } diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index 3e3a36b77ed..43dc956c40e 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -328,7 +328,7 @@ public class ServerManagerTest } @Override - public String getSegmentIdentifier() + public String getIdentifier() { return version; } diff --git a/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java b/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java index dc1f640fa5b..29d784d3631 100644 --- a/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java +++ b/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java @@ -35,7 +35,7 @@ public class NoopSegmentLoader implements SegmentLoader return new Segment() { @Override - public String getSegmentIdentifier() + public String getIdentifier() { return segment.getIdentifier(); } diff --git a/server/src/test/java/com/metamx/druid/query/QueryRunnerTestHelper.java b/server/src/test/java/com/metamx/druid/query/QueryRunnerTestHelper.java index de3734c39f8..69e74181faa 100644 --- a/server/src/test/java/com/metamx/druid/query/QueryRunnerTestHelper.java +++ b/server/src/test/java/com/metamx/druid/query/QueryRunnerTestHelper.java @@ -22,7 +22,6 @@ package com.metamx.druid.query; import com.google.common.collect.Lists; import com.metamx.druid.Query; import com.metamx.druid.QueryGranularity; -import com.metamx.druid.StorageAdapter; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.CountAggregatorFactory; import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; @@ -35,13 +34,6 @@ import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.QueryableIndexSegment; import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.IncrementalIndex; -import com.metamx.druid.index.v1.IncrementalIndexStorageAdapter; -import com.metamx.druid.index.v1.Index; -import com.metamx.druid.index.v1.IndexStorageAdapter; -import com.metamx.druid.index.v1.MMappedIndex; -import com.metamx.druid.index.v1.MMappedIndexQueryableIndex; -import com.metamx.druid.index.v1.MMappedIndexStorageAdapter; -import com.metamx.druid.index.v1.QueryableIndexStorageAdapter; import com.metamx.druid.index.v1.TestIndex; import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; import com.metamx.druid.query.segment.QuerySegmentSpec; @@ -132,7 +124,7 @@ public class QueryRunnerTestHelper ); } - private static QueryRunner makeQueryRunner( + public static QueryRunner makeQueryRunner( QueryRunnerFactory> factory, Segment adapter ) diff --git a/server/src/test/java/com/metamx/druid/query/metadata/SegmentAnalyzerTest.java b/server/src/test/java/com/metamx/druid/query/metadata/SegmentAnalyzerTest.java new file mode 100644 index 00000000000..6805bb6c23b --- /dev/null +++ b/server/src/test/java/com/metamx/druid/query/metadata/SegmentAnalyzerTest.java @@ -0,0 +1,102 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.query.metadata; + +import com.google.common.collect.Lists; +import com.metamx.common.guava.Sequences; +import com.metamx.druid.index.IncrementalIndexSegment; +import com.metamx.druid.index.QueryableIndexSegment; +import com.metamx.druid.index.Segment; +import com.metamx.druid.index.column.ValueType; +import com.metamx.druid.index.v1.TestIndex; +import com.metamx.druid.query.QueryRunner; +import com.metamx.druid.query.QueryRunnerFactory; +import com.metamx.druid.query.QueryRunnerTestHelper; +import com.metamx.druid.query.segment.QuerySegmentSpecs; +import junit.framework.Assert; +import org.junit.Test; + +import java.util.List; +import java.util.Map; + +/** + */ +public class SegmentAnalyzerTest +{ + @Test + public void testIncrementalDoesNotWork() throws Exception + { + final List results = getSegmentAnalysises( + new IncrementalIndexSegment(TestIndex.getIncrementalTestIndex()) + ); + + Assert.assertEquals(0, results.size()); + } + + @Test + public void testMappedWorks() throws Exception + { + final List results = getSegmentAnalysises( + new QueryableIndexSegment("test_1", TestIndex.getMMappedTestIndex()) + ); + + Assert.assertEquals(1, results.size()); + + final SegmentAnalysis analysis = results.get(0); + Assert.assertEquals("test_1", analysis.getId()); + + final Map columns = analysis.getColumns(); + Assert.assertEquals(TestIndex.COLUMNS.length, columns.size()); // All columns including time + + for (String dimension : TestIndex.DIMENSIONS) { + final ColumnAnalysis columnAnalysis = columns.get(dimension.toLowerCase()); + + Assert.assertEquals(dimension, ValueType.STRING.name(), columnAnalysis.getType()); + Assert.assertTrue(dimension, columnAnalysis.getSize() > 0); + Assert.assertTrue(dimension, columnAnalysis.getCardinality() > 0); + } + + for (String metric : TestIndex.METRICS) { + final ColumnAnalysis columnAnalysis = columns.get(metric.toLowerCase()); + + Assert.assertEquals(metric, ValueType.FLOAT.name(), columnAnalysis.getType()); + Assert.assertTrue(metric, columnAnalysis.getSize() > 0); + Assert.assertNull(metric, columnAnalysis.getCardinality()); + } + } + + /** + * *Awesome* method name auto-generated by IntelliJ! I love IntelliJ! + * + * @param index + * @return + */ + private List getSegmentAnalysises(Segment index) + { + final QueryRunner runner = QueryRunnerTestHelper.makeQueryRunner( + (QueryRunnerFactory) new SegmentMetadataQueryRunnerFactory(), index + ); + + final SegmentMetadataQuery query = new SegmentMetadataQuery( + "test", QuerySegmentSpecs.create("2011/2012"), null, null, null + ); + return Sequences.toList(query.run(runner), Lists.newArrayList()); + } +} From 8af3ae09d65ebc20a67249926d1659606d45d787 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Tue, 29 Jan 2013 11:38:35 -0600 Subject: [PATCH 58/92] 1) Fix bug with loading old indexes that might have mixed case column names --- .../src/main/java/com/metamx/druid/index/v1/IndexIO.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index afedbb6a742..2ea63704e0d 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -649,8 +649,12 @@ public class IndexIO } Set colSet = Sets.newTreeSet(); - colSet.addAll(Lists.newArrayList(index.getAvailableDimensions())); - colSet.addAll(Lists.newArrayList(index.getAvailableMetrics())); + for (String dimension : index.getAvailableDimensions()) { + colSet.add(dimension.toLowerCase()); + } + for (String metric : index.getAvailableMetrics()) { + colSet.add(metric.toLowerCase()); + } String[] cols = colSet.toArray(new String[colSet.size()]); From da914b835ef9407cb703744b883751ff4d1f0bba Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Tue, 29 Jan 2013 13:04:57 -0600 Subject: [PATCH 59/92] 1) Add caching for SegmentMetadataQuery --- .../druid/client/CachingClusteredClient.java | 7 ++- .../com/metamx/druid/query/CacheStrategy.java | 9 ++- .../metamx/druid/query/QueryToolChest.java | 2 +- .../group/GroupByQueryQueryToolChest.java | 2 +- .../query/metadata/AllColumnIncluderator.java | 6 ++ .../query/metadata/ColumnIncluderator.java | 5 ++ .../metadata/ListColumnIncluderator.java | 26 ++++++++ .../metadata/NoneColumnIncluderator.java | 6 ++ .../SegmentMetadataQueryQueryToolChest.java | 59 +++++++++++++++++-- .../search/SearchQueryQueryToolChest.java | 14 ++++- .../TimeBoundaryQueryQueryToolChest.java | 13 +++- .../TimeseriesQueryQueryToolChest.java | 15 +++-- .../druid/coordination/ServerManagerTest.java | 2 +- 13 files changed, 145 insertions(+), 21 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index 5d640dc4e38..163f1986a53 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -54,6 +54,7 @@ import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.result.BySegmentResultValueClass; import com.metamx.druid.result.Result; import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.type.TypeReference; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -110,7 +111,7 @@ public class CachingClusteredClient implements QueryRunner public Sequence run(final Query query) { final QueryToolChest> toolChest = warehouse.getToolChest(query); - final CacheStrategy> strategy = toolChest.getCacheStrategy(query); + final CacheStrategy> strategy = toolChest.getCacheStrategy(query); final Map> serverSegments = Maps.newTreeMap(); @@ -241,6 +242,7 @@ public class CachingClusteredClient implements QueryRunner } final Function pullFromCacheFunction = strategy.pullFromCache(); + final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); for (Pair cachedResultPair : cachedResults) { final byte[] cachedResult = cachedResultPair.rhs; Sequence cachedSequence = new BaseSequence>( @@ -255,7 +257,8 @@ public class CachingClusteredClient implements QueryRunner } return objectMapper.readValues( - objectMapper.getJsonFactory().createJsonParser(cachedResult), Object.class + objectMapper.getJsonFactory().createJsonParser(cachedResult), + cacheObjectClazz ); } catch (IOException e) { diff --git a/client/src/main/java/com/metamx/druid/query/CacheStrategy.java b/client/src/main/java/com/metamx/druid/query/CacheStrategy.java index abdbe4da259..f8f5098f6ca 100644 --- a/client/src/main/java/com/metamx/druid/query/CacheStrategy.java +++ b/client/src/main/java/com/metamx/druid/query/CacheStrategy.java @@ -22,16 +22,19 @@ package com.metamx.druid.query; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; +import org.codehaus.jackson.type.TypeReference; /** */ -public interface CacheStrategy> +public interface CacheStrategy> { public byte[] computeCacheKey(QueryType query); - public Function prepareForCache(); + public TypeReference getCacheObjectClazz(); - public Function pullFromCache(); + public Function prepareForCache(); + + public Function pullFromCache(); public Sequence mergeSequences(Sequence> seqOfSequences); } diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index ebf77f64af4..bec2170ec92 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -44,7 +44,7 @@ public interface QueryToolChest> public ServiceMetricEvent.Builder makeMetricBuilder(QueryType query); public Function makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); public TypeReference getResultTypeReference(); - public CacheStrategy getCacheStrategy(QueryType query); + public CacheStrategy getCacheStrategy(QueryType query); public QueryRunner preMergeQueryDecoration(QueryRunner runner); public QueryRunner postMergeQueryDecoration(QueryRunner runner); } diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index 6ce071f8068..9dcf6110322 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -178,7 +178,7 @@ public class GroupByQueryQueryToolChest implements QueryToolChest getCacheStrategy(GroupByQuery query) + public CacheStrategy getCacheStrategy(GroupByQuery query) { return null; } diff --git a/client/src/main/java/com/metamx/druid/query/metadata/AllColumnIncluderator.java b/client/src/main/java/com/metamx/druid/query/metadata/AllColumnIncluderator.java index 8687f213503..cd96b5d718a 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/AllColumnIncluderator.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/AllColumnIncluderator.java @@ -28,4 +28,10 @@ public class AllColumnIncluderator implements ColumnIncluderator { return true; } + + @Override + public byte[] getCacheKey() + { + return ALL_CACHE_PREFIX; + } } diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java b/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java index 466167d48fd..90533c4eaca 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java @@ -32,5 +32,10 @@ import org.codehaus.jackson.annotate.JsonTypeInfo; }) public interface ColumnIncluderator { + public static final byte[] NONE_CACHE_PREFIX = new byte[]{0x0}; + public static final byte[] ALL_CACHE_PREFIX = new byte[]{0x1}; + public static final byte[] LIST_CACHE_PREFIX = new byte[]{0x2}; + public boolean include(String columnName); + public byte[] getCacheKey(); } diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java b/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java index 4048dcd420a..e74661d6822 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java @@ -19,10 +19,16 @@ package com.metamx.druid.query.metadata; +import com.google.common.base.Charsets; +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; import com.google.common.collect.Sets; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Collections; import java.util.List; import java.util.Set; @@ -53,4 +59,24 @@ public class ListColumnIncluderator implements ColumnIncluderator { return columns.contains(columnName); } + + @Override + public byte[] getCacheKey() + { + int size = 1; + List columns = Lists.newArrayListWithExpectedSize(this.columns.size()); + + for (String column : this.columns) { + final byte[] bytes = column.getBytes(Charsets.UTF_8); + columns.add(bytes); + size += bytes.length; + } + + final ByteBuffer bytes = ByteBuffer.allocate(size).put(LIST_CACHE_PREFIX); + for (byte[] column : columns) { + bytes.put(column); + } + + return bytes.array(); + } } diff --git a/client/src/main/java/com/metamx/druid/query/metadata/NoneColumnIncluderator.java b/client/src/main/java/com/metamx/druid/query/metadata/NoneColumnIncluderator.java index 56e6842b7ad..d1d66d26778 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/NoneColumnIncluderator.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/NoneColumnIncluderator.java @@ -28,4 +28,10 @@ public class NoneColumnIncluderator implements ColumnIncluderator { return false; } + + @Override + public byte[] getCacheKey() + { + return NONE_CACHE_PREFIX; + } } diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 447bba2e5c6..160c23cd958 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -27,7 +27,7 @@ import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.metamx.common.ISE; -import com.metamx.common.guava.Comparators; +import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.nary.BinaryFn; import com.metamx.druid.Query; @@ -44,8 +44,7 @@ import org.joda.time.Interval; import org.joda.time.Minutes; import javax.annotation.Nullable; -import java.util.Comparator; -import java.util.Iterator; +import java.nio.ByteBuffer; import java.util.List; import java.util.Map; import java.util.Set; @@ -54,6 +53,7 @@ import java.util.Set; public class SegmentMetadataQueryQueryToolChest implements QueryToolChest { private static final TypeReference TYPE_REFERENCE = new TypeReference(){}; + private static final byte[] SEGMENT_METADATA_CACHE_PREFIX = new byte[]{0x4}; @Override public QueryRunner mergeResults(final QueryRunner runner) @@ -165,9 +165,58 @@ public class SegmentMetadataQueryQueryToolChest implements QueryToolChest getCacheStrategy(SegmentMetadataQuery query) + public CacheStrategy getCacheStrategy(SegmentMetadataQuery query) { - return null; + return new CacheStrategy() + { + @Override + public byte[] computeCacheKey(SegmentMetadataQuery query) + { + byte[] includerBytes = query.getToInclude().getCacheKey(); + return ByteBuffer.allocate(1 + includerBytes.length) + .put(SEGMENT_METADATA_CACHE_PREFIX) + .put(includerBytes) + .array(); + } + + @Override + public TypeReference getCacheObjectClazz() + { + return getResultTypeReference(); + } + + @Override + public Function prepareForCache() + { + return new Function() + { + @Override + public SegmentAnalysis apply(@Nullable SegmentAnalysis input) + { + return input; + } + }; + } + + @Override + public Function pullFromCache() + { + return new Function() + { + @Override + public SegmentAnalysis apply(@Nullable SegmentAnalysis input) + { + return input; + } + }; + } + + @Override + public Sequence mergeSequences(Sequence> seqOfSequences) + { + return new MergeSequence(getOrdering(), seqOfSequences); + } + }; } @Override diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java index 95757fc60b1..ce3fcc86114 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java @@ -82,6 +82,10 @@ public class SearchQueryQueryToolChest implements QueryToolChest OBJECT_TYPE_REFERENCE = new TypeReference() + { + }; + @Override public QueryRunner> mergeResults(QueryRunner> runner) { @@ -143,9 +147,9 @@ public class SearchQueryQueryToolChest implements QueryToolChest, SearchQuery> getCacheStrategy(SearchQuery query) + public CacheStrategy, Object, SearchQuery> getCacheStrategy(SearchQuery query) { - return new CacheStrategy, SearchQuery>() + return new CacheStrategy, Object, SearchQuery>() { @Override public byte[] computeCacheKey(SearchQuery query) @@ -183,6 +187,12 @@ public class SearchQueryQueryToolChest implements QueryToolChest getCacheObjectClazz() + { + return OBJECT_TYPE_REFERENCE; + } + @Override public Function, Object> prepareForCache() { diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 9d65ab9b47c..5ee6c321bbb 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -53,6 +53,9 @@ public class TimeBoundaryQueryQueryToolChest private static final TypeReference> TYPE_REFERENCE = new TypeReference>() { }; + private static final TypeReference OBJECT_TYPE_REFERENCE = new TypeReference() + { + }; @Override public QueryRunner> mergeResults( @@ -106,9 +109,9 @@ public class TimeBoundaryQueryQueryToolChest } @Override - public CacheStrategy, TimeBoundaryQuery> getCacheStrategy(TimeBoundaryQuery query) + public CacheStrategy, Object, TimeBoundaryQuery> getCacheStrategy(TimeBoundaryQuery query) { - return new CacheStrategy, TimeBoundaryQuery>() + return new CacheStrategy, Object, TimeBoundaryQuery>() { @Override public byte[] computeCacheKey(TimeBoundaryQuery query) @@ -119,6 +122,12 @@ public class TimeBoundaryQueryQueryToolChest .array(); } + @Override + public TypeReference getCacheObjectClazz() + { + return OBJECT_TYPE_REFERENCE; + } + @Override public Function, Object> prepareForCache() { diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index d8ef9802dad..9c633507ec5 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -68,6 +68,9 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest> TYPE_REFERENCE = new TypeReference>() { }; + private static final TypeReference OBJECT_TYPE_REFERENCE = new TypeReference() + { + }; @Override public QueryRunner> mergeResults(QueryRunner> queryRunner) @@ -155,9 +158,9 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest, TimeseriesQuery> getCacheStrategy(final TimeseriesQuery query) + public CacheStrategy, Object, TimeseriesQuery> getCacheStrategy(final TimeseriesQuery query) { - return new CacheStrategy, TimeseriesQuery>() + return new CacheStrategy, Object, TimeseriesQuery>() { private final List aggs = query.getAggregatorSpecs(); private final List postAggs = query.getPostAggregatorSpecs(); @@ -179,6 +182,12 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest getCacheObjectClazz() + { + return OBJECT_TYPE_REFERENCE; + } + @Override public Function, Object> prepareForCache() { @@ -259,6 +268,4 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest getCacheStrategy(QueryType query) + public CacheStrategy getCacheStrategy(QueryType query) { return null; } From 61569f62be5cbe798c2411bbc35e45f08e962e4b Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 29 Jan 2013 12:53:52 -0800 Subject: [PATCH 60/92] TimeseriesQueryRunner tests with small incremental indexes --- .../TimeseriesQueryRunnerBonusTest.java | 105 ++++++++++++++++++ 1 file changed, 105 insertions(+) create mode 100644 server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java diff --git a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java new file mode 100644 index 00000000000..91865f47c7d --- /dev/null +++ b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java @@ -0,0 +1,105 @@ +package com.metamx.druid.query.timeseries; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.metamx.common.guava.Sequences; +import com.metamx.druid.Druids; +import com.metamx.druid.Query; +import com.metamx.druid.QueryGranularity; +import com.metamx.druid.aggregation.AggregatorFactory; +import com.metamx.druid.aggregation.CountAggregatorFactory; +import com.metamx.druid.index.IncrementalIndexSegment; +import com.metamx.druid.index.Segment; +import com.metamx.druid.index.v1.IncrementalIndex; +import com.metamx.druid.input.MapBasedInputRow; +import com.metamx.druid.query.FinalizeResultsQueryRunner; +import com.metamx.druid.query.QueryRunner; +import com.metamx.druid.query.QueryRunnerFactory; +import com.metamx.druid.result.Result; +import com.metamx.druid.result.TimeseriesResultValue; +import junit.framework.Assert; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Test; + +import java.util.List; + +public class TimeseriesQueryRunnerBonusTest +{ + @Test + public void testOneRowAtATime() throws Exception + { + final IncrementalIndex oneRowIndex = new IncrementalIndex( + new DateTime("2012-01-01T00:00:00Z").getMillis(), QueryGranularity.NONE, new AggregatorFactory[]{} + ); + + List> results; + + oneRowIndex.add( + new MapBasedInputRow( + new DateTime("2012-01-01T00:00:00Z").getMillis(), + ImmutableList.of("dim1"), + ImmutableMap.of("dim1", "x") + ) + ); + + results = runTimeseriesCount(oneRowIndex); + + Assert.assertEquals("index size", 1, oneRowIndex.size()); + Assert.assertEquals("result size", 1, results.size()); + Assert.assertEquals("result timestamp", new DateTime("2012-01-01T00:00:00Z"), results.get(0).getTimestamp()); + Assert.assertEquals("result count metric", 1, (long) results.get(0).getValue().getLongMetric("rows")); + + oneRowIndex.add( + new MapBasedInputRow( + new DateTime("2012-01-01T00:00:00Z").getMillis(), + ImmutableList.of("dim1"), + ImmutableMap.of("dim1", "y") + ) + ); + + results = runTimeseriesCount(oneRowIndex); + + Assert.assertEquals("index size", 2, oneRowIndex.size()); + Assert.assertEquals("result size", 1, results.size()); + Assert.assertEquals("result timestamp", new DateTime("2012-01-01T00:00:00Z"), results.get(0).getTimestamp()); + Assert.assertEquals("result count metric", 2, (long) results.get(0).getValue().getLongMetric("rows")); + } + + private static List> runTimeseriesCount(IncrementalIndex index) + { + final QueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(); + final QueryRunner> runner = makeQueryRunner( + factory, + new IncrementalIndexSegment(index) + ); + + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("xxx") + .granularity(QueryGranularity.ALL) + .intervals(ImmutableList.of(new Interval("2012-01-01T00:00:00Z/P1D"))) + .aggregators( + ImmutableList.of( + new CountAggregatorFactory("rows") + ) + ) + .build(); + + return Sequences.toList( + runner.run(query), + Lists.>newArrayList() + ); + } + + private static QueryRunner makeQueryRunner( + QueryRunnerFactory> factory, + Segment adapter + ) + { + return new FinalizeResultsQueryRunner( + factory.createRunner(adapter), + factory.getToolchest() + ); + } +} From 8fc10a61a0bde0d00e2251b330f078dee514638f Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Tue, 29 Jan 2013 13:23:57 -0800 Subject: [PATCH 61/92] bug fix for incremental index and 1 row --- .../metamx/druid/index/v1/IncrementalIndexStorageAdapter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java b/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java index ef5ecef5d56..bf8e0ac9653 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java +++ b/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java @@ -215,7 +215,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter while (baseIter.hasNext()) { currEntry.set(baseIter.next()); if (filterMatcher.matches()) { - break; + return; } numAdvanced++; From 049ca2bad4c0805c61711d8a8a279c2a278adca2 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 30 Jan 2013 13:47:08 -0800 Subject: [PATCH 62/92] [maven-release-plugin] prepare release druid-0.2.0 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 9917a339787..697a038e02f 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.0-SNAPSHOT + 0.2.0 diff --git a/common/pom.xml b/common/pom.xml index b4b900311dd..a11c468ad02 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.0-SNAPSHOT + 0.2.0 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index d514c32514d..16760c6e2e8 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.0-SNAPSHOT + 0.2.0 com.metamx druid - 0.2.0-SNAPSHOT + 0.2.0 diff --git a/examples/pom.xml b/examples/pom.xml index 350660500d1..8142d72f900 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.0-SNAPSHOT + 0.2.0 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 0e186d40b83..df91608e659 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.0-SNAPSHOT + 0.2.0 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index f5a8cc28141..3884c3bf1e5 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.0-SNAPSHOT + 0.2.0 diff --git a/index-common/pom.xml b/index-common/pom.xml index a270b9cca22..3b175630d47 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.0-SNAPSHOT + 0.2.0 diff --git a/indexer/pom.xml b/indexer/pom.xml index 433aea2dc13..efc06a3c17a 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.0-SNAPSHOT + 0.2.0 diff --git a/merger/pom.xml b/merger/pom.xml index 64beb8d6b84..74e39857a39 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.0-SNAPSHOT + 0.2.0 diff --git a/pom.xml b/pom.xml index 4310add56c6..7d0b5964086 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.0-SNAPSHOT + 0.2.0 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 5cf54f122d2..ee95ef1c2d5 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.0-SNAPSHOT + 0.2.0 diff --git a/server/pom.xml b/server/pom.xml index 354aecaee60..dbacc4c4d3b 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.0-SNAPSHOT + 0.2.0 From 2a0de2090b32e9d28e5fbcc1a11f7d886f9f87af Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 30 Jan 2013 13:47:15 -0800 Subject: [PATCH 63/92] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 697a038e02f..b2ff808c5cd 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.0 + 0.2.1-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index a11c468ad02..f22c3453c75 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.0 + 0.2.1-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 16760c6e2e8..ba57828d84b 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.0 + 0.2.1-SNAPSHOT com.metamx druid - 0.2.0 + 0.2.1-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 8142d72f900..d1fb86905a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.0 + 0.2.1-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index df91608e659..cf0ca3ef9ed 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.0 + 0.2.1-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 3884c3bf1e5..f682ce722c7 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.0 + 0.2.1-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 3b175630d47..7c6371a3476 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.0 + 0.2.1-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index efc06a3c17a..e3fd6229752 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.0 + 0.2.1-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 74e39857a39..77a17ce4517 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.0 + 0.2.1-SNAPSHOT diff --git a/pom.xml b/pom.xml index 7d0b5964086..0099b42fa8b 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.0 + 0.2.1-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index ee95ef1c2d5..4aa0360e1b5 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.0 + 0.2.1-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index dbacc4c4d3b..bd7cd568cd2 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.0 + 0.2.1-SNAPSHOT From 934207be0d6e47f6d8eafcb75d017c503a7e831f Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 30 Jan 2013 16:26:24 -0800 Subject: [PATCH 64/92] updating sdk version to fix indexer in aws --- indexer/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexer/pom.xml b/indexer/pom.xml index e3fd6229752..db60864de6f 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -51,7 +51,7 @@ com.amazonaws aws-java-sdk - 1.2.15 + 1.3.27 javax.mail From 0b116a8fe4c469e54fcf4c47f3f877ad0c4c6ab9 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 30 Jan 2013 16:28:22 -0800 Subject: [PATCH 65/92] [maven-release-plugin] prepare release druid-0.2.1 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index b2ff808c5cd..884cfc98179 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.1-SNAPSHOT + 0.2.1 diff --git a/common/pom.xml b/common/pom.xml index f22c3453c75..cb5603d61f7 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.1-SNAPSHOT + 0.2.1 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index ba57828d84b..bea1fc7114e 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.1-SNAPSHOT + 0.2.1 com.metamx druid - 0.2.1-SNAPSHOT + 0.2.1 diff --git a/examples/pom.xml b/examples/pom.xml index d1fb86905a7..d523bf3ef71 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.1-SNAPSHOT + 0.2.1 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index cf0ca3ef9ed..f5b3b5eb986 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.1-SNAPSHOT + 0.2.1 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index f682ce722c7..c3af6e562f7 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.1-SNAPSHOT + 0.2.1 diff --git a/index-common/pom.xml b/index-common/pom.xml index 7c6371a3476..a0098a2b25f 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.1-SNAPSHOT + 0.2.1 diff --git a/indexer/pom.xml b/indexer/pom.xml index db60864de6f..6d8a216df1d 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.1-SNAPSHOT + 0.2.1 diff --git a/merger/pom.xml b/merger/pom.xml index 77a17ce4517..a9439e9eb7c 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.1-SNAPSHOT + 0.2.1 diff --git a/pom.xml b/pom.xml index 0099b42fa8b..936e8e8c110 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.1-SNAPSHOT + 0.2.1 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 4aa0360e1b5..9b2c9c66bb5 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.1-SNAPSHOT + 0.2.1 diff --git a/server/pom.xml b/server/pom.xml index bd7cd568cd2..88159d0575a 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.1-SNAPSHOT + 0.2.1 From fa733565e83e8cf23261d3eb2a04b3212853b173 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 30 Jan 2013 16:28:28 -0800 Subject: [PATCH 66/92] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 884cfc98179..60384ef547c 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.1 + 0.2.2-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index cb5603d61f7..54459a2cb7c 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.1 + 0.2.2-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index bea1fc7114e..044e6cab0e0 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.1 + 0.2.2-SNAPSHOT com.metamx druid - 0.2.1 + 0.2.2-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index d523bf3ef71..3585fb84ce2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.1 + 0.2.2-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index f5b3b5eb986..485a2db0b85 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.1 + 0.2.2-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index c3af6e562f7..8affe3f7039 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.1 + 0.2.2-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index a0098a2b25f..c1554ec0ff3 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.1 + 0.2.2-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 6d8a216df1d..9aa3a4ac405 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.1 + 0.2.2-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index a9439e9eb7c..2bb370e9324 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.1 + 0.2.2-SNAPSHOT diff --git a/pom.xml b/pom.xml index 936e8e8c110..ab653a6f9ab 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.1 + 0.2.2-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 9b2c9c66bb5..bc3355d872b 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.1 + 0.2.2-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 88159d0575a..a10f3d13913 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.1 + 0.2.2-SNAPSHOT From 26ed96f05defaa20415e313d179cd718a5527bc5 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 30 Jan 2013 17:06:02 -0800 Subject: [PATCH 67/92] bug fix for autoscaling termination --- .../merger/coordinator/RemoteTaskRunner.java | 29 +++++++++---------- .../config/RemoteTaskRunnerConfig.java | 2 +- 2 files changed, 14 insertions(+), 17 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 5537a6b6420..a95f64cb623 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -177,22 +177,19 @@ public class RemoteTaskRunner implements TaskRunner return; } - List thoseLazyWorkers = Lists.newArrayList( - FunctionalIterable - .create(zkWorkers.values()) - .filter( - new Predicate() - { - @Override - public boolean apply(WorkerWrapper input) - { - return input.getRunningTasks().isEmpty() - && System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() - > config.getMaxWorkerIdleTimeMillisBeforeDeletion(); - } - } - ) - ); + int workerCount = 0; + List thoseLazyWorkers = Lists.newArrayList(); + for (WorkerWrapper workerWrapper : zkWorkers.values()) { + workerCount++; + + if (workerCount > workerSetupManager.getWorkerSetupData().getMinNumWorkers() && + workerWrapper.getRunningTasks().isEmpty() && + System.currentTimeMillis() - workerWrapper.getLastCompletedTaskTime().getMillis() + > config.getMaxWorkerIdleTimeMillisBeforeDeletion() + ) { + thoseLazyWorkers.add(workerWrapper); + } + } AutoScalingData terminated = strategy.terminate( Lists.transform( diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java index c9badf7ef88..44b3a1d4c8c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java @@ -38,7 +38,7 @@ public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig public abstract DateTime getTerminateResourcesOriginDateTime(); @Config("druid.indexer.maxWorkerIdleTimeMillisBeforeDeletion") - @Default("10000") + @Default("600000") public abstract int getMaxWorkerIdleTimeMillisBeforeDeletion(); @Config("druid.indexer.maxScalingDuration") From ef19de8270dbc1b6aaff2ba8ea1207dc727b1006 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 30 Jan 2013 17:09:18 -0800 Subject: [PATCH 68/92] [maven-release-plugin] prepare release druid-0.2.2 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 60384ef547c..c2edbe14811 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.2-SNAPSHOT + 0.2.2 diff --git a/common/pom.xml b/common/pom.xml index 54459a2cb7c..89c0776bcfa 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.2-SNAPSHOT + 0.2.2 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 044e6cab0e0..c5ec5700946 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.2-SNAPSHOT + 0.2.2 com.metamx druid - 0.2.2-SNAPSHOT + 0.2.2 diff --git a/examples/pom.xml b/examples/pom.xml index 3585fb84ce2..63b4fdab6e5 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.2-SNAPSHOT + 0.2.2 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 485a2db0b85..a9fd8c69fd2 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.2-SNAPSHOT + 0.2.2 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 8affe3f7039..36c8dffc501 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.2-SNAPSHOT + 0.2.2 diff --git a/index-common/pom.xml b/index-common/pom.xml index c1554ec0ff3..f10558477b5 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.2-SNAPSHOT + 0.2.2 diff --git a/indexer/pom.xml b/indexer/pom.xml index 9aa3a4ac405..c86509d428b 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.2-SNAPSHOT + 0.2.2 diff --git a/merger/pom.xml b/merger/pom.xml index 2bb370e9324..1332392f034 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.2-SNAPSHOT + 0.2.2 diff --git a/pom.xml b/pom.xml index ab653a6f9ab..e060df7b13c 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.2-SNAPSHOT + 0.2.2 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index bc3355d872b..4ad4d15d035 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.2-SNAPSHOT + 0.2.2 diff --git a/server/pom.xml b/server/pom.xml index a10f3d13913..84fac582cef 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.2-SNAPSHOT + 0.2.2 From 00e6ac1e10c777f6457e82c328ea9138cc8e6754 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 30 Jan 2013 17:09:23 -0800 Subject: [PATCH 69/92] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index c2edbe14811..bf428cf9c31 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.2 + 0.2.3-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 89c0776bcfa..25e9713082c 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.2 + 0.2.3-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index c5ec5700946..0ac7ac24edc 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.2 + 0.2.3-SNAPSHOT com.metamx druid - 0.2.2 + 0.2.3-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 63b4fdab6e5..2041b23753f 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.2 + 0.2.3-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index a9fd8c69fd2..e8972cf8ae3 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.2 + 0.2.3-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 36c8dffc501..16dae001fb9 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.2 + 0.2.3-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index f10558477b5..5ca9cceea5d 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.2 + 0.2.3-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index c86509d428b..fadfa67b486 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.2 + 0.2.3-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 1332392f034..32effdb03e1 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.2 + 0.2.3-SNAPSHOT diff --git a/pom.xml b/pom.xml index e060df7b13c..b29e612de39 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.2 + 0.2.3-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 4ad4d15d035..a4777cb0618 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.2 + 0.2.3-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 84fac582cef..22dbcc83e27 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.2 + 0.2.3-SNAPSHOT From 9179718db5513b4742893360d65e214c0ede42c9 Mon Sep 17 00:00:00 2001 From: xvrl Date: Thu, 31 Jan 2013 14:53:50 -0800 Subject: [PATCH 70/92] hash cache keys to stay under memcached limit of 250 characters --- .../druid/client/cache/MemcachedCache.java | 30 ++++++++++++++----- .../client/cache/MemcachedCacheConfig.java | 3 ++ .../cache/MemcachedCacheBrokerBenchmark.java | 1 + .../cache/MemcachedCacheBrokerTest.java | 2 +- 4 files changed, 28 insertions(+), 8 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java index 86cbf5153e5..d8cd5e5ae8e 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java @@ -20,9 +20,9 @@ package com.metamx.druid.client.cache; import com.google.common.base.Function; +import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Maps; -import net.iharder.base64.Base64; import net.spy.memcached.AddrUtil; import net.spy.memcached.ConnectionFactoryBuilder; import net.spy.memcached.DefaultHashAlgorithm; @@ -31,6 +31,7 @@ import net.spy.memcached.MemcachedClient; import net.spy.memcached.MemcachedClientIF; import net.spy.memcached.internal.BulkFuture; import net.spy.memcached.transcoders.SerializingTranscoder; +import org.apache.commons.codec.digest.DigestUtils; import javax.annotation.Nullable; import java.io.IOException; @@ -62,6 +63,7 @@ public class MemcachedCache implements Cache .build(), AddrUtil.getAddresses(config.getHosts()) ), + config.getMemcachedPrefix(), config.getTimeout(), config.getExpiration() ); @@ -72,6 +74,7 @@ public class MemcachedCache implements Cache private final int timeout; private final int expiration; + private final String memcachedPrefix; private final MemcachedClientIF client; @@ -79,10 +82,15 @@ public class MemcachedCache implements Cache private final AtomicLong missCount = new AtomicLong(0); private final AtomicLong timeoutCount = new AtomicLong(0); - MemcachedCache(MemcachedClientIF client, int timeout, int expiration) { + MemcachedCache(MemcachedClientIF client, String memcachedPrefix, int timeout, int expiration) { + Preconditions.checkArgument(memcachedPrefix.length() <= MAX_PREFIX_LENGTH, + "memcachedPrefix length [%d] exceeds maximum length [%d]", + memcachedPrefix.length(), + MAX_PREFIX_LENGTH); this.timeout = timeout; this.expiration = expiration; this.client = client; + this.memcachedPrefix = memcachedPrefix; } @Override @@ -101,7 +109,7 @@ public class MemcachedCache implements Cache @Override public byte[] get(NamedKey key) { - Future future = client.asyncGet(computeKeyString(key)); + Future future = client.asyncGet(computeKeyString(memcachedPrefix, key)); try { byte[] bytes = (byte[]) future.get(timeout, TimeUnit.MILLISECONDS); if(bytes != null) { @@ -129,7 +137,7 @@ public class MemcachedCache implements Cache @Override public void put(NamedKey key, byte[] value) { - client.set(computeKeyString(key), expiration, value); + client.set(computeKeyString(memcachedPrefix, key), expiration, value); } @Override @@ -144,7 +152,7 @@ public class MemcachedCache implements Cache @Nullable NamedKey input ) { - return computeKeyString(input); + return computeKeyString(memcachedPrefix, input); } } ); @@ -186,7 +194,15 @@ public class MemcachedCache implements Cache // no resources to cleanup } - private static String computeKeyString(NamedKey key) { - return key.namespace + ":" + Base64.encodeBytes(key.key, Base64.DONT_BREAK_LINES); + public static final int MAX_PREFIX_LENGTH = + MemcachedClientIF.MAX_KEY_LENGTH + - 40 // length of namespace hash + - 40 // length of key hash + - 2 // length of separators + ; + + private static String computeKeyString(String memcachedPrefix, NamedKey key) { + // hash keys to keep things under 250 characters for memcached + return memcachedPrefix + ":" + DigestUtils.sha1Hex(key.namespace) + ":" + DigestUtils.sha1Hex(key.key); } } diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java index 83f626d8641..c2a277fb72f 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java @@ -18,4 +18,7 @@ public abstract class MemcachedCacheConfig @Config("${prefix}.maxObjectSize") public abstract int getMaxObjectSize(); + + @Config("${prefix}.memcachedPrefix") + public abstract String getMemcachedPrefix(); } diff --git a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java index d87dfd5f7a1..7827e0779b0 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java @@ -56,6 +56,7 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark cache = new MemcachedCache( client, + "druid-memcached-benchmark", 30000, // 30 seconds 3600 // 1 hour ); diff --git a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java index 87c1dcdd9f4..a87441d5b36 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java @@ -60,7 +60,7 @@ public class MemcachedCacheBrokerTest public void setUp() throws Exception { MemcachedClientIF client = new MockMemcachedClient(); - cache = new MemcachedCache(client, 500, 3600); + cache = new MemcachedCache(client, "druid-memcached-test", 500, 3600); } @Test From d5cf7cfdb3fa2065c9855a994fde1d620a547c06 Mon Sep 17 00:00:00 2001 From: xvrl Date: Thu, 31 Jan 2013 14:55:02 -0800 Subject: [PATCH 71/92] remove references to 'broker' --- .../cache/{MapCacheBrokerTest.java => MapCacheTest.java} | 2 +- ...CacheBrokerBenchmark.java => MemcachedCacheBenchmark.java} | 4 ++-- ...{MemcachedCacheBrokerTest.java => MemcachedCacheTest.java} | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) rename client/src/test/java/com/metamx/druid/client/cache/{MapCacheBrokerTest.java => MapCacheTest.java} (98%) rename client/src/test/java/com/metamx/druid/client/cache/{MemcachedCacheBrokerBenchmark.java => MemcachedCacheBenchmark.java} (96%) rename client/src/test/java/com/metamx/druid/client/cache/{MemcachedCacheBrokerTest.java => MemcachedCacheTest.java} (99%) diff --git a/client/src/test/java/com/metamx/druid/client/cache/MapCacheBrokerTest.java b/client/src/test/java/com/metamx/druid/client/cache/MapCacheTest.java similarity index 98% rename from client/src/test/java/com/metamx/druid/client/cache/MapCacheBrokerTest.java rename to client/src/test/java/com/metamx/druid/client/cache/MapCacheTest.java index 78f071ca539..23a3bd1d641 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MapCacheBrokerTest.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MapCacheTest.java @@ -26,7 +26,7 @@ import org.junit.Test; /** */ -public class MapCacheBrokerTest +public class MapCacheTest { private static final byte[] HI = "hi".getBytes(); private static final byte[] HO = "ho".getBytes(); diff --git a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBenchmark.java similarity index 96% rename from client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java rename to client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBenchmark.java index 7827e0779b0..3a746c9484b 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerBenchmark.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBenchmark.java @@ -17,7 +17,7 @@ import java.util.Map; import java.util.Random; import java.util.concurrent.TimeUnit; -public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark +public class MemcachedCacheBenchmark extends SimpleBenchmark { private static final String BASE_KEY = "test_2012-11-26T00:00:00.000Z_2012-11-27T00:00:00.000Z_2012-11-27T04:11:25.979Z_"; public static final String NAMESPACE = "default"; @@ -114,6 +114,6 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark } public static void main(String[] args) throws Exception { - Runner.main(MemcachedCacheBrokerBenchmark.class, args); + Runner.main(MemcachedCacheBenchmark.class, args); } } diff --git a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheTest.java similarity index 99% rename from client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java rename to client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheTest.java index a87441d5b36..287d208db62 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBrokerTest.java +++ b/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheTest.java @@ -50,7 +50,7 @@ import java.util.concurrent.TimeoutException; /** */ -public class MemcachedCacheBrokerTest +public class MemcachedCacheTest { private static final byte[] HI = "hi".getBytes(); private static final byte[] HO = "ho".getBytes(); From 3aef020fe0529da90d95e27fbc76082021fdab1f Mon Sep 17 00:00:00 2001 From: xvrl Date: Thu, 31 Jan 2013 15:54:52 -0800 Subject: [PATCH 72/92] include actual key in value to detect improbable hash collisions --- .../com/metamx/druid/client/cache/Cache.java | 11 +++++ .../druid/client/cache/MemcachedCache.java | 42 +++++++++++++++---- 2 files changed, 46 insertions(+), 7 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/cache/Cache.java b/client/src/main/java/com/metamx/druid/client/cache/Cache.java index 9bf0cde33e7..6e9463deb56 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/Cache.java +++ b/client/src/main/java/com/metamx/druid/client/cache/Cache.java @@ -19,8 +19,11 @@ package com.metamx.druid.client.cache; +import com.google.common.base.Charsets; import com.google.common.base.Preconditions; +import com.google.common.primitives.Ints; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Map; @@ -48,6 +51,14 @@ public interface Cache this.key = key; } + public byte[] toByteArray() { + final byte[] nsBytes = this.namespace.getBytes(Charsets.UTF_8); + return ByteBuffer.allocate(Ints.BYTES + nsBytes.length + this.key.length) + .putInt(nsBytes.length) + .put(nsBytes) + .put(this.key).array(); + } + @Override public boolean equals(Object o) { diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java index d8cd5e5ae8e..ed7f5292e8a 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java @@ -23,6 +23,7 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Maps; +import com.google.common.primitives.Ints; import net.spy.memcached.AddrUtil; import net.spy.memcached.ConnectionFactoryBuilder; import net.spy.memcached.DefaultHashAlgorithm; @@ -35,6 +36,8 @@ import org.apache.commons.codec.digest.DigestUtils; import javax.annotation.Nullable; import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; @@ -109,7 +112,7 @@ public class MemcachedCache implements Cache @Override public byte[] get(NamedKey key) { - Future future = client.asyncGet(computeKeyString(memcachedPrefix, key)); + Future future = client.asyncGet(computeKeyHash(memcachedPrefix, key)); try { byte[] bytes = (byte[]) future.get(timeout, TimeUnit.MILLISECONDS); if(bytes != null) { @@ -118,7 +121,7 @@ public class MemcachedCache implements Cache else { missCount.incrementAndGet(); } - return bytes; + return bytes == null ? null : deserializeValue(key, bytes); } catch(TimeoutException e) { timeoutCount.incrementAndGet(); @@ -137,7 +140,30 @@ public class MemcachedCache implements Cache @Override public void put(NamedKey key, byte[] value) { - client.set(computeKeyString(memcachedPrefix, key), expiration, value); + client.set(computeKeyHash(memcachedPrefix, key), expiration, serializeValue(key, value)); + } + + private static byte[] serializeValue(NamedKey key, byte[] value) { + byte[] keyBytes = key.toByteArray(); + return ByteBuffer.allocate(Ints.BYTES + keyBytes.length + value.length) + .putInt(keyBytes.length) + .put(keyBytes) + .put(value) + .array(); + } + + private static byte[] deserializeValue(NamedKey key, byte[] bytes) { + ByteBuffer buf = ByteBuffer.wrap(bytes); + + final int keyLength = buf.getInt(); + byte[] keyBytes = new byte[keyLength]; + buf.get(keyBytes); + byte[] value = new byte[buf.remaining()]; + buf.get(value); + + Preconditions.checkState(Arrays.equals(keyBytes, key.toByteArray()), + "Keys do not match, possible hash collision?"); + return value; } @Override @@ -152,7 +178,7 @@ public class MemcachedCache implements Cache @Nullable NamedKey input ) { - return computeKeyString(memcachedPrefix, input); + return computeKeyHash(memcachedPrefix, input); } } ); @@ -171,9 +197,11 @@ public class MemcachedCache implements Cache Map results = Maps.newHashMap(); for(Map.Entry entry : some.entrySet()) { + final NamedKey key = keyLookup.get(entry.getKey()); + final byte[] value = (byte[]) entry.getValue(); results.put( - keyLookup.get(entry.getKey()), - (byte[])entry.getValue() + key, + value == null ? null : deserializeValue(key, value) ); } @@ -201,7 +229,7 @@ public class MemcachedCache implements Cache - 2 // length of separators ; - private static String computeKeyString(String memcachedPrefix, NamedKey key) { + private static String computeKeyHash(String memcachedPrefix, NamedKey key) { // hash keys to keep things under 250 characters for memcached return memcachedPrefix + ":" + DigestUtils.sha1Hex(key.namespace) + ":" + DigestUtils.sha1Hex(key.key); } From 7de5c7806e8c91def566faa8007b676abfa6020c Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Fri, 1 Feb 2013 16:19:17 -0600 Subject: [PATCH 73/92] 1) Some whitespace changes 2) Fix up the alerting to actually include the segment and server for when the MasterBalancer has something stuck in the queue --- .../com/metamx/druid/http/ComputeNode.java | 4 ++- .../com/metamx/druid/master/DruidMaster.java | 29 +++++++++---------- .../druid/master/DruidMasterBalancer.java | 14 +++------ 3 files changed, 21 insertions(+), 26 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/http/ComputeNode.java b/server/src/main/java/com/metamx/druid/http/ComputeNode.java index c559a82d3bc..c6d284403c0 100644 --- a/server/src/main/java/com/metamx/druid/http/ComputeNode.java +++ b/server/src/main/java/com/metamx/druid/http/ComputeNode.java @@ -126,7 +126,9 @@ public class ComputeNode extends BaseServerNode getConfigFactory().buildWithReplacements( ExecutorServiceConfig.class, ImmutableMap.of("base_path", "druid.processing") ) - ), emitter, new ServiceMetricEvent.Builder() + ), + emitter, + new ServiceMetricEvent.Builder() ); final ServerManager serverManager = new ServerManager(segmentLoader, conglomerate, emitter, executorService); diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index 80172c432fa..cf2b07bf62f 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -594,21 +594,20 @@ public class DruidMaster public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) { // Display info about all historical servers - Iterable servers = - FunctionalIterable.create(serverInventoryManager.getInventory()) - .filter( - new Predicate() - { - @Override - public boolean apply( - @Nullable DruidServer input - ) - { - return input.getType() - .equalsIgnoreCase("historical"); - } - } - ); + Iterable servers =FunctionalIterable + .create(serverInventoryManager.getInventory()) + .filter( + new Predicate() + { + @Override + public boolean apply( + @Nullable DruidServer input + ) + { + return input.getType().equalsIgnoreCase("historical"); + } + } + ); if (log.isDebugEnabled()) { log.debug("Servers"); for (DruidServer druidServer : servers) { diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index 75d3534fef6..42e8a4868cb 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -19,8 +19,6 @@ package com.metamx.druid.master; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.collect.Sets; @@ -30,7 +28,6 @@ import com.metamx.druid.client.DruidServer; import com.metamx.emitter.EmittingLogger; import java.util.Comparator; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeSet; @@ -71,13 +68,10 @@ public class DruidMasterBalancer implements DruidMasterHelper for (BalancerSegmentHolder holder : currentlyMovingSegments.get(tier).values()) { holder.reduceLifetime(); if (holder.getLifetime() <= 0) { - log.makeAlert( - "[%s]: Balancer move segments queue has a segment stuck", - tier, - ImmutableMap.builder() - .put("segment", holder.getSegment().getIdentifier()) - .build() - ).emit(); + log.makeAlert("[%s]: Balancer move segments queue has a segment stuck", tier) + .addData("segment", holder.getSegment().getIdentifier()) + .addData("server", holder.getServer()) + .emit(); } } } From 1a9edbebd3ff7d06d86b88e2143556eae52cb793 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Fri, 1 Feb 2013 16:21:26 -0600 Subject: [PATCH 74/92] 1) whitespace --- .../metamx/druid/master/DruidMasterBalancer.java | 16 +++------------- 1 file changed, 3 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index 42e8a4868cb..4ad3a839b4b 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -91,11 +91,7 @@ public class DruidMasterBalancer implements DruidMasterHelper if (!currentlyMovingSegments.get(tier).isEmpty()) { reduceLifetimes(tier); - log.info( - "[%s]: Still waiting on %,d segments to be moved", - tier, - currentlyMovingSegments.size() - ); + log.info("[%s]: Still waiting on %,d segments to be moved", tier, currentlyMovingSegments.size()); continue; } @@ -104,8 +100,7 @@ public class DruidMasterBalancer implements DruidMasterHelper if (serversByPercentUsed.size() <= 1) { log.info( - "[%s]: No unique values found for highest and lowest percent used servers: nothing to balance", - tier + "[%s]: No unique values found for highest and lowest percent used servers: nothing to balance", tier ); continue; } @@ -169,12 +164,7 @@ public class DruidMasterBalancer implements DruidMasterHelper if (!toPeon.getSegmentsToLoad().contains(segmentToMove) && (server.getSegment(segmentName) == null) && new ServerHolder(server, toPeon).getAvailableSize() > segmentToMove.getSize()) { - log.info( - "Moving [%s] from [%s] to [%s]", - segmentName, - fromServer, - toServer - ); + log.info("Moving [%s] from [%s] to [%s]", segmentName, fromServer, toServer); try { master.moveSegment( fromServer, From 31743ed43837f2d9707caadffd1af3a3943a1de8 Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 1 Feb 2013 16:14:44 -0800 Subject: [PATCH 75/92] defaults + max memcached expiration is 30 days --- .../com/metamx/druid/client/cache/MemcachedCacheConfig.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java index c2a277fb72f..2bd7b84e4cc 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java @@ -6,7 +6,7 @@ import org.skife.config.Default; public abstract class MemcachedCacheConfig { @Config("${prefix}.expiration") - @Default("31536000") + @Default("2592000") public abstract int getExpiration(); @Config("${prefix}.timeout") @@ -17,8 +17,10 @@ public abstract class MemcachedCacheConfig public abstract String getHosts(); @Config("${prefix}.maxObjectSize") + @Default("52428800") public abstract int getMaxObjectSize(); @Config("${prefix}.memcachedPrefix") + @Default("druid") public abstract String getMemcachedPrefix(); } From a239f6215870bc277cc1dff95db75fce97a10b72 Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 1 Feb 2013 16:25:35 -0800 Subject: [PATCH 76/92] [maven-release-plugin] prepare release druid-0.2.3 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index bf428cf9c31..0d8deccfa38 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.3-SNAPSHOT + 0.2.3 diff --git a/common/pom.xml b/common/pom.xml index 25e9713082c..7b656fb905d 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.3-SNAPSHOT + 0.2.3 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 0ac7ac24edc..2650d486bf7 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.3-SNAPSHOT + 0.2.3 com.metamx druid - 0.2.3-SNAPSHOT + 0.2.3 diff --git a/examples/pom.xml b/examples/pom.xml index 2041b23753f..16f1abbfb4c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.3-SNAPSHOT + 0.2.3 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index e8972cf8ae3..54b3409add2 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.3-SNAPSHOT + 0.2.3 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 16dae001fb9..c4f202e3908 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.3-SNAPSHOT + 0.2.3 diff --git a/index-common/pom.xml b/index-common/pom.xml index 5ca9cceea5d..910dbff4f85 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.3-SNAPSHOT + 0.2.3 diff --git a/indexer/pom.xml b/indexer/pom.xml index fadfa67b486..544138ec266 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.3-SNAPSHOT + 0.2.3 diff --git a/merger/pom.xml b/merger/pom.xml index 32effdb03e1..1653c349884 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.3-SNAPSHOT + 0.2.3 diff --git a/pom.xml b/pom.xml index b29e612de39..a9db0aff9bc 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.3-SNAPSHOT + 0.2.3 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index a4777cb0618..5fb1562f12a 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.3-SNAPSHOT + 0.2.3 diff --git a/server/pom.xml b/server/pom.xml index 22dbcc83e27..4111e02a345 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.3-SNAPSHOT + 0.2.3 From 7ec3563e7be19bba0517ed922c6f792170ba0a7a Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 1 Feb 2013 16:25:42 -0800 Subject: [PATCH 77/92] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 0d8deccfa38..eab24d41e32 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.3 + 0.2.4-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 7b656fb905d..5675d7b0c37 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.3 + 0.2.4-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 2650d486bf7..1eeb116d4ca 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.3 + 0.2.4-SNAPSHOT com.metamx druid - 0.2.3 + 0.2.4-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 16f1abbfb4c..0e98418b951 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.3 + 0.2.4-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 54b3409add2..97d8740b6bf 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.3 + 0.2.4-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index c4f202e3908..c572c108f4a 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.3 + 0.2.4-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 910dbff4f85..406c62e1fbd 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.3 + 0.2.4-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 544138ec266..d7362ec5dbb 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.3 + 0.2.4-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 1653c349884..39e6e217295 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.3 + 0.2.4-SNAPSHOT diff --git a/pom.xml b/pom.xml index a9db0aff9bc..1fc1976f658 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.3 + 0.2.4-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 5fb1562f12a..61dea22ac6f 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.3 + 0.2.4-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 4111e02a345..3efab6f4cc7 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.3 + 0.2.4-SNAPSHOT From 34b4383934cd828a12ace433a5a70bde1d89786d Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Fri, 1 Feb 2013 18:59:33 -0600 Subject: [PATCH 78/92] 1) Adjust DataSegment to have a binaryVersion field that indicates the expected binary version of the segment files 2) Eliminate race condition from RemoteTaskRunnerTest.testAlreadyExecutedTask() --- .../com/metamx/druid/client/DataSegment.java | 24 ++++++++++++ .../metamx/druid/client/DataSegmentTest.java | 5 ++- .../com/metamx/druid/index/v1/IndexIO.java | 4 +- .../druid/indexer/IndexGeneratorJob.java | 1 + .../common/index/YeOldePlumberSchool.java | 7 ++-- .../druid/merger/common/task/MergeTask.java | 24 ++++++------ .../metamx/druid/merger/common/task/Task.java | 2 +- .../merger/common/task/V8toV9UpgradeTask.java | 39 +++++++++++++++++++ .../merger/common/task/MergeTaskTest.java | 14 ++++--- .../coordinator/RemoteTaskRunnerTest.java | 37 ++++++++++++++++-- .../merger/coordinator/TaskQueueTest.java | 1 + .../java/com/metamx/druid/realtime/Sink.java | 2 + .../metamx/druid/index/v1/IndexMerger.java | 4 +- .../metamx/druid/loading/S3SegmentPusher.java | 26 ++++++------- .../SegmentChangeRequestDropTest.java | 5 ++- .../SegmentChangeRequestLoadTest.java | 5 ++- .../druid/coordination/ServerManagerTest.java | 3 ++ .../druid/coordination/ZkCoordinatorTest.java | 2 + .../master/DruidMasterRuleRunnerTest.java | 2 + 19 files changed, 161 insertions(+), 46 deletions(-) create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java diff --git a/client/src/main/java/com/metamx/druid/client/DataSegment.java b/client/src/main/java/com/metamx/druid/client/DataSegment.java index 1f7fcc42d19..9b29f3507ee 100644 --- a/client/src/main/java/com/metamx/druid/client/DataSegment.java +++ b/client/src/main/java/com/metamx/druid/client/DataSegment.java @@ -48,6 +48,8 @@ import java.util.Map; public class DataSegment implements Comparable { public static String delimiter = "_"; + private final Integer binaryVersion; + public static String makeDataSegmentIdentifier( String dataSource, DateTime start, @@ -89,6 +91,7 @@ public class DataSegment implements Comparable @JsonProperty("dimensions") @JsonDeserialize(using = CommaListJoinDeserializer.class) List dimensions, @JsonProperty("metrics") @JsonDeserialize(using = CommaListJoinDeserializer.class) List metrics, @JsonProperty("shardSpec") ShardSpec shardSpec, + @JsonProperty("binaryVersion") Integer binaryVersion, @JsonProperty("size") long size ) { @@ -112,6 +115,7 @@ public class DataSegment implements Comparable ? ImmutableList.of() : ImmutableList.copyOf(Iterables.filter(metrics, nonEmpty)); this.shardSpec = (shardSpec == null) ? new NoneShardSpec() : shardSpec; + this.binaryVersion = binaryVersion; this.size = size; this.identifier = makeDataSegmentIdentifier( @@ -172,6 +176,12 @@ public class DataSegment implements Comparable return shardSpec; } + @JsonProperty + public Integer getBinaryVersion() + { + return binaryVersion; + } + @JsonProperty public long getSize() { @@ -209,6 +219,11 @@ public class DataSegment implements Comparable return builder(this).version(version).build(); } + public DataSegment withBinaryVersion(int binaryVersion) + { + return builder(this).binaryVersion(binaryVersion).build(); + } + @Override public int compareTo(DataSegment dataSegment) { @@ -287,6 +302,7 @@ public class DataSegment implements Comparable private List dimensions; private List metrics; private ShardSpec shardSpec; + private Integer binaryVersion; private long size; public Builder() @@ -307,6 +323,7 @@ public class DataSegment implements Comparable this.dimensions = segment.getDimensions(); this.metrics = segment.getMetrics(); this.shardSpec = segment.getShardSpec(); + this.binaryVersion = segment.getBinaryVersion(); this.size = segment.getSize(); } @@ -352,6 +369,12 @@ public class DataSegment implements Comparable return this; } + public Builder binaryVersion(Integer binaryVersion) + { + this.binaryVersion = binaryVersion; + return this; + } + public Builder size(long size) { this.size = size; @@ -374,6 +397,7 @@ public class DataSegment implements Comparable dimensions, metrics, shardSpec, + binaryVersion, size ); } diff --git a/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java b/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java index c29d10781e2..24d5986a8f8 100644 --- a/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java +++ b/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.SingleDimensionShardSpec; @@ -60,12 +61,13 @@ public class DataSegmentTest Arrays.asList("dim1", "dim2"), Arrays.asList("met1", "met2"), new NoneShardSpec(), + IndexIO.CURRENT_VERSION_ID, 1 ); final Map objectMap = mapper.readValue(mapper.writeValueAsString(segment), new TypeReference>(){}); - Assert.assertEquals(9, objectMap.size()); + Assert.assertEquals(10, objectMap.size()); Assert.assertEquals("something", objectMap.get("dataSource")); Assert.assertEquals(interval.toString(), objectMap.get("interval")); Assert.assertEquals("1", objectMap.get("version")); @@ -73,6 +75,7 @@ public class DataSegmentTest Assert.assertEquals("dim1,dim2", objectMap.get("dimensions")); Assert.assertEquals("met1,met2", objectMap.get("metrics")); Assert.assertEquals(ImmutableMap.of("type", "none"), objectMap.get("shardSpec")); + Assert.assertEquals(IndexIO.CURRENT_VERSION_ID, objectMap.get("binaryVersion")); Assert.assertEquals(1, objectMap.get("size")); DataSegment deserializedSegment = mapper.readValue(mapper.writeValueAsString(segment), DataSegment.class); diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index 2ea63704e0d..34a8e06962d 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -109,7 +109,7 @@ public class IndexIO private static volatile IndexIOHandler handler = null; - public static final byte CURRENT_VERSION_ID = 0x8; + public static final int CURRENT_VERSION_ID = V9_VERSION; public static Index readIndex(File inDir) throws IOException { @@ -170,7 +170,7 @@ public class IndexIO } } - private static int getVersionFromDir(File inDir) throws IOException + public static int getVersionFromDir(File inDir) throws IOException { File versionFile = new File(inDir, "version.bin"); if (versionFile.exists()) { diff --git a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java index a2ffe8566c8..d8eba264c11 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java @@ -393,6 +393,7 @@ public class IndexGeneratorJob implements Jobby dimensionNames, metricNames, config.getShardSpec(bucket).getActualSpec(), + IndexIO.getVersionFromDir(mergedBase), size ); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java index c5a2bc11826..634fe65ebaf 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java @@ -83,9 +83,7 @@ public class YeOldePlumberSchool implements PlumberSchool final Sink theSink = new Sink(interval, schema); // Temporary directory to hold spilled segments. - final File persistDir = new File( - tmpSegmentDir, theSink.getSegment().withVersion(version).getIdentifier() - ); + final File persistDir = new File(tmpSegmentDir, theSink.getSegment().withVersion(version).getIdentifier()); // Set of spilled segments. Will be merged at the end. final Set spilled = Sets.newHashSet(); @@ -144,7 +142,8 @@ public class YeOldePlumberSchool implements PlumberSchool final DataSegment segmentToUpload = theSink.getSegment() .withDimensions(ImmutableList.copyOf(mappedSegment.getAvailableDimensions())) - .withVersion(version); + .withVersion(version) + .withBinaryVersion(IndexIO.getVersionFromDir(fileToUpload)); segmentPusher.push(fileToUpload, segmentToUpload); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 7388058bad0..20fefa0014f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -127,17 +127,19 @@ public abstract class MergeTask extends AbstractTask final long startTime = System.currentTimeMillis(); log.info( - "Starting merge of id[%s], segments: %s", getId(), Lists.transform( - segments, - new Function() - { - @Override - public String apply(@Nullable DataSegment input) - { - return input.getIdentifier(); - } - } - ) + "Starting merge of id[%s], segments: %s", + getId(), + Lists.transform( + segments, + new Function() + { + @Override + public String apply(@Nullable DataSegment input) + { + return input.getIdentifier(); + } + } + ) ); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index b2059210b58..807172d11ae 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -64,7 +64,7 @@ public interface Task /** * Execute preflight checks for a task. This typically runs on the coordinator, and will be run while - * holding a lock on our dataSouce and interval. If this method throws an exception, the task should be + * holding a lock on our dataSource and interval. If this method throws an exception, the task should be * considered a failure. * * @param context Context for this task, gathered under indexer lock diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java new file mode 100644 index 00000000000..1a718fdaf20 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java @@ -0,0 +1,39 @@ +package com.metamx.druid.merger.common.task; + +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.coordinator.TaskContext; +import org.codehaus.jackson.annotate.JsonProperty; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +/** + */ +public class V8toV9UpgradeTask extends AbstractTask +{ + public V8toV9UpgradeTask( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval + ) + { + super( + String.format("v8tov9_%s_%s_%s", dataSource, interval.toString().replace("/", "_"), new DateTime()), + dataSource, + interval + ); + } + + @Override + public Type getType() + { + throw new UnsupportedOperationException("Do we really need to return a Type?"); + } + + @Override + public TaskStatus run( + TaskContext context, TaskToolbox toolbox + ) throws Exception + { + throw new UnsupportedOperationException(); + } +} diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java index c0c8a88b798..fcecac91d5b 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java @@ -32,12 +32,14 @@ import java.util.Map; public class MergeTaskTest { - final List segments = - ImmutableList - .builder() - .add(new DataSegment("foo", new Interval("2012-01-04/2012-01-06"), "V1", null, null, null, null, -1)) - .add(new DataSegment("foo", new Interval("2012-01-05/2012-01-07"), "V1", null, null, null, null, -1)) - .add(new DataSegment("foo", new Interval("2012-01-03/2012-01-05"), "V1", null, null, null, null, -1)) + private final DataSegment.Builder segmentBuilder = DataSegment.builder() + .dataSource("foo") + .version("V1"); + + final List segments = ImmutableList.builder() + .add(segmentBuilder.interval(new Interval("2012-01-04/2012-01-06")).build()) + .add(segmentBuilder.interval(new Interval("2012-01-05/2012-01-07")).build()) + .add(segmentBuilder.interval(new Interval("2012-01-03/2012-01-05")).build()) .build(); final MergeTask testMergeTask = new MergeTask("foo", segments) diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index 1e31efa121c..b7107ed72d2 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -44,6 +44,7 @@ import org.junit.Test; import java.io.File; import java.util.List; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -68,7 +69,7 @@ public class RemoteTaskRunnerTest private ScheduledExecutorService scheduledExec; - private Task task1; + private TestTask task1; private Worker worker1; @@ -111,6 +112,7 @@ public class RemoteTaskRunnerTest null, null, null, + null, 0 ) ), Lists.newArrayList() @@ -141,13 +143,28 @@ public class RemoteTaskRunnerTest @Test public void testAlreadyExecutedTask() throws Exception { - remoteTaskRunner.run(task1, new TaskContext(new DateTime().toString(), Sets.newHashSet()), null); + final CountDownLatch latch = new CountDownLatch(1); + remoteTaskRunner.run( + new TestTask(task1){ + @Override + public TaskStatus run( + TaskContext context, TaskToolbox toolbox + ) throws Exception + { + latch.await(); + return super.run(context, toolbox); + } + }, + new TaskContext(new DateTime().toString(), Sets.newHashSet()), + null + ); try { remoteTaskRunner.run(task1, new TaskContext(new DateTime().toString(), Sets.newHashSet()), null); + latch.countDown(); fail("ISE expected"); } catch (ISE expected) { - + latch.countDown(); } } @@ -169,6 +186,7 @@ public class RemoteTaskRunnerTest null, null, null, + null, 0 ) ), Lists.newArrayList() @@ -366,7 +384,7 @@ public class RemoteTaskRunnerTest jsonMapper.writeValueAsBytes(worker1) ); while (remoteTaskRunner.getNumWorkers() == 0) { - Thread.sleep(500); + Thread.sleep(50); } } @@ -473,6 +491,9 @@ public class RemoteTaskRunnerTest private static class TestTask extends DefaultMergeTask { private final String id; + private final String dataSource; + private final List segments; + private final List aggregators; public TestTask( @JsonProperty("id") String id, @@ -484,6 +505,14 @@ public class RemoteTaskRunnerTest super(dataSource, segments, aggregators); this.id = id; + this.dataSource = dataSource; + this.segments = segments; + this.aggregators = aggregators; + } + + public TestTask(TestTask task) + { + this(task.id, task.dataSource, task.segments, task.aggregators); } @Override diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java index 47913404eed..1a88f49885e 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java @@ -327,6 +327,7 @@ public class TaskQueueTest null, null, null, + null, -1 ) ) diff --git a/realtime/src/main/java/com/metamx/druid/realtime/Sink.java b/realtime/src/main/java/com/metamx/druid/realtime/Sink.java index 051550e3c41..42acc191b63 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/Sink.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/Sink.java @@ -30,6 +30,7 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IncrementalIndex; +import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.input.InputRow; import org.joda.time.Interval; @@ -134,6 +135,7 @@ public class Sink implements Iterable } }), schema.getShardSpec(), + null, 0 ); } diff --git a/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java b/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java index 57828855b38..3cd9170f9e2 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java +++ b/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java @@ -407,7 +407,7 @@ public class IndexMerger try { fileOutputStream = new FileOutputStream(indexFile); channel = fileOutputStream.getChannel(); - channel.write(ByteBuffer.wrap(new byte[]{IndexIO.CURRENT_VERSION_ID})); + channel.write(ByteBuffer.wrap(new byte[]{IndexIO.V8_VERSION})); GenericIndexed.fromIterable(mergedDimensions, GenericIndexed.stringStrategy).writeToChannel(channel); GenericIndexed.fromIterable(mergedMetrics, GenericIndexed.stringStrategy).writeToChannel(channel); @@ -770,7 +770,7 @@ public class IndexMerger } createIndexDrdFile( - IndexIO.CURRENT_VERSION_ID, + IndexIO.V8_VERSION, v8OutDir, GenericIndexed.fromIterable(mergedDimensions, GenericIndexed.stringStrategy), GenericIndexed.fromIterable(mergedMetrics, GenericIndexed.stringStrategy), diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java index f4099154d2d..4d0c1148593 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java @@ -25,6 +25,7 @@ import com.google.common.io.Closeables; import com.metamx.common.ISE; import com.metamx.common.StreamUtils; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.v1.IndexIO; import com.metamx.emitter.EmittingLogger; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; @@ -63,9 +64,9 @@ public class S3SegmentPusher implements SegmentPusher } @Override - public DataSegment push(File file, DataSegment segment) throws IOException + public DataSegment push(final File indexFilesDir, DataSegment segment) throws IOException { - log.info("Uploading [%s] to S3", file); + log.info("Uploading [%s] to S3", indexFilesDir); String outputKey = JOINER.join( config.getBaseKey().isEmpty() ? null : config.getBaseKey(), segment.getDataSource(), @@ -78,8 +79,6 @@ public class S3SegmentPusher implements SegmentPusher segment.getShardSpec().getPartitionNum() ); - File indexFilesDir = file; - long indexSize = 0; final File zipOutFile = File.createTempFile("druid", "index.zip"); ZipOutputStream zipOut = null; @@ -110,14 +109,15 @@ public class S3SegmentPusher implements SegmentPusher log.info("Pushing %s.", toPush); s3Client.putObject(outputBucket, toPush); - DataSegment outputSegment = segment.withSize(indexSize) - .withLoadSpec( - ImmutableMap.of( - "type", "s3_zip", - "bucket", outputBucket, - "key", toPush.getKey() - ) - ); + segment = segment.withSize(indexSize) + .withLoadSpec( + ImmutableMap.of( + "type", "s3_zip", + "bucket", outputBucket, + "key", toPush.getKey() + ) + ) + .withBinaryVersion(IndexIO.getVersionFromDir(indexFilesDir)); File descriptorFile = File.createTempFile("druid", "descriptor.json"); StreamUtils.copyToFileAndClose(new ByteArrayInputStream(jsonMapper.writeValueAsBytes(segment)), descriptorFile); @@ -137,7 +137,7 @@ public class S3SegmentPusher implements SegmentPusher log.info("Deleting descriptor file[%s]", descriptorFile); descriptorFile.delete(); - return outputSegment; + return segment; } catch (NoSuchAlgorithmException e) { throw new IOException(e); diff --git a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java index 9673d15b3c0..62cb939e0ed 100644 --- a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java @@ -21,6 +21,7 @@ package com.metamx.druid.coordination; import com.google.common.collect.ImmutableMap; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.NoneShardSpec; import org.codehaus.jackson.map.ObjectMapper; @@ -52,6 +53,7 @@ public class SegmentChangeRequestDropTest Arrays.asList("dim1", "dim2"), Arrays.asList("met1", "met2"), new NoneShardSpec(), + IndexIO.CURRENT_VERSION_ID, 1 ); @@ -61,7 +63,7 @@ public class SegmentChangeRequestDropTest mapper.writeValueAsString(segmentDrop), new TypeReference>(){} ); - Assert.assertEquals(10, objectMap.size()); + Assert.assertEquals(11, objectMap.size()); Assert.assertEquals("drop", objectMap.get("action")); Assert.assertEquals("something", objectMap.get("dataSource")); Assert.assertEquals(interval.toString(), objectMap.get("interval")); @@ -70,6 +72,7 @@ public class SegmentChangeRequestDropTest Assert.assertEquals("dim1,dim2", objectMap.get("dimensions")); Assert.assertEquals("met1,met2", objectMap.get("metrics")); Assert.assertEquals(ImmutableMap.of("type", "none"), objectMap.get("shardSpec")); + Assert.assertEquals(IndexIO.CURRENT_VERSION_ID, objectMap.get("binaryVersion")); Assert.assertEquals(1, objectMap.get("size")); } } diff --git a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java index eaedcde0b6a..122b779e922 100644 --- a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java @@ -21,6 +21,7 @@ package com.metamx.druid.coordination; import com.google.common.collect.ImmutableMap; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.NoneShardSpec; import org.codehaus.jackson.map.ObjectMapper; @@ -52,6 +53,7 @@ public class SegmentChangeRequestLoadTest Arrays.asList("dim1", "dim2"), Arrays.asList("met1", "met2"), new NoneShardSpec(), + IndexIO.CURRENT_VERSION_ID, 1 ); @@ -61,7 +63,7 @@ public class SegmentChangeRequestLoadTest mapper.writeValueAsString(segmentDrop), new TypeReference>(){} ); - Assert.assertEquals(10, objectMap.size()); + Assert.assertEquals(11, objectMap.size()); Assert.assertEquals("load", objectMap.get("action")); Assert.assertEquals("something", objectMap.get("dataSource")); Assert.assertEquals(interval.toString(), objectMap.get("interval")); @@ -70,6 +72,7 @@ public class SegmentChangeRequestLoadTest Assert.assertEquals("dim1,dim2", objectMap.get("dimensions")); Assert.assertEquals("met1,met2", objectMap.get("metrics")); Assert.assertEquals(ImmutableMap.of("type", "none"), objectMap.get("shardSpec")); + Assert.assertEquals(IndexIO.CURRENT_VERSION_ID, objectMap.get("binaryVersion")); Assert.assertEquals(1, objectMap.get("size")); } } diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index ed8912d3371..84982d05737 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -39,6 +39,7 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.Segment; import com.metamx.druid.index.brita.Filter; +import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.SegmentIdAttachedStorageAdapter; import com.metamx.druid.index.v1.processing.Cursor; import com.metamx.druid.loading.SegmentLoader; @@ -238,6 +239,7 @@ public class ServerManagerTest Arrays.asList("dim1", "dim2", "dim3"), Arrays.asList("metric1", "metric2"), new NoneShardSpec(), + IndexIO.CURRENT_VERSION_ID, 123l ) ); @@ -259,6 +261,7 @@ public class ServerManagerTest Arrays.asList("dim1", "dim2", "dim3"), Arrays.asList("metric1", "metric2"), new NoneShardSpec(), + IndexIO.CURRENT_VERSION_ID, 123l ) ); diff --git a/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java b/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java index c493193372c..5165647b24f 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java @@ -27,6 +27,7 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; import com.metamx.druid.client.DruidServerConfig; import com.metamx.druid.client.ZKPhoneBook; +import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.NoopSegmentLoader; import com.metamx.druid.metrics.NoopServiceEmitter; @@ -196,6 +197,7 @@ public class ZkCoordinatorTest Arrays.asList("dim1", "dim2", "dim3"), Arrays.asList("metric1", "metric2"), new NoneShardSpec(), + IndexIO.CURRENT_VERSION_ID, 123l ); } diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java index 2ac32578cbf..d1d87fcf075 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java @@ -28,6 +28,7 @@ import com.google.common.collect.Sets; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; import com.metamx.druid.db.DatabaseRuleManager; +import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.master.rules.IntervalDropRule; import com.metamx.druid.master.rules.IntervalLoadRule; import com.metamx.druid.master.rules.Rule; @@ -78,6 +79,7 @@ public class DruidMasterRuleRunnerTest Lists.newArrayList(), Lists.newArrayList(), new NoneShardSpec(), + IndexIO.CURRENT_VERSION_ID, 1 ) ); From f4e843c8dcd42992231554883d6097201f08a12a Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Sun, 3 Feb 2013 11:02:38 -0600 Subject: [PATCH 79/92] 1) Make BasePhoneBook not quite so zealous about throwing exceptions when things didn't line up exactly as it thought they should. --- .../src/main/java/com/metamx/phonebook/StoppedPhoneBook.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/client/src/main/java/com/metamx/phonebook/StoppedPhoneBook.java b/client/src/main/java/com/metamx/phonebook/StoppedPhoneBook.java index 907ee186b06..99abd2964cd 100644 --- a/client/src/main/java/com/metamx/phonebook/StoppedPhoneBook.java +++ b/client/src/main/java/com/metamx/phonebook/StoppedPhoneBook.java @@ -112,7 +112,8 @@ class StoppedPhoneBook implements PhoneBook } if (! serviceAnnouncements.containsKey(nodeName)) { - throw new IAE("Cannot unannounce node[%s] on service[%s]", nodeName, serviceName); + log.warn("Cannot unannounce[%s]: it doesn't exist for service[%s]", nodeName, serviceName); + return; } serviceAnnouncements.remove(nodeName); From 97999c06c43d6fab16d057844a3ac6547bea2be5 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 6 Feb 2013 11:08:44 -0800 Subject: [PATCH 80/92] throttle replicants --- .../com/metamx/druid/master/DruidMaster.java | 2 + .../master/DruidMasterReplicationManager.java | 148 ++++++++++++++++++ .../druid/master/DruidMasterRuleRunner.java | 17 +- .../master/DruidMasterRuntimeParams.java | 22 ++- .../metamx/druid/master/rules/LoadRule.java | 27 +++- 5 files changed, 208 insertions(+), 8 deletions(-) create mode 100644 server/src/main/java/com/metamx/druid/master/DruidMasterReplicationManager.java diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index 80172c432fa..aa4130b4257 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -62,6 +62,7 @@ import java.util.Map; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ScheduledExecutorService; /** @@ -622,6 +623,7 @@ public class DruidMaster // Find all historical servers, group them by subType and sort by ascending usage final DruidCluster cluster = new DruidCluster(); + final Map> currentlyReplicatingSegments; for (DruidServer server : servers) { if (!loadManagementPeons.containsKey(server.getName())) { String basePath = yp.combineParts(Arrays.asList(config.getLoadQueuePath(), server.getName())); diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterReplicationManager.java b/server/src/main/java/com/metamx/druid/master/DruidMasterReplicationManager.java new file mode 100644 index 00000000000..1ee479933b9 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterReplicationManager.java @@ -0,0 +1,148 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.master; + +import com.google.common.collect.Maps; +import com.metamx.emitter.EmittingLogger; + +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentSkipListSet; + +/** + * The DruidMasterReplicationManager is used to throttle the number of replicants that are created and destroyed. + */ +public class DruidMasterReplicationManager +{ + private static final EmittingLogger log = new EmittingLogger(DruidMasterReplicationManager.class); + private final int maxReplicants; + private final int maxLifetime; + + private final Map replicatingLookup = Maps.newHashMap(); + private final Map terminatingLookup = Maps.newHashMap(); + private final ReplicatorSegmentHolder currentlyReplicating = new ReplicatorSegmentHolder(); + private final ReplicatorSegmentHolder currentlyTerminating = new ReplicatorSegmentHolder(); + + public DruidMasterReplicationManager(int maxReplicants, int maxLifetime) + { + this.maxReplicants = maxReplicants; + this.maxLifetime = maxLifetime; + } + + public void updateReplicationState(String tier) + { + update(tier, currentlyReplicating, replicatingLookup, "create"); + } + + public void updateTerminationState(String tier) + { + update(tier, currentlyTerminating, terminatingLookup, "terminate"); + } + + private void update(String tier, ReplicatorSegmentHolder holder, Map lookup, String type) + { + int size = holder.getNumProcessing(tier); + if (size != 0) { + log.info("[%s]: Replicant %s queue still has %d segments", tier, type, size); + holder.reduceLifetime(); + + if (holder.getLifetime() < 0) { + log.makeAlert("[%s]: Replicant %s queue stuck after %d+ runs!", tier, type, maxReplicants).emit(); + } + lookup.put(tier, false); + } + + lookup.put(tier, true); + } + + public boolean canAddReplicant(String tier) + { + return replicatingLookup.get(tier); + } + + public boolean canDestroyReplicant(String tier) + { + return terminatingLookup.get(tier); + } + + public boolean registerReplicantCreation(String tier, String segmentId) + { + return currentlyReplicating.addSegment(tier, segmentId); + } + + public void unregisterReplicantCreation(String tier, String segmentId) + { + currentlyReplicating.removeSegment(tier, segmentId); + } + + public boolean registerReplicantTermination(String tier, String segmentId) + { + return currentlyTerminating.addSegment(tier, segmentId); + } + + public void unregisterReplicantTermination(String tier, String segmentId) + { + currentlyTerminating.removeSegment(tier, segmentId); + } + + private class ReplicatorSegmentHolder + { + private final Map> currentlyProcessingSegments = Maps.newHashMap(); + private volatile int lifetime = maxLifetime; + + public boolean addSegment(String tier, String segmentId) + { + if (currentlyProcessingSegments.size() < maxReplicants) { + Set segments = currentlyProcessingSegments.get(tier); + if (segments == null) { + segments = new ConcurrentSkipListSet(); + } + segments.add(segmentId); + return true; + } + + return false; + } + + public void removeSegment(String tier, String segmentId) + { + Set segments = currentlyProcessingSegments.get(tier); + if (segments != null) { + segments.remove(segmentId); + } + } + + public int getNumProcessing(String tier) + { + Set segments = currentlyProcessingSegments.get(tier); + return (segments == null) ? 0 : currentlyProcessingSegments.size(); + } + + public int getLifetime() + { + return lifetime; + } + + public void reduceLifetime() + { + lifetime--; + } + } +} diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java b/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java index 5c33a257f26..96e181ce9b9 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java @@ -33,6 +33,8 @@ public class DruidMasterRuleRunner implements DruidMasterHelper { private static final EmittingLogger log = new EmittingLogger(DruidMasterRuleRunner.class); + private final DruidMasterReplicationManager replicationManager = new DruidMasterReplicationManager(10, 15); + private final DruidMaster master; public DruidMasterRuleRunner(DruidMaster master) @@ -51,6 +53,15 @@ public class DruidMasterRuleRunner implements DruidMasterHelper return params; } + for (String tier : params.getDruidCluster().getTierNames()) { + replicationManager.updateCreationState(tier); + replicationManager.updateTerminationState(tier); + } + + DruidMasterRuntimeParams paramsWithReplicationManager = params.buildFromExisting() + .withReplicationManager(replicationManager) + .build(); + // Run through all matched rules for available segments DateTime now = new DateTime(); DatabaseRuleManager databaseRuleManager = params.getDatabaseRuleManager(); @@ -76,8 +87,8 @@ public class DruidMasterRuleRunner implements DruidMasterHelper } } - return params.buildFromExisting() - .withMasterStats(stats) - .build(); + return paramsWithReplicationManager.buildFromExisting() + .withMasterStats(stats) + .build(); } } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java b/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java index a8658937e68..1cd6de64ca3 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java @@ -25,13 +25,13 @@ import com.metamx.common.guava.Comparators; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidDataSource; import com.metamx.druid.db.DatabaseRuleManager; -import com.metamx.druid.master.rules.RuleMap; import com.metamx.emitter.service.ServiceEmitter; import java.util.Collection; import java.util.Collections; import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentSkipListSet; /** */ @@ -44,6 +44,7 @@ public class DruidMasterRuntimeParams private final Set dataSources; private final Set availableSegments; private final Map loadManagementPeons; + private final DruidMasterReplicationManager replicationManager; private final ServiceEmitter emitter; private final long millisToWaitBeforeDeleting; private final MasterStats stats; @@ -58,6 +59,7 @@ public class DruidMasterRuntimeParams Set dataSources, Set availableSegments, Map loadManagementPeons, + DruidMasterReplicationManager replicationManager, ServiceEmitter emitter, long millisToWaitBeforeDeleting, MasterStats stats, @@ -72,6 +74,7 @@ public class DruidMasterRuntimeParams this.dataSources = dataSources; this.availableSegments = availableSegments; this.loadManagementPeons = loadManagementPeons; + this.replicationManager = replicationManager; this.emitter = emitter; this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting; this.stats = stats; @@ -114,6 +117,11 @@ public class DruidMasterRuntimeParams return loadManagementPeons; } + public DruidMasterReplicationManager getReplicationManager() + { + return replicationManager; + } + public ServiceEmitter getEmitter() { return emitter; @@ -159,6 +167,7 @@ public class DruidMasterRuntimeParams dataSources, availableSegments, loadManagementPeons, + replicationManager, emitter, millisToWaitBeforeDeleting, stats, @@ -176,6 +185,7 @@ public class DruidMasterRuntimeParams private final Set dataSources; private final Set availableSegments; private final Map loadManagementPeons; + private DruidMasterReplicationManager replicationManager; private ServiceEmitter emitter; private long millisToWaitBeforeDeleting; private MasterStats stats; @@ -191,6 +201,7 @@ public class DruidMasterRuntimeParams this.dataSources = Sets.newHashSet(); this.availableSegments = Sets.newTreeSet(Comparators.inverse(DataSegment.bucketMonthComparator())); this.loadManagementPeons = Maps.newHashMap(); + this.replicationManager = null; this.emitter = null; this.millisToWaitBeforeDeleting = 0; this.stats = new MasterStats(); @@ -206,6 +217,7 @@ public class DruidMasterRuntimeParams Set dataSources, Set availableSegments, Map loadManagementPeons, + DruidMasterReplicationManager replicationManager, ServiceEmitter emitter, long millisToWaitBeforeDeleting, MasterStats stats, @@ -220,6 +232,7 @@ public class DruidMasterRuntimeParams this.dataSources = dataSources; this.availableSegments = availableSegments; this.loadManagementPeons = loadManagementPeons; + this.replicationManager = replicationManager; this.emitter = emitter; this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting; this.stats = stats; @@ -237,6 +250,7 @@ public class DruidMasterRuntimeParams dataSources, availableSegments, loadManagementPeons, + replicationManager, emitter, millisToWaitBeforeDeleting, stats, @@ -287,6 +301,12 @@ public class DruidMasterRuntimeParams return this; } + public Builder withReplicationManager(DruidMasterReplicationManager replicationManager) + { + this.replicationManager = replicationManager; + return this; + } + public Builder withEmitter(ServiceEmitter emitter) { this.emitter = emitter; diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index 5bf232c9474..488019fffeb 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -24,6 +24,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.MinMaxPriorityQueue; import com.metamx.druid.client.DataSegment; import com.metamx.druid.master.DruidMaster; +import com.metamx.druid.master.DruidMasterReplicationManager; import com.metamx.druid.master.DruidMasterRuntimeParams; import com.metamx.druid.master.LoadPeonCallback; import com.metamx.druid.master.MasterStats; @@ -55,17 +56,18 @@ public abstract class LoadRule implements Rule return stats; } - stats.accumulate(assign(expectedReplicants, totalReplicants, serverQueue, segment)); + stats.accumulate(assign(params.getReplicationManager(), expectedReplicants, totalReplicants, serverQueue, segment)); stats.accumulate(drop(expectedReplicants, clusterReplicants, segment, params)); return stats; } private MasterStats assign( + final DruidMasterReplicationManager replicationManager, int expectedReplicants, int totalReplicants, MinMaxPriorityQueue serverQueue, - DataSegment segment + final DataSegment segment ) { MasterStats stats = new MasterStats(); @@ -109,6 +111,13 @@ public abstract class LoadRule implements Rule break; } + if (totalReplicants > 0) { // don't throttle if there's only 1 copy of this segment in the cluster + if (!replicationManager.canAddReplicant(getTier()) || + !replicationManager.registerReplicantCreation(getTier(), segment.getIdentifier())) { + break; + } + } + holder.getPeon().loadSegment( segment, new LoadPeonCallback() @@ -116,6 +125,7 @@ public abstract class LoadRule implements Rule @Override protected void execute() { + replicationManager.unregisterReplicantCreation(getTier(), segment.getIdentifier()); } } ); @@ -132,11 +142,12 @@ public abstract class LoadRule implements Rule private MasterStats drop( int expectedReplicants, int clusterReplicants, - DataSegment segment, - DruidMasterRuntimeParams params + final DataSegment segment, + final DruidMasterRuntimeParams params ) { MasterStats stats = new MasterStats(); + final DruidMasterReplicationManager replicationManager = params.getReplicationManager(); if (!params.hasDeletionWaitTimeElapsed()) { return stats; @@ -168,6 +179,13 @@ public abstract class LoadRule implements Rule break; } + if (actualNumReplicantsForType > 1) { // don't throttle unless we are removing extra replicants + if (!replicationManager.canDestroyReplicant(getTier()) || + !replicationManager.registerReplicantTermination(getTier(), segment.getIdentifier())) { + break; + } + } + if (holder.isServingSegment(segment)) { holder.getPeon().dropSegment( segment, @@ -176,6 +194,7 @@ public abstract class LoadRule implements Rule @Override protected void execute() { + replicationManager.unregisterReplicantTermination(getTier(), segment.getIdentifier()); } } ); From 84baef0462558b6a286eccae01e90749312bc037 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 6 Feb 2013 13:57:56 -0800 Subject: [PATCH 81/92] fix some bugs from last commit --- .../com/metamx/druid/master/DruidMaster.java | 13 +- .../druid/master/DruidMasterConfig.java | 8 + .../druid/master/DruidMasterRuleRunner.java | 15 +- .../master/DruidMasterRuntimeParams.java | 13 +- ...Manager.java => ReplicationThrottler.java} | 62 +++--- .../metamx/druid/master/rules/LoadRule.java | 34 ++-- .../master/DruidMasterRuleRunnerTest.java | 183 +++++++++++++++++- .../metamx/druid/master/DruidMasterTest.java | 13 +- 8 files changed, 279 insertions(+), 62 deletions(-) rename server/src/main/java/com/metamx/druid/master/{DruidMasterReplicationManager.java => ReplicationThrottler.java} (69%) diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index aa4130b4257..724f0058097 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -21,7 +21,6 @@ package com.metamx.druid.master; import com.google.common.base.Function; import com.google.common.base.Predicate; -import com.google.common.collect.Collections2; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -36,7 +35,6 @@ import com.metamx.common.guava.Comparators; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidDataSource; import com.metamx.druid.client.DruidServer; @@ -56,13 +54,11 @@ import org.joda.time.Duration; import javax.annotation.Nullable; import java.util.Arrays; -import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ScheduledExecutorService; /** @@ -565,7 +561,6 @@ public class DruidMaster DruidMasterRuntimeParams.newBuilder() .withStartTime(startTime) .withDatasources(databaseSegmentManager.getInventory()) - .withLoadManagementPeons(loadManagementPeons) .withMillisToWaitBeforeDeleting(config.getMillisToWaitBeforeDeleting()) .withEmitter(emitter) .withMergeBytesLimit(config.getMergeBytesLimit()) @@ -623,7 +618,6 @@ public class DruidMaster // Find all historical servers, group them by subType and sort by ascending usage final DruidCluster cluster = new DruidCluster(); - final Map> currentlyReplicatingSegments; for (DruidServer server : servers) { if (!loadManagementPeons.containsKey(server.getName())) { String basePath = yp.combineParts(Arrays.asList(config.getLoadQueuePath(), server.getName())); @@ -670,11 +664,16 @@ public class DruidMaster return params.buildFromExisting() .withDruidCluster(cluster) .withDatabaseRuleManager(databaseRuleManager) + .withLoadManagementPeons(loadManagementPeons) .withSegmentReplicantLookup(segmentReplicantLookup) .build(); } }, - new DruidMasterRuleRunner(DruidMaster.this), + new DruidMasterRuleRunner( + DruidMaster.this, + config.getReplicantLifetime(), + config.getReplicantThrottleLimit() + ), new DruidMasterCleanup(DruidMaster.this), new DruidMasterBalancer(DruidMaster.this, new BalancerAnalyzer()), new DruidMasterLogger() diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterConfig.java b/server/src/main/java/com/metamx/druid/master/DruidMasterConfig.java index 87df78d4f5e..a382fa382e1 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterConfig.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterConfig.java @@ -80,4 +80,12 @@ public abstract class DruidMasterConfig { return Integer.MAX_VALUE; } + + @Config("druid.master.replicant.lifetime") + @Default("15") + public abstract int getReplicantLifetime(); + + @Config("druid.master.replicant.throttleLimit") + @Default("10") + public abstract int getReplicantThrottleLimit(); } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java b/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java index 96e181ce9b9..e30c5ec31d5 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java @@ -33,13 +33,14 @@ public class DruidMasterRuleRunner implements DruidMasterHelper { private static final EmittingLogger log = new EmittingLogger(DruidMasterRuleRunner.class); - private final DruidMasterReplicationManager replicationManager = new DruidMasterReplicationManager(10, 15); + private final ReplicationThrottler replicationManager; private final DruidMaster master; - public DruidMasterRuleRunner(DruidMaster master) + public DruidMasterRuleRunner(DruidMaster master, int replicantLifeTime, int replicantThrottleLimit) { this.master = master; + this.replicationManager = new ReplicationThrottler(replicantThrottleLimit, replicantLifeTime); } @Override @@ -53,8 +54,8 @@ public class DruidMasterRuleRunner implements DruidMasterHelper return params; } - for (String tier : params.getDruidCluster().getTierNames()) { - replicationManager.updateCreationState(tier); + for (String tier : cluster.getTierNames()) { + replicationManager.updateReplicationState(tier); replicationManager.updateTerminationState(tier); } @@ -64,14 +65,14 @@ public class DruidMasterRuleRunner implements DruidMasterHelper // Run through all matched rules for available segments DateTime now = new DateTime(); - DatabaseRuleManager databaseRuleManager = params.getDatabaseRuleManager(); - for (DataSegment segment : params.getAvailableSegments()) { + DatabaseRuleManager databaseRuleManager = paramsWithReplicationManager.getDatabaseRuleManager(); + for (DataSegment segment : paramsWithReplicationManager.getAvailableSegments()) { List rules = databaseRuleManager.getRulesWithDefault(segment.getDataSource()); boolean foundMatchingRule = false; for (Rule rule : rules) { if (rule.appliesTo(segment, now)) { - stats.accumulate(rule.run(master, params, segment)); + stats.accumulate(rule.run(master, paramsWithReplicationManager, segment)); foundMatchingRule = true; break; } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java b/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java index 1cd6de64ca3..a5164128316 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java @@ -31,7 +31,6 @@ import java.util.Collection; import java.util.Collections; import java.util.Map; import java.util.Set; -import java.util.concurrent.ConcurrentSkipListSet; /** */ @@ -44,7 +43,7 @@ public class DruidMasterRuntimeParams private final Set dataSources; private final Set availableSegments; private final Map loadManagementPeons; - private final DruidMasterReplicationManager replicationManager; + private final ReplicationThrottler replicationManager; private final ServiceEmitter emitter; private final long millisToWaitBeforeDeleting; private final MasterStats stats; @@ -59,7 +58,7 @@ public class DruidMasterRuntimeParams Set dataSources, Set availableSegments, Map loadManagementPeons, - DruidMasterReplicationManager replicationManager, + ReplicationThrottler replicationManager, ServiceEmitter emitter, long millisToWaitBeforeDeleting, MasterStats stats, @@ -117,7 +116,7 @@ public class DruidMasterRuntimeParams return loadManagementPeons; } - public DruidMasterReplicationManager getReplicationManager() + public ReplicationThrottler getReplicationManager() { return replicationManager; } @@ -185,7 +184,7 @@ public class DruidMasterRuntimeParams private final Set dataSources; private final Set availableSegments; private final Map loadManagementPeons; - private DruidMasterReplicationManager replicationManager; + private ReplicationThrottler replicationManager; private ServiceEmitter emitter; private long millisToWaitBeforeDeleting; private MasterStats stats; @@ -217,7 +216,7 @@ public class DruidMasterRuntimeParams Set dataSources, Set availableSegments, Map loadManagementPeons, - DruidMasterReplicationManager replicationManager, + ReplicationThrottler replicationManager, ServiceEmitter emitter, long millisToWaitBeforeDeleting, MasterStats stats, @@ -301,7 +300,7 @@ public class DruidMasterRuntimeParams return this; } - public Builder withReplicationManager(DruidMasterReplicationManager replicationManager) + public Builder withReplicationManager(ReplicationThrottler replicationManager) { this.replicationManager = replicationManager; return this; diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterReplicationManager.java b/server/src/main/java/com/metamx/druid/master/ReplicationThrottler.java similarity index 69% rename from server/src/main/java/com/metamx/druid/master/DruidMasterReplicationManager.java rename to server/src/main/java/com/metamx/druid/master/ReplicationThrottler.java index 1ee479933b9..e899ec919bc 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterReplicationManager.java +++ b/server/src/main/java/com/metamx/druid/master/ReplicationThrottler.java @@ -27,11 +27,11 @@ import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; /** - * The DruidMasterReplicationManager is used to throttle the number of replicants that are created and destroyed. + * The ReplicationThrottler is used to throttle the number of replicants that are created and destroyed. */ -public class DruidMasterReplicationManager +public class ReplicationThrottler { - private static final EmittingLogger log = new EmittingLogger(DruidMasterReplicationManager.class); + private static final EmittingLogger log = new EmittingLogger(ReplicationThrottler.class); private final int maxReplicants; private final int maxLifetime; @@ -40,7 +40,7 @@ public class DruidMasterReplicationManager private final ReplicatorSegmentHolder currentlyReplicating = new ReplicatorSegmentHolder(); private final ReplicatorSegmentHolder currentlyTerminating = new ReplicatorSegmentHolder(); - public DruidMasterReplicationManager(int maxReplicants, int maxLifetime) + public ReplicationThrottler(int maxReplicants, int maxLifetime) { this.maxReplicants = maxReplicants; this.maxLifetime = maxLifetime; @@ -60,16 +60,23 @@ public class DruidMasterReplicationManager { int size = holder.getNumProcessing(tier); if (size != 0) { - log.info("[%s]: Replicant %s queue still has %d segments", tier, type, size); - holder.reduceLifetime(); - - if (holder.getLifetime() < 0) { - log.makeAlert("[%s]: Replicant %s queue stuck after %d+ runs!", tier, type, maxReplicants).emit(); - } + log.info( + "[%s]: Replicant %s queue still has %d segments. Lifetime[%d]", + tier, + type, + size, + holder.getLifetime(tier) + ); + holder.reduceLifetime(tier); lookup.put(tier, false); - } - lookup.put(tier, true); + if (holder.getLifetime(tier) < 0) { + log.makeAlert("[%s]: Replicant %s queue stuck after %d+ runs!", tier, type, maxLifetime).emit(); + } + } else { + log.info("[%s]: Replicant %s queue is empty.", tier, type); + lookup.put(tier, true); + } } public boolean canAddReplicant(String tier) @@ -105,15 +112,16 @@ public class DruidMasterReplicationManager private class ReplicatorSegmentHolder { private final Map> currentlyProcessingSegments = Maps.newHashMap(); - private volatile int lifetime = maxLifetime; + private final Map lifetimes = Maps.newHashMap(); public boolean addSegment(String tier, String segmentId) { - if (currentlyProcessingSegments.size() < maxReplicants) { - Set segments = currentlyProcessingSegments.get(tier); - if (segments == null) { - segments = new ConcurrentSkipListSet(); - } + ConcurrentSkipListSet segments = currentlyProcessingSegments.get(tier); + if (segments == null) { + segments = new ConcurrentSkipListSet(); + currentlyProcessingSegments.put(tier, segments); + } + if (segments.size() < maxReplicants) { segments.add(segmentId); return true; } @@ -132,17 +140,27 @@ public class DruidMasterReplicationManager public int getNumProcessing(String tier) { Set segments = currentlyProcessingSegments.get(tier); - return (segments == null) ? 0 : currentlyProcessingSegments.size(); + return (segments == null) ? 0 : segments.size(); } - public int getLifetime() + public int getLifetime(String tier) { + Integer lifetime = lifetimes.get(tier); + if (lifetime == null) { + lifetime = maxLifetime; + lifetimes.put(tier, lifetime); + } return lifetime; } - public void reduceLifetime() + public void reduceLifetime(String tier) { - lifetime--; + Integer lifetime = lifetimes.get(tier); + if (lifetime == null) { + lifetime = maxLifetime; + lifetimes.put(tier, lifetime); + } + lifetimes.put(tier, --lifetime); } } } diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index 488019fffeb..9ca7fd4c091 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -24,7 +24,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.MinMaxPriorityQueue; import com.metamx.druid.client.DataSegment; import com.metamx.druid.master.DruidMaster; -import com.metamx.druid.master.DruidMasterReplicationManager; +import com.metamx.druid.master.ReplicationThrottler; import com.metamx.druid.master.DruidMasterRuntimeParams; import com.metamx.druid.master.LoadPeonCallback; import com.metamx.druid.master.MasterStats; @@ -63,7 +63,7 @@ public abstract class LoadRule implements Rule } private MasterStats assign( - final DruidMasterReplicationManager replicationManager, + final ReplicationThrottler replicationManager, int expectedReplicants, int totalReplicants, MinMaxPriorityQueue serverQueue, @@ -74,6 +74,13 @@ public abstract class LoadRule implements Rule List assignedServers = Lists.newArrayList(); while (totalReplicants < expectedReplicants) { + if (totalReplicants > 0) { // don't throttle if there's only 1 copy of this segment in the cluster + if (!replicationManager.canAddReplicant(getTier()) || + !replicationManager.registerReplicantCreation(getTier(), segment.getIdentifier())) { + break; + } + } + ServerHolder holder = serverQueue.pollFirst(); if (holder == null) { log.warn( @@ -111,13 +118,6 @@ public abstract class LoadRule implements Rule break; } - if (totalReplicants > 0) { // don't throttle if there's only 1 copy of this segment in the cluster - if (!replicationManager.canAddReplicant(getTier()) || - !replicationManager.registerReplicantCreation(getTier(), segment.getIdentifier())) { - break; - } - } - holder.getPeon().loadSegment( segment, new LoadPeonCallback() @@ -147,7 +147,7 @@ public abstract class LoadRule implements Rule ) { MasterStats stats = new MasterStats(); - final DruidMasterReplicationManager replicationManager = params.getReplicationManager(); + final ReplicationThrottler replicationManager = params.getReplicationManager(); if (!params.hasDeletionWaitTimeElapsed()) { return stats; @@ -173,19 +173,19 @@ public abstract class LoadRule implements Rule List droppedServers = Lists.newArrayList(); while (actualNumReplicantsForType > expectedNumReplicantsForType) { - ServerHolder holder = serverQueue.pollLast(); - if (holder == null) { - log.warn("Wtf, holder was null? I have no servers serving [%s]?", segment.getIdentifier()); - break; - } - - if (actualNumReplicantsForType > 1) { // don't throttle unless we are removing extra replicants + if (expectedNumReplicantsForType > 0) { // don't throttle unless we are removing extra replicants if (!replicationManager.canDestroyReplicant(getTier()) || !replicationManager.registerReplicantTermination(getTier(), segment.getIdentifier())) { break; } } + ServerHolder holder = serverQueue.pollLast(); + if (holder == null) { + log.warn("Wtf, holder was null? I have no servers serving [%s]?", segment.getIdentifier()); + break; + } + if (holder.isServingSegment(segment)) { holder.getPeon().dropSegment( segment, diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java index 2ac32578cbf..b2aeaad0d66 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java @@ -84,7 +84,7 @@ public class DruidMasterRuleRunnerTest start = start.plusHours(1); } - ruleRunner = new DruidMasterRuleRunner(master); + ruleRunner = new DruidMasterRuleRunner(master, 1, 24); } @After @@ -848,4 +848,185 @@ public class DruidMasterRuleRunnerTest EasyMock.verify(mockPeon); EasyMock.verify(anotherMockPeon); } + + /** + * Nodes: + * hot - 2 replicants + * + * @throws Exception + */ + @Test + public void testReplicantThrottle() throws Exception + { + mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + EasyMock.expect(mockPeon.getSegmentsToLoad()).andReturn(Sets.newHashSet()).atLeastOnce(); + EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(0L).atLeastOnce(); + EasyMock.replay(mockPeon); + + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Lists.newArrayList( + new IntervalLoadRule(new Interval("2012-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z"), 2, "hot") + ) + ).atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + DruidCluster druidCluster = new DruidCluster( + ImmutableMap.of( + "hot", + MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( + Arrays.asList( + new ServerHolder( + new DruidServer( + "serverHot", + "hostHot", + 1000, + "historical", + "hot" + ), + mockPeon + ), + new ServerHolder( + new DruidServer( + "serverHot2", + "hostHot2", + 1000, + "historical", + "hot" + ), + mockPeon + ) + ) + ) + ) + ); + + DruidMasterRuntimeParams params = + new DruidMasterRuntimeParams.Builder() + .withDruidCluster(druidCluster) + .withAvailableSegments(availableSegments) + .withDatabaseRuleManager(databaseRuleManager) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) + .build(); + + DruidMasterRuntimeParams afterParams = ruleRunner.run(params); + MasterStats stats = afterParams.getMasterStats(); + + Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("hot").get() == 48); + Assert.assertTrue(stats.getPerTierStats().get("unassignedCount") == null); + Assert.assertTrue(stats.getPerTierStats().get("unassignedSize") == null); + + DataSegment overFlowSegment = new DataSegment( + "test", + new Interval("2012-02-01/2012-02-02"), + new DateTime().toString(), + Maps.newHashMap(), + Lists.newArrayList(), + Lists.newArrayList(), + new NoneShardSpec(), + 1 + ); + + afterParams = ruleRunner.run( + new DruidMasterRuntimeParams.Builder() + .withDruidCluster(druidCluster) + .withEmitter(emitter) + .withAvailableSegments(Arrays.asList(overFlowSegment)) + .withDatabaseRuleManager(databaseRuleManager) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) + .build() + ); + stats = afterParams.getMasterStats(); + + Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("hot").get() == 1); + Assert.assertTrue(stats.getPerTierStats().get("unassignedCount") == null); + Assert.assertTrue(stats.getPerTierStats().get("unassignedSize") == null); + + EasyMock.verify(mockPeon); + } + + @Test + public void testDropReplicantThrottle() throws Exception + { + mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + EasyMock.expect(mockPeon.getSegmentsToLoad()).andReturn(Sets.newHashSet()).atLeastOnce(); + EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(0L).atLeastOnce(); + EasyMock.replay(mockPeon); + + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Lists.newArrayList( + new IntervalLoadRule(new Interval("2012-01-01T00:00:00.000Z/2013-01-02T00:00:00.000Z"), 1, "normal") + ) + ).atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + DataSegment overFlowSegment = new DataSegment( + "test", + new Interval("2012-02-01/2012-02-02"), + new DateTime().toString(), + Maps.newHashMap(), + Lists.newArrayList(), + Lists.newArrayList(), + new NoneShardSpec(), + 1 + ); + List longerAvailableSegments = Lists.newArrayList(availableSegments); + longerAvailableSegments.add(overFlowSegment); + + DruidServer server1 = new DruidServer( + "serverNorm1", + "hostNorm1", + 1000, + "historical", + "normal" + ); + for (DataSegment availableSegment : longerAvailableSegments) { + server1.addDataSegment(availableSegment.getIdentifier(), availableSegment); + } + DruidServer server2 = new DruidServer( + "serverNorm2", + "hostNorm2", + 1000, + "historical", + "normal" + ); + for (DataSegment availableSegment : longerAvailableSegments) { + server2.addDataSegment(availableSegment.getIdentifier(), availableSegment); + } + + DruidCluster druidCluster = new DruidCluster( + ImmutableMap.of( + "normal", + MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( + Arrays.asList( + new ServerHolder( + server1, + mockPeon + ), + new ServerHolder( + server2, + mockPeon + ) + ) + ) + ) + ); + + SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); + + DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder() + .withDruidCluster(druidCluster) + .withMillisToWaitBeforeDeleting(0L) + .withAvailableSegments(longerAvailableSegments) + .withDatabaseRuleManager(databaseRuleManager) + .withSegmentReplicantLookup(segmentReplicantLookup) + .build(); + + DruidMasterRuntimeParams afterParams = ruleRunner.run(params); + MasterStats stats = afterParams.getMasterStats(); + + Assert.assertTrue(stats.getPerTierStats().get("droppedCount").get("normal").get() == 24); + EasyMock.verify(mockPeon); + } } diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java index c54a9a66564..43c7bfc5874 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java @@ -20,7 +20,6 @@ package com.metamx.druid.master; import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; import com.metamx.druid.client.ServerInventoryManager; @@ -124,6 +123,18 @@ public class DruidMasterTest { return ""; } + + @Override + public int getReplicantLifetime() + { + return 0; + } + + @Override + public int getReplicantThrottleLimit() + { + return 0; + } }, null, null, From 7540dd58cad2662684c60f7ebc5d86b4841382c3 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 6 Feb 2013 16:52:24 -0800 Subject: [PATCH 82/92] fix broken UT --- .../com/metamx/druid/master/DruidMasterRuleRunnerTest.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java index 81e8d5ed508..8679daffef4 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java @@ -926,7 +926,8 @@ public class DruidMasterRuleRunnerTest Lists.newArrayList(), Lists.newArrayList(), new NoneShardSpec(), - 1 + 1, + 0 ); afterParams = ruleRunner.run( @@ -971,7 +972,8 @@ public class DruidMasterRuleRunnerTest Lists.newArrayList(), Lists.newArrayList(), new NoneShardSpec(), - 1 + 1, + 0 ); List longerAvailableSegments = Lists.newArrayList(availableSegments); longerAvailableSegments.add(overFlowSegment); From d0fdf4ccf0d9cde0760da94c3e361d4b5985fbe9 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 6 Feb 2013 16:54:16 -0800 Subject: [PATCH 83/92] [maven-release-plugin] prepare release druid-0.2.4 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index eab24d41e32..816a41a8e17 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.4-SNAPSHOT + 0.2.4 diff --git a/common/pom.xml b/common/pom.xml index 5675d7b0c37..330d9ad1a60 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.4-SNAPSHOT + 0.2.4 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 1eeb116d4ca..593314dc17b 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.4-SNAPSHOT + 0.2.4 com.metamx druid - 0.2.4-SNAPSHOT + 0.2.4 diff --git a/examples/pom.xml b/examples/pom.xml index 0e98418b951..e36ccb9082c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.4-SNAPSHOT + 0.2.4 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 97d8740b6bf..82e8026565e 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.4-SNAPSHOT + 0.2.4 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index c572c108f4a..6fc9603b7fc 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.4-SNAPSHOT + 0.2.4 diff --git a/index-common/pom.xml b/index-common/pom.xml index 406c62e1fbd..8f4faa125cc 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.4-SNAPSHOT + 0.2.4 diff --git a/indexer/pom.xml b/indexer/pom.xml index d7362ec5dbb..4800e53c27d 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.4-SNAPSHOT + 0.2.4 diff --git a/merger/pom.xml b/merger/pom.xml index 39e6e217295..1f334fbd65f 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.4-SNAPSHOT + 0.2.4 diff --git a/pom.xml b/pom.xml index 1fc1976f658..15b6a490255 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.4-SNAPSHOT + 0.2.4 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 61dea22ac6f..d86b2d02c5b 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.4-SNAPSHOT + 0.2.4 diff --git a/server/pom.xml b/server/pom.xml index 3efab6f4cc7..1ad0405bdb9 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.4-SNAPSHOT + 0.2.4 From d023d219aeb1345dd9fe1c86284aa1101c52fd70 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 6 Feb 2013 16:54:21 -0800 Subject: [PATCH 84/92] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 816a41a8e17..5669b2358d6 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.4 + 0.2.5-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 330d9ad1a60..d305715a1a6 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.4 + 0.2.5-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 593314dc17b..eed282297c1 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.4 + 0.2.5-SNAPSHOT com.metamx druid - 0.2.4 + 0.2.5-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index e36ccb9082c..f801409d675 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.4 + 0.2.5-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 82e8026565e..43f857b2150 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.4 + 0.2.5-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 6fc9603b7fc..e3b13a88307 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.4 + 0.2.5-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 8f4faa125cc..400b96575fe 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.4 + 0.2.5-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 4800e53c27d..5d6baa4bb46 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.4 + 0.2.5-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 1f334fbd65f..1f70eb48432 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.4 + 0.2.5-SNAPSHOT diff --git a/pom.xml b/pom.xml index 15b6a490255..d2475f3fe74 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.4 + 0.2.5-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index d86b2d02c5b..6b0662267c0 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.4 + 0.2.5-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 1ad0405bdb9..3e85ad1c114 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.4 + 0.2.5-SNAPSHOT From 0d303bee96f7f144e3d5eaf207011a1883fa14fa Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 6 Feb 2013 17:41:08 -0800 Subject: [PATCH 85/92] reset lifetimes for the replication throttle --- .../com/metamx/druid/master/DruidMasterRuleRunner.java | 10 +++++----- .../com/metamx/druid/master/ReplicationThrottler.java | 6 ++++++ 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java b/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java index e30c5ec31d5..2bd4870ff2c 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java @@ -33,14 +33,14 @@ public class DruidMasterRuleRunner implements DruidMasterHelper { private static final EmittingLogger log = new EmittingLogger(DruidMasterRuleRunner.class); - private final ReplicationThrottler replicationManager; + private final ReplicationThrottler replicatorThrottler; private final DruidMaster master; public DruidMasterRuleRunner(DruidMaster master, int replicantLifeTime, int replicantThrottleLimit) { this.master = master; - this.replicationManager = new ReplicationThrottler(replicantThrottleLimit, replicantLifeTime); + this.replicatorThrottler = new ReplicationThrottler(replicantThrottleLimit, replicantLifeTime); } @Override @@ -55,12 +55,12 @@ public class DruidMasterRuleRunner implements DruidMasterHelper } for (String tier : cluster.getTierNames()) { - replicationManager.updateReplicationState(tier); - replicationManager.updateTerminationState(tier); + replicatorThrottler.updateReplicationState(tier); + replicatorThrottler.updateTerminationState(tier); } DruidMasterRuntimeParams paramsWithReplicationManager = params.buildFromExisting() - .withReplicationManager(replicationManager) + .withReplicationManager(replicatorThrottler) .build(); // Run through all matched rules for available segments diff --git a/server/src/main/java/com/metamx/druid/master/ReplicationThrottler.java b/server/src/main/java/com/metamx/druid/master/ReplicationThrottler.java index e899ec919bc..3b3e90f78c2 100644 --- a/server/src/main/java/com/metamx/druid/master/ReplicationThrottler.java +++ b/server/src/main/java/com/metamx/druid/master/ReplicationThrottler.java @@ -76,6 +76,7 @@ public class ReplicationThrottler } else { log.info("[%s]: Replicant %s queue is empty.", tier, type); lookup.put(tier, true); + holder.resetLifetime(tier); } } @@ -162,5 +163,10 @@ public class ReplicationThrottler } lifetimes.put(tier, --lifetime); } + + public void resetLifetime(String tier) + { + lifetimes.put(tier, maxLifetime); + } } } From 668353e89576a79f96dfedf82a92c00126974d7c Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 6 Feb 2013 17:42:33 -0800 Subject: [PATCH 86/92] [maven-release-plugin] prepare release druid-0.2.5 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 5669b2358d6..ff839221686 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.5-SNAPSHOT + 0.2.5 diff --git a/common/pom.xml b/common/pom.xml index d305715a1a6..c18c1044ca2 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.5-SNAPSHOT + 0.2.5 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index eed282297c1..1bd546ae571 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.5-SNAPSHOT + 0.2.5 com.metamx druid - 0.2.5-SNAPSHOT + 0.2.5 diff --git a/examples/pom.xml b/examples/pom.xml index f801409d675..df73fa009a2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.5-SNAPSHOT + 0.2.5 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 43f857b2150..d0e34d6777e 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.5-SNAPSHOT + 0.2.5 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index e3b13a88307..3b96cf2d72a 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.5-SNAPSHOT + 0.2.5 diff --git a/index-common/pom.xml b/index-common/pom.xml index 400b96575fe..a77c9bfc25e 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.5-SNAPSHOT + 0.2.5 diff --git a/indexer/pom.xml b/indexer/pom.xml index 5d6baa4bb46..78a4cd1ce4f 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.5-SNAPSHOT + 0.2.5 diff --git a/merger/pom.xml b/merger/pom.xml index 1f70eb48432..951b719fec8 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.5-SNAPSHOT + 0.2.5 diff --git a/pom.xml b/pom.xml index d2475f3fe74..08176361d11 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.5-SNAPSHOT + 0.2.5 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 6b0662267c0..8dd8cc48057 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.5-SNAPSHOT + 0.2.5 diff --git a/server/pom.xml b/server/pom.xml index 3e85ad1c114..9b97a60ae0f 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.5-SNAPSHOT + 0.2.5 From e09c65835217dbfc9c798aa342a1577a98361811 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 6 Feb 2013 17:42:39 -0800 Subject: [PATCH 87/92] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index ff839221686..215e37b3aaf 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.5 + 0.2.6-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index c18c1044ca2..9d1b304bd3e 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.5 + 0.2.6-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 1bd546ae571..9a1f45ad9ee 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.5 + 0.2.6-SNAPSHOT com.metamx druid - 0.2.5 + 0.2.6-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index df73fa009a2..09916afaaa1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.5 + 0.2.6-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index d0e34d6777e..1de64c1cabb 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.5 + 0.2.6-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 3b96cf2d72a..3b0b26e0ed7 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.5 + 0.2.6-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index a77c9bfc25e..903d6465bac 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.5 + 0.2.6-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 78a4cd1ce4f..cd1885b4113 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.5 + 0.2.6-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 951b719fec8..6aca81fdeb6 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.5 + 0.2.6-SNAPSHOT diff --git a/pom.xml b/pom.xml index 08176361d11..c30ecaa985c 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.5 + 0.2.6-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 8dd8cc48057..a3d06c46861 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.5 + 0.2.6-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 9b97a60ae0f..be844c39716 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.5 + 0.2.6-SNAPSHOT From 267c797023a3a3a61d8f301d0e68dc2ecb136544 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 6 Feb 2013 18:40:29 -0800 Subject: [PATCH 88/92] fix bug in throttle replicants --- .../metamx/druid/master/rules/LoadRule.java | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index 9ca7fd4c091..a7b3b28358c 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -24,10 +24,10 @@ import com.google.common.collect.Lists; import com.google.common.collect.MinMaxPriorityQueue; import com.metamx.druid.client.DataSegment; import com.metamx.druid.master.DruidMaster; -import com.metamx.druid.master.ReplicationThrottler; import com.metamx.druid.master.DruidMasterRuntimeParams; import com.metamx.druid.master.LoadPeonCallback; import com.metamx.druid.master.MasterStats; +import com.metamx.druid.master.ReplicationThrottler; import com.metamx.druid.master.ServerHolder; import com.metamx.emitter.EmittingLogger; @@ -74,13 +74,6 @@ public abstract class LoadRule implements Rule List assignedServers = Lists.newArrayList(); while (totalReplicants < expectedReplicants) { - if (totalReplicants > 0) { // don't throttle if there's only 1 copy of this segment in the cluster - if (!replicationManager.canAddReplicant(getTier()) || - !replicationManager.registerReplicantCreation(getTier(), segment.getIdentifier())) { - break; - } - } - ServerHolder holder = serverQueue.pollFirst(); if (holder == null) { log.warn( @@ -118,6 +111,14 @@ public abstract class LoadRule implements Rule break; } + if (totalReplicants > 0) { // don't throttle if there's only 1 copy of this segment in the cluster + if (!replicationManager.canAddReplicant(getTier()) || + !replicationManager.registerReplicantCreation(getTier(), segment.getIdentifier())) { + serverQueue.add(holder); + break; + } + } + holder.getPeon().loadSegment( segment, new LoadPeonCallback() @@ -173,19 +174,20 @@ public abstract class LoadRule implements Rule List droppedServers = Lists.newArrayList(); while (actualNumReplicantsForType > expectedNumReplicantsForType) { - if (expectedNumReplicantsForType > 0) { // don't throttle unless we are removing extra replicants - if (!replicationManager.canDestroyReplicant(getTier()) || - !replicationManager.registerReplicantTermination(getTier(), segment.getIdentifier())) { - break; - } - } - ServerHolder holder = serverQueue.pollLast(); if (holder == null) { log.warn("Wtf, holder was null? I have no servers serving [%s]?", segment.getIdentifier()); break; } + if (expectedNumReplicantsForType > 0) { // don't throttle unless we are removing extra replicants + if (!replicationManager.canDestroyReplicant(getTier()) || + !replicationManager.registerReplicantTermination(getTier(), segment.getIdentifier())) { + serverQueue.add(holder); + break; + } + } + if (holder.isServingSegment(segment)) { holder.getPeon().dropSegment( segment, From 22d9607169a660c68114d856a7547982e9a505b4 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 6 Feb 2013 18:43:34 -0800 Subject: [PATCH 89/92] [maven-release-plugin] prepare release druid-0.2.6 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 215e37b3aaf..334ae770a01 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.6-SNAPSHOT + 0.2.6 diff --git a/common/pom.xml b/common/pom.xml index 9d1b304bd3e..7f1add3b4a6 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.6-SNAPSHOT + 0.2.6 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 9a1f45ad9ee..4a373b1e996 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.6-SNAPSHOT + 0.2.6 com.metamx druid - 0.2.6-SNAPSHOT + 0.2.6 diff --git a/examples/pom.xml b/examples/pom.xml index 09916afaaa1..d76e8b103e8 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.6-SNAPSHOT + 0.2.6 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 1de64c1cabb..ea94ae82757 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.6-SNAPSHOT + 0.2.6 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 3b0b26e0ed7..cbb5148d292 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.6-SNAPSHOT + 0.2.6 diff --git a/index-common/pom.xml b/index-common/pom.xml index 903d6465bac..8a8a90e8320 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.6-SNAPSHOT + 0.2.6 diff --git a/indexer/pom.xml b/indexer/pom.xml index cd1885b4113..905438c5be8 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.6-SNAPSHOT + 0.2.6 diff --git a/merger/pom.xml b/merger/pom.xml index 6aca81fdeb6..fdf45b6966d 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.6-SNAPSHOT + 0.2.6 diff --git a/pom.xml b/pom.xml index c30ecaa985c..425d7eb23d2 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.6-SNAPSHOT + 0.2.6 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index a3d06c46861..b53c1472eeb 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.6-SNAPSHOT + 0.2.6 diff --git a/server/pom.xml b/server/pom.xml index be844c39716..14a34c5e70b 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.6-SNAPSHOT + 0.2.6 From ec2b906fada9255f779d9d3ebfb13ab9f0ef9129 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 6 Feb 2013 18:43:42 -0800 Subject: [PATCH 90/92] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 334ae770a01..f5025092b8f 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.6 + 0.2.7-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 7f1add3b4a6..6daa2117d7a 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.6 + 0.2.7-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 4a373b1e996..7e7aaf65fa6 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.6 + 0.2.7-SNAPSHOT com.metamx druid - 0.2.6 + 0.2.7-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index d76e8b103e8..ea253aae545 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.6 + 0.2.7-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index ea94ae82757..0ef7456c907 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.6 + 0.2.7-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index cbb5148d292..7c572cd2f32 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.6 + 0.2.7-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 8a8a90e8320..7fa9e7cabd7 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.6 + 0.2.7-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 905438c5be8..13293775b2f 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.6 + 0.2.7-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index fdf45b6966d..80d5f0095c5 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.6 + 0.2.7-SNAPSHOT diff --git a/pom.xml b/pom.xml index 425d7eb23d2..195845004a9 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.6 + 0.2.7-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index b53c1472eeb..d74016fb34b 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.6 + 0.2.7-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 14a34c5e70b..1e33320d0df 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.6 + 0.2.7-SNAPSHOT From 9ac5eeebb3925ca3155f8255a7f9bbd1723c5ddc Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Wed, 6 Feb 2013 23:34:46 -0800 Subject: [PATCH 91/92] 1) Fix NPE in DeterminePartitionsJob when it fails --- .../java/com/metamx/druid/indexer/DeterminePartitionsJob.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index 0e6a686a0f7..13490e3ef00 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -146,7 +146,7 @@ public class DeterminePartitionsJob implements Jobby log.info("Job %s submitted, status available at: %s", groupByJob.getJobName(), groupByJob.getTrackingURL()); if(!groupByJob.waitForCompletion(true)) { - log.error("Job failed: %s", groupByJob.getJobID().toString()); + log.error("Job failed: %s", groupByJob.getJobID()); return false; } } else { From 6ff42f379cfc131151ff615eb7af9b11d079e6f6 Mon Sep 17 00:00:00 2001 From: xvrl Date: Thu, 7 Feb 2013 10:11:36 -0800 Subject: [PATCH 92/92] fix cut-n-paste mistake in error message --- .../query/timeboundary/TimeBoundaryQueryRunnerFactory.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index 4006238786d..6c40e6774d9 100644 --- a/server/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -29,7 +29,6 @@ import com.metamx.druid.query.ChainedExecutionQueryRunner; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.QueryToolChest; -import com.metamx.druid.query.group.GroupByQuery; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; @@ -78,7 +77,7 @@ public class TimeBoundaryQueryRunnerFactory public Sequence> run(Query> input) { if (!(input instanceof TimeBoundaryQuery)) { - throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class); + throw new ISE("Got a [%s] which isn't a %s", input.getClass(), TimeBoundaryQuery.class); } final TimeBoundaryQuery legacyQuery = (TimeBoundaryQuery) input;