From 77c9afe92f601a5e622b88c44dc2f0b258af88c1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 23 Sep 2014 18:53:29 -0700 Subject: [PATCH 01/69] try to make javadoc happy --- .../query/aggregation/histogram/ApproximateHistogram.java | 2 -- .../aggregation/hyperloglog/HyperLogLogCollector.java | 5 ----- .../java/io/druid/query/groupby/having/HavingSpec.java | 2 -- .../segment/data/CompressedFloatsIndexedSupplier.java | 2 -- .../druid/segment/data/CompressedLongsIndexedSupplier.java | 1 - .../io/druid/segment/incremental/IncrementalIndex.java | 7 +++---- .../incremental/IncrementalIndexStorageAdapter.java | 6 ++---- .../java/io/druid/server/coordination/ZkCoordinator.java | 2 +- 8 files changed, 6 insertions(+), 21 deletions(-) diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java index b44fbd524e1..6f481d89161 100644 --- a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java @@ -1019,8 +1019,6 @@ public class ApproximateHistogram * @param count current size of the heap * @param heapIndex index of the item to be deleted * @param values values stored in the heap - * - * @return */ private static int heapDelete(int[] heap, int[] reverseIndex, int count, int heapIndex, float[] values) { diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java index b93c7ace592..7ccbdac2805 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java @@ -200,9 +200,6 @@ public abstract class HyperLogLogCollector implements Comparable /** * For testing. Do not depend on unless you like things breaking. - * - * @return */ GenericIndexed> getBaseFloatBuffers() { diff --git a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java index 8ad267168a5..aa8b9590f54 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java @@ -184,7 +184,6 @@ public class CompressedLongsIndexedSupplier implements Supplier /** * For testing. Do not use unless you like things breaking - * @return */ GenericIndexed> getBaseLongBuffers() { diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 87789e4ceba..da1317f6e4b 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -673,10 +673,9 @@ public class IncrementalIndex implements Iterable falseIdsReverse = biMap.inverse(); } - /** - * Returns the interned String value to allow fast comparisons using `==` instead of `.equals()` - * @see io.druid.segment.incremental.IncrementalIndexStorageAdapter.EntryHolderValueMatcherFactory#makeValueMatcher(String, String) - */ + + // Returns the interned String value to allow fast comparisons using `==` instead of `.equals()` + // see io.druid.segment.incremental.IncrementalIndexStorageAdapter.EntryHolderValueMatcherFactory#makeValueMatcher(String, String) public String get(String value) { return value == null ? null : poorMansInterning.get(value); diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java index 76cf0774861..22d25fa579b 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -532,10 +532,8 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter } for (String dimVal : dims[dimIndex]) { - /** - * using == here instead of .equals() to speed up lookups made possible by - * {@link io.druid.segment.incremental.IncrementalIndex.DimDim#poorMansInterning} - */ + // using == here instead of .equals() to speed up lookups made possible by + // io.druid.segment.incremental.IncrementalIndex.DimDim#poorMansInterning if (id == dimVal) { return true; } diff --git a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java index 34a0ade4574..ba139e8a370 100644 --- a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java @@ -169,7 +169,7 @@ public class ZkCoordinator extends BaseZkCoordinator catch (IOException e) { throw new SegmentLoadingException(e, "Failed to announce segment[%s]", segment.getIdentifier()); } - }; + } } catch (SegmentLoadingException e) { log.makeAlert(e, "Failed to load segment for dataSource") From 1a67d6cb2e678572925c34044dcc2e65e965ccd2 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 1 Oct 2014 15:10:23 -0700 Subject: [PATCH 02/69] some tests for broken approx histo --- ...ApproximateHistogramAggregatorFactory.java | 30 ++ ...mateHistogramFoldingAggregatorFactory.java | 3 +- .../ApproximateHistogramGroupByQueryTest.java | 176 +++++++ ...=> ApproximateHistogramTopNQueryTest.java} | 21 +- .../io/druid/query/groupby/GroupByQuery.java | 18 +- .../query/groupby/GroupByQueryRunnerTest.java | 479 ++++++++---------- .../groupby/GroupByQueryRunnerTestHelper.java | 72 +++ 7 files changed, 528 insertions(+), 271 deletions(-) create mode 100644 histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java rename histogram/src/test/java/io/druid/query/aggregation/histogram/{ApproximateHistogramQueryTest.java => ApproximateHistogramTopNQueryTest.java} (99%) create mode 100644 processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java index 458486423f9..28a707a9598 100644 --- a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java @@ -29,8 +29,14 @@ import com.google.common.primitives.Floats; import com.google.common.primitives.Ints; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.Aggregators; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.hyperloglog.HyperLogLogCollector; +import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregator; +import io.druid.query.aggregation.hyperloglog.HyperUniquesBufferAggregator; import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.FloatColumnSelector; +import io.druid.segment.ObjectColumnSelector; import org.apache.commons.codec.binary.Base64; import java.nio.ByteBuffer; @@ -78,6 +84,18 @@ public class ApproximateHistogramAggregatorFactory implements AggregatorFactory @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { + ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); + + if (selector == null) { + return Aggregators.noopAggregator(); + } + + final Class classOfObject = selector.classOfObject(); + if (classOfObject != null && (classOfObject.equals(Object.class) + || ApproximateHistogramAggregator.class.isAssignableFrom(classOfObject))) { + System.out.println("here"); + //return new HyperUniquesAggregator(name, selector); + } return new ApproximateHistogramAggregator( name, metricFactory.makeFloatColumnSelector(fieldName), @@ -90,6 +108,18 @@ public class ApproximateHistogramAggregatorFactory implements AggregatorFactory @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { + //ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); + // + //if (selector == null) { + // return Aggregators.noopBufferAggregator(); + //} + // + //final Class classOfObject = selector.classOfObject(); + //if (classOfObject != null && (classOfObject.equals(Object.class) || ApproximateHistogramAggregator.class.isAssignableFrom(classOfObject))) { + // System.out.println("here"); + // //return new HyperUniquesAggregator(name, selector); + //} + return new ApproximateHistogramBufferAggregator( metricFactory.makeFloatColumnSelector(fieldName), resolution, diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java index 04dc43a804b..f84b03ae5b8 100644 --- a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java @@ -76,7 +76,8 @@ public class ApproximateHistogramFoldingAggregatorFactory extends ApproximateHis }; } - if (ApproximateHistogram.class.isAssignableFrom(selector.classOfObject())) { + final Class classOfObject = selector.classOfObject(); + if (classOfObject.equals(Object.class) || ApproximateHistogram.class.isAssignableFrom(classOfObject)) { return new ApproximateHistogramFoldingAggregator( name, selector, diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java new file mode 100644 index 00000000000..33ee0b224d9 --- /dev/null +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -0,0 +1,176 @@ +package io.druid.query.aggregation.histogram; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import io.druid.collections.StupidPool; +import io.druid.data.input.Row; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.MaxAggregatorFactory; +import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.dimension.LegacyDimensionSpec; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.groupby.GroupByQueryConfig; +import io.druid.query.groupby.GroupByQueryEngine; +import io.druid.query.groupby.GroupByQueryQueryToolChest; +import io.druid.query.groupby.GroupByQueryRunnerFactory; +import io.druid.query.groupby.GroupByQueryRunnerTestHelper; +import io.druid.segment.TestHelper; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +/** + */ +@RunWith(Parameterized.class) +public class ApproximateHistogramGroupByQueryTest +{ + private final QueryRunner runner; + private GroupByQueryRunnerFactory factory; + private Supplier configSupplier; + + @Before + public void setUp() throws Exception + { + configSupplier = Suppliers.ofInstance(new GroupByQueryConfig()); + } + + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + final ObjectMapper mapper = new DefaultObjectMapper(); + final StupidPool pool = new StupidPool( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(1024 * 1024); + } + } + ); + + final GroupByQueryConfig config = new GroupByQueryConfig(); + config.setMaxIntermediateRows(10000); + + final Supplier configSupplier = Suppliers.ofInstance(config); + final GroupByQueryEngine engine = new GroupByQueryEngine(configSupplier, pool); + + final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( + engine, + QueryRunnerTestHelper.NOOP_QUERYWATCHER, + configSupplier, + new GroupByQueryQueryToolChest(configSupplier, mapper, engine) + ); + + GroupByQueryConfig singleThreadedConfig = new GroupByQueryConfig() + { + @Override + public boolean isSingleThreaded() + { + return true; + } + }; + singleThreadedConfig.setMaxIntermediateRows(10000); + + final Supplier singleThreadedConfigSupplier = Suppliers.ofInstance(singleThreadedConfig); + final GroupByQueryEngine singleThreadEngine = new GroupByQueryEngine(singleThreadedConfigSupplier, pool); + + final GroupByQueryRunnerFactory singleThreadFactory = new GroupByQueryRunnerFactory( + singleThreadEngine, + QueryRunnerTestHelper.NOOP_QUERYWATCHER, + singleThreadedConfigSupplier, + new GroupByQueryQueryToolChest(singleThreadedConfigSupplier, mapper, singleThreadEngine) + ); + + + Function function = new Function() + { + @Override + public Object apply(@Nullable Object input) + { + return new Object[]{factory, ((Object[]) input)[0]}; + } + }; + + return Lists.newArrayList( + Iterables.concat( + Iterables.transform( + QueryRunnerTestHelper.makeQueryRunners(factory), + function + ), + Iterables.transform( + QueryRunnerTestHelper.makeQueryRunners(singleThreadFactory), + function + ) + ) + ); + } + + public ApproximateHistogramGroupByQueryTest(GroupByQueryRunnerFactory factory, QueryRunner runner) + { + this.factory = factory; + this.runner = runner; + } + + @Test + public void testGroupByNWithApproximateHistogramAgg() + { + ApproximateHistogramAggregatorFactory aggFactory = new ApproximateHistogramAggregatorFactory( + "apphisto", + "index", + 10, + 5, + Float.NEGATIVE_INFINITY, + Float.POSITIVE_INFINITY + ); + + GroupByQuery query = new GroupByQuery.Builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setGranularity(QueryRunnerTestHelper.allGran) + .setDimensions(Arrays.asList(new LegacyDimensionSpec(QueryRunnerTestHelper.providerDimension))) + .setInterval(QueryRunnerTestHelper.fullOnInterval) + .setAggregatorSpecs( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index"), + aggFactory + ) + ) + ) + ) + .setPostAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.addRowsIndexConstant, + QueryRunnerTestHelper.dependentPostAgg, + new QuantilePostAggregator("quantile", "apphisto", 0.5f) + ) + ) + .build(); + + List expectedResults = Arrays.asList( + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, "approx-histo"); + } +} diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java similarity index 99% rename from histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramQueryTest.java rename to histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java index dd4406c5076..e362b4ad9b2 100644 --- a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramQueryTest.java +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java @@ -52,18 +52,8 @@ import java.util.List; import java.util.Map; @RunWith(Parameterized.class) -public class ApproximateHistogramQueryTest +public class ApproximateHistogramTopNQueryTest { - - private final QueryRunner runner; - - public ApproximateHistogramQueryTest( - QueryRunner runner - ) - { - this.runner = runner; - } - @Parameterized.Parameters public static Collection constructorFeeder() throws IOException { @@ -99,6 +89,15 @@ public class ApproximateHistogramQueryTest return retVal; } + private final QueryRunner runner; + + public ApproximateHistogramTopNQueryTest( + QueryRunner runner + ) + { + this.runner = runner; + } + @Test public void testTopNWithApproximateHistogramAgg() { diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index a0fab78da07..eedc90946f3 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -50,6 +50,7 @@ import io.druid.query.groupby.orderby.NoopLimitSpec; import io.druid.query.groupby.orderby.OrderByColumnSpec; import io.druid.query.spec.LegacySegmentSpec; import io.druid.query.spec.QuerySegmentSpec; +import org.joda.time.Interval; import java.util.List; import java.util.Map; @@ -344,7 +345,22 @@ public class GroupByQuery extends BaseQuery return this; } - public Builder setInterval(Object interval) + public Builder setInterval(QuerySegmentSpec interval) + { + return setQuerySegmentSpec(interval); + } + + public Builder setInterval(List intervals) + { + return setQuerySegmentSpec(new LegacySegmentSpec(intervals)); + } + + public Builder setInterval(Interval interval) + { + return setQuerySegmentSpec(new LegacySegmentSpec(interval)); + } + + public Builder setInterval(String interval) { return setQuerySegmentSpec(new LegacySegmentSpec(interval)); } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 9eb25885432..6beab76793d 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -21,27 +21,22 @@ package io.druid.query.groupby; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; -import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.collections.StupidPool; -import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.granularity.PeriodGranularity; import io.druid.granularity.QueryGranularity; import io.druid.jackson.DefaultObjectMapper; -import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; -import io.druid.query.QueryToolChest; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.JavaScriptAggregatorFactory; @@ -84,7 +79,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.Comparator; import java.util.List; -import java.util.Map; @RunWith(Parameterized.class) public class GroupByQueryRunnerTest @@ -195,28 +189,28 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), - createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), - createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -237,7 +231,7 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow( + GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", "rows", 26L, @@ -246,7 +240,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -267,7 +261,7 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow( + GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", "rows", 26L, @@ -276,7 +270,7 @@ public class GroupByQueryRunnerTest ) ); - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -306,26 +300,26 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "a", "rows", 1L, "idx", 135L), - createExpectedRow("2011-04-01", "alias", "b", "rows", 1L, "idx", 118L), - createExpectedRow("2011-04-01", "alias", "e", "rows", 1L, "idx", 158L), - createExpectedRow("2011-04-01", "alias", "h", "rows", 1L, "idx", 120L), - createExpectedRow("2011-04-01", "alias", "m", "rows", 3L, "idx", 2870L), - createExpectedRow("2011-04-01", "alias", "n", "rows", 1L, "idx", 121L), - createExpectedRow("2011-04-01", "alias", "p", "rows", 3L, "idx", 2900L), - createExpectedRow("2011-04-01", "alias", "t", "rows", 2L, "idx", 197L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "a", "rows", 1L, "idx", 135L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "b", "rows", 1L, "idx", 118L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "e", "rows", 1L, "idx", 158L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "h", "rows", 1L, "idx", 120L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "m", "rows", 3L, "idx", 2870L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "n", "rows", 1L, "idx", 121L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "p", "rows", 3L, "idx", 2900L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "t", "rows", 2L, "idx", 197L), - createExpectedRow("2011-04-02", "alias", "a", "rows", 1L, "idx", 147L), - createExpectedRow("2011-04-02", "alias", "b", "rows", 1L, "idx", 112L), - createExpectedRow("2011-04-02", "alias", "e", "rows", 1L, "idx", 166L), - createExpectedRow("2011-04-02", "alias", "h", "rows", 1L, "idx", 113L), - createExpectedRow("2011-04-02", "alias", "m", "rows", 3L, "idx", 2447L), - createExpectedRow("2011-04-02", "alias", "n", "rows", 1L, "idx", 114L), - createExpectedRow("2011-04-02", "alias", "p", "rows", 3L, "idx", 2505L), - createExpectedRow("2011-04-02", "alias", "t", "rows", 2L, "idx", 223L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "a", "rows", 1L, "idx", 147L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "b", "rows", 1L, "idx", 112L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "e", "rows", 1L, "idx", 166L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "h", "rows", 1L, "idx", 113L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "m", "rows", 3L, "idx", 2447L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "n", "rows", 1L, "idx", 114L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "p", "rows", 3L, "idx", 2505L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "t", "rows", 2L, "idx", 223L) ); - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -364,28 +358,28 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "automotive", "rows", 1L, "idx", 135L), - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "business", "rows", 1L, "idx", 118L), - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "entertainment", "rows", 1L, "idx", 158L), - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "health", "rows", 1L, "idx", 120L), - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "mezzanine", "rows", 3L, "idx", 2870L), - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "news", "rows", 1L, "idx", 121L), - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "premium", "rows", 3L, "idx", 2900L), - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "technology", "rows", 1L, "idx", 78L), - createExpectedRow(new DateTime("2011-03-31", tz), "alias", "travel", "rows", 1L, "idx", 119L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "automotive", "rows", 1L, "idx", 135L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "business", "rows", 1L, "idx", 118L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "entertainment", "rows", 1L, "idx", 158L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "health", "rows", 1L, "idx", 120L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "mezzanine", "rows", 3L, "idx", 2870L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "news", "rows", 1L, "idx", 121L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "premium", "rows", 3L, "idx", 2900L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "technology", "rows", 1L, "idx", 78L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "travel", "rows", 1L, "idx", 119L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "automotive", "rows", 1L, "idx", 147L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "business", "rows", 1L, "idx", 112L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "entertainment", "rows", 1L, "idx", 166L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "health", "rows", 1L, "idx", 113L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "mezzanine", "rows", 3L, "idx", 2447L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "news", "rows", 1L, "idx", 114L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "premium", "rows", 3L, "idx", 2505L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "technology", "rows", 1L, "idx", 97L), - createExpectedRow(new DateTime("2011-04-01", tz), "alias", "travel", "rows", 1L, "idx", 126L) + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "automotive", "rows", 1L, "idx", 147L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "business", "rows", 1L, "idx", 112L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "entertainment", "rows", 1L, "idx", 166L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "health", "rows", 1L, "idx", 113L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "mezzanine", "rows", 3L, "idx", 2447L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "news", "rows", 1L, "idx", 114L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "premium", "rows", 3L, "idx", 2505L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "technology", "rows", 1L, "idx", 97L), + GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "travel", "rows", 1L, "idx", 126L) ); - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -427,30 +421,30 @@ public class GroupByQueryRunnerTest ); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), - createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), - createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) ); TestHelper.assertExpectedObjects(expectedResults, runner.run(fullQuery), "direct"); TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery), "merged"); List allGranExpectedResults = Arrays.asList( - createExpectedRow("2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L), - createExpectedRow("2011-04-02", "alias", "business", "rows", 2L, "idx", 217L), - createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 2L, "idx", 319L), - createExpectedRow("2011-04-02", "alias", "health", "rows", 2L, "idx", 216L), - createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - createExpectedRow("2011-04-02", "alias", "news", "rows", 2L, "idx", 221L), - createExpectedRow("2011-04-02", "alias", "premium", "rows", 6L, "idx", 4416L), - createExpectedRow("2011-04-02", "alias", "technology", "rows", 2L, "idx", 177L), - createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 2L, "idx", 217L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 2L, "idx", 319L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 2L, "idx", 216L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 2L, "idx", 221L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 6L, "idx", 4416L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 2L, "idx", 177L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) ); TestHelper.assertExpectedObjects(allGranExpectedResults, runner.run(allGranQuery), "direct"); @@ -484,15 +478,15 @@ public class GroupByQueryRunnerTest final GroupByQuery fullQuery = builder.build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), - createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), - createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); @@ -559,15 +553,15 @@ public class GroupByQueryRunnerTest }; List allResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), - createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), - createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) ); List> expectedResults = Lists.newArrayList( @@ -642,15 +636,15 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), - createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), - createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), - createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L) ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); @@ -682,15 +676,15 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), - createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), - createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), - createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 177L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 221L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L) ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); @@ -721,15 +715,15 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4423.6533203125D), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4418.61865234375D), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319.94403076171875D), - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 270.3977966308594D), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243.65843200683594D), - createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 222.20980834960938D), - createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 218.7224884033203D), - createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216.97836303710938D), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 178.24917602539062D) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4423.6533203125D), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4418.61865234375D), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319.94403076171875D), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 270.3977966308594D), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243.65843200683594D), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 222.20980834960938D), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 218.7224884033203D), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216.97836303710938D), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 178.24917602539062D) ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); @@ -743,9 +737,9 @@ public class GroupByQueryRunnerTest public void testHavingSpec() { List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L) ); GroupByQuery.Builder builder = GroupByQuery @@ -811,7 +805,7 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L) ); final GroupByQueryEngine engine = new GroupByQueryEngine( @@ -855,15 +849,15 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "business", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "entertainment", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "health", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "mezzanine", "rows", 6L), - createExpectedRow("2011-04-01", "quality", "news", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "premium", "rows", 6L), - createExpectedRow("2011-04-01", "quality", "technology", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "business", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "entertainment", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "health", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "mezzanine", "rows", 6L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "news", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "premium", "rows", 6L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "technology", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L) ); TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); @@ -908,15 +902,15 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "business", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "entertainment", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "health", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "mezzanine", "rows", 6L), - createExpectedRow("2011-04-01", "quality", "news", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "premium", "rows", 6L), - createExpectedRow("2011-04-01", "quality", "technology", "rows", 2L), - createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "business", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "entertainment", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "health", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "mezzanine", "rows", 6L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "news", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "premium", "rows", 6L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "technology", "rows", 2L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L) ); TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); @@ -976,29 +970,29 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), - createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), - createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); // Subqueries are handled by the ToolChest - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -1032,11 +1026,11 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "idx", 2900.0), - createExpectedRow("2011-04-02", "idx", 2505.0) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "idx", 2900.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "idx", 2505.0) ); - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -1070,10 +1064,10 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-02", "idx", 2505.0) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "idx", 2505.0) ); - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -1106,7 +1100,7 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); Assert.assertFalse(results.iterator().hasNext()); } @@ -1165,29 +1159,29 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), - createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), - createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 13870.0), - createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 13900.0), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 13870.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 13900.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0), - createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), - createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), - createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), - createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), - createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), - createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), - createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0), - createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0), - createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0) ); // Subqueries are handled by the ToolChest - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -1265,27 +1259,27 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), - createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), - createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), - createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0), - createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), - createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), - createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), - createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), - createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), - createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), - createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0), - createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0), - createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0) ); // Subqueries are handled by the ToolChest - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -1382,7 +1376,7 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow( + GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", "alias", "travel", @@ -1393,7 +1387,7 @@ public class GroupByQueryRunnerTest "js_outer_agg", 123.92274475097656 ), - createExpectedRow( + GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", "alias", "technology", @@ -1404,7 +1398,7 @@ public class GroupByQueryRunnerTest "js_outer_agg", 82.62254333496094 ), - createExpectedRow( + GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", "alias", "news", @@ -1415,7 +1409,7 @@ public class GroupByQueryRunnerTest "js_outer_agg", 125.58358001708984 ), - createExpectedRow( + GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", "alias", "health", @@ -1426,7 +1420,7 @@ public class GroupByQueryRunnerTest "js_outer_agg", 124.13470458984375 ), - createExpectedRow( + GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", "alias", "entertainment", @@ -1440,7 +1434,7 @@ public class GroupByQueryRunnerTest ); // Subqueries are handled by the ToolChest - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -1478,50 +1472,19 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 282L, "uniq", 1.0002442201269182), - createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 230L, "uniq", 1.0002442201269182), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 324L, "uniq", 1.0002442201269182), - createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 233L, "uniq", 1.0002442201269182), - createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 5317L, "uniq", 1.0002442201269182), - createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 235L, "uniq", 1.0002442201269182), - createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 5405L, "uniq", 1.0002442201269182), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 175L, "uniq", 1.0002442201269182), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 245L, "uniq", 1.0002442201269182) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 282L, "uniq", 1.0002442201269182), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 230L, "uniq", 1.0002442201269182), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 324L, "uniq", 1.0002442201269182), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 233L, "uniq", 1.0002442201269182), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 5317L, "uniq", 1.0002442201269182), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 235L, "uniq", 1.0002442201269182), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 5405L, "uniq", 1.0002442201269182), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 175L, "uniq", 1.0002442201269182), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 245L, "uniq", 1.0002442201269182) ); // Subqueries are handled by the ToolChest - Iterable results = runQuery(query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } - - private Iterable runQuery(GroupByQuery query) - { - - QueryToolChest toolChest = factory.getToolchest(); - QueryRunner theRunner = new FinalizeResultsQueryRunner<>( - toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)), - toolChest - ); - - Sequence queryResult = theRunner.run(query); - return Sequences.toList(queryResult, Lists.newArrayList()); - } - - private Row createExpectedRow(final String timestamp, Object... vals) - { - return createExpectedRow(new DateTime(timestamp), vals); - } - - private Row createExpectedRow(final DateTime timestamp, Object... vals) - { - Preconditions.checkArgument(vals.length % 2 == 0); - - Map theVals = Maps.newHashMap(); - for (int i = 0; i < vals.length; i += 2) { - theVals.put(vals[i].toString(), vals[i + 1]); - } - - DateTime ts = new DateTime(timestamp); - return new MapBasedRow(ts, theVals); - } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java new file mode 100644 index 00000000000..74dc5091a9d --- /dev/null +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java @@ -0,0 +1,72 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.groupby; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import io.druid.data.input.MapBasedRow; +import io.druid.data.input.Row; +import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; +import org.joda.time.DateTime; + +import java.util.Map; + +/** + */ +public class GroupByQueryRunnerTestHelper +{ + public static Iterable runQuery(QueryRunnerFactory factory, QueryRunner runner, GroupByQuery query) + { + + QueryToolChest toolChest = factory.getToolchest(); + QueryRunner theRunner = new FinalizeResultsQueryRunner<>( + toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)), + toolChest + ); + + Sequence queryResult = theRunner.run(query); + return Sequences.toList(queryResult, Lists.newArrayList()); + } + + public static Row createExpectedRow(final String timestamp, Object... vals) + { + return createExpectedRow(new DateTime(timestamp), vals); + } + + public static Row createExpectedRow(final DateTime timestamp, Object... vals) + { + Preconditions.checkArgument(vals.length % 2 == 0); + + Map theVals = Maps.newHashMap(); + for (int i = 0; i < vals.length; i += 2) { + theVals.put(vals[i].toString(), vals[i + 1]); + } + + DateTime ts = new DateTime(timestamp); + return new MapBasedRow(ts, theVals); + } + +} From 81641d78876c9795edf2ca30ca2e706fd371e8c5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 6 Oct 2014 12:47:16 -0700 Subject: [PATCH 03/69] http accessible while loading + status endpoint --- .../coordination/BaseZkCoordinator.java | 6 +++- .../druid/server/http/HistoricalResource.java | 32 +++++++++++++++++++ .../main/java/io/druid/cli/CliHistorical.java | 7 ++-- 3 files changed, 42 insertions(+), 3 deletions(-) create mode 100644 server/src/main/java/io/druid/server/http/HistoricalResource.java diff --git a/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java b/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java index f31cfb6311c..43776480829 100644 --- a/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java @@ -37,7 +37,6 @@ import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; import org.apache.curator.utils.ZKPaths; import java.io.IOException; -import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; /** @@ -212,6 +211,11 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler } } + public boolean isStarted() + { + return started; + } + public abstract void loadLocalCache(); public abstract DataSegmentChangeHandler getDataSegmentChangeHandler(); diff --git a/server/src/main/java/io/druid/server/http/HistoricalResource.java b/server/src/main/java/io/druid/server/http/HistoricalResource.java new file mode 100644 index 00000000000..a4f5956a0cc --- /dev/null +++ b/server/src/main/java/io/druid/server/http/HistoricalResource.java @@ -0,0 +1,32 @@ +package io.druid.server.http; + +import com.google.common.collect.ImmutableMap; +import io.druid.server.coordination.ZkCoordinator; + +import javax.inject.Inject; +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Response; + +@Path("/druid/historical/v1") +public class HistoricalResource +{ + private final ZkCoordinator coordinator; + + @Inject + public HistoricalResource( + ZkCoordinator coordinator + ) + { + this.coordinator = coordinator; + } + + @GET + @Path("/loadstatus") + @Produces("application/json") + public Response getLoadStatus() + { + return Response.ok(ImmutableMap.of("cacheInitialized", coordinator.isStarted())).build(); + } +} diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 6d4152b9226..2fda1a6dc75 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -38,6 +38,7 @@ import io.druid.query.QuerySegmentWalker; import io.druid.server.QueryResource; import io.druid.server.coordination.ServerManager; import io.druid.server.coordination.ZkCoordinator; +import io.druid.server.http.HistoricalResource; import io.druid.server.initialization.JettyServerInitializer; import io.druid.server.metrics.MetricsModule; import org.eclipse.jetty.server.Server; @@ -68,6 +69,8 @@ public class CliHistorical extends ServerRunnable @Override public void configure(Binder binder) { + // register Server before binding ZkCoordinator to ensure HTTP endpoints are available immediately + LifecycleModule.register(binder, Server.class); binder.bind(ServerManager.class).in(LazySingleton.class); binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); binder.bind(QuerySegmentWalker.class).to(ServerManager.class).in(LazySingleton.class); @@ -75,10 +78,10 @@ public class CliHistorical extends ServerRunnable binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig("historical")); binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class).in(LazySingleton.class); Jerseys.addResource(binder, QueryResource.class); + Jerseys.addResource(binder, HistoricalResource.class); LifecycleModule.register(binder, QueryResource.class); - + LifecycleModule.register(binder, HistoricalResource.class); LifecycleModule.register(binder, ZkCoordinator.class); - LifecycleModule.register(binder, Server.class); binder.bind(Cache.class).toProvider(CacheProvider.class).in(ManageLifecycle.class); JsonConfigProvider.bind(binder, "druid.historical.cache", CacheProvider.class); From 546fca2d67e56909b8eb4e0365a5cc5249733f6d Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 6 Oct 2014 14:18:25 -0700 Subject: [PATCH 04/69] docs for router node --- docs/content/Router.md | 121 +++++++++++++++++++++++++++++++++++++++++ 1 file changed, 121 insertions(+) create mode 100644 docs/content/Router.md diff --git a/docs/content/Router.md b/docs/content/Router.md new file mode 100644 index 00000000000..1bb923e3e91 --- /dev/null +++ b/docs/content/Router.md @@ -0,0 +1,121 @@ +--- +layout: doc_page +--- + +Router Node +=========== + +You should only ever need the router node if you have a Druid cluster well into the terabyte range. The router node can be used to route queries to different broker instances. By default, the broker routes queries based on how [Rules](Rules.html) are set up. For example, if 1 month of recent data is loaded into a `hot` cluster, queries that fall within the recent month can be routed to a dedicated set of brokers. Queries outside this range are routed to another set of brokers. This set up provides query isolation such that queries for more important data are not impacted by queries for less important data. + +Running +------- + +``` +io.druid.cli.Main server historical +``` + +Example Production Configuration +-------------------------------- + +In this example, we have two tiers in our production cluster: `hot` and `_default_tier`. Queries for the `hot` tier are routed through the `broker-hot` set of brokers, and queries for the `_default_tier` are routed through the `broker-cold` set of brokers. If any exceptions or network problems occur, queries are routed to the `broker-cold` set of brokers. In our example, we are running with a c3.2xlarge EC2 node. + +JVM settings: + +``` +-server +-server +-Xmx13g +-Xms13g +-XX:NewSize=256m +-XX:MaxNewSize=256m +-XX:+UseConcMarkSweepGC +-XX:+PrintGCDetails +-XX:+PrintGCTimeStamps +-XX:+UseLargePages +-XX:+HeapDumpOnOutOfMemoryError +-XX:HeapDumpPath=/mnt/galaxy/deploy/current/ +-Duser.timezone=UTC +-Dfile.encoding=UTF-8 +-Djava.io.tmpdir=/mnt/tmp + +-Dcom.sun.management.jmxremote.port=17071 +-Dcom.sun.management.jmxremote.authenticate=false +-Dcom.sun.management.jmxremote.ssl=false +``` + +Runtime.properties: + +``` +druid.host=#{IP_ADDR}:8080 +druid.port=8080 +druid.service=druid/prod/router + +druid.extensions.remoteRepositories=[] +druid.extensions.localRepository=lib +druid.extensions.coordinates=["io.druid.extensions:druid-histogram:0.6.156"] + +druid.zk.service.host=#{ZK_IPs} +druid.zk.paths.base=/druid/prod + +druid.discovery.curator.path=/prod/discovery + +druid.processing.numThreads=1 +druid.router.defaultBrokerServiceName=druid:prod:broker-cold +druid.router.coordinatorServiceName=druid:prod:coordinator +druid.router.tierToBrokerMap={"hot":"druid:prod:broker-hot","_default_tier":"druid:prod:broker-cold"} +druid.router.http.numConnections=50 +druid.router.http.readTimeout=PT5M + +druid.server.http.numThreads=100 + +druid.request.logging.type=emitter +druid.request.logging.feed=druid_requests + +druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor"] + +druid.emitter=http +druid.emitter.http.recipientBaseUrl=#{URL} + +druid.curator.compress=true +``` + +Runtime Configuration +--------------------- + +The router module uses several of the default modules in [Configuration](Configuration.html) and has the following set of configurations as well: + +|Property|Possible Values|Description|Default| +|--------|---------------|-----------|-------| +|`druid.router.defaultBrokerServiceName`|Any string.|The default broker to connect to in case service discovery fails.|""| +|`druid.router.tierToBrokerMap`|An ordered JSON map of tiers to broker names. The priority of brokers is based on the ordering.|Queries for a certain tier of data are routed to their appropriate broker.|`highestPriority`| +|`druid.router.defaultRule`|Any string.|The default rule for all datasources.|"_default"| +|`druid.router.rulesEndpoint`|Any string.|The coordinator endpoint to extract rules from.|"/druid/coordinator/v1/rules"| +|`druid.router.coordinatorServiceName`|Any string.|The service discovery name of the coordinator.|null. Must be set.| +|`druid.router.pollPeriod`|Any ISO8601 duration.|How often to poll for new rules.|PT1M| +|`druid.router.strategies`|An ordered JSON array of objects.|All custom strategies to use for routing.|[{"type":"timeBoundary"},{"type":"priority"}]| + +Router Strategies +----------------- +The router has a configurable list of strategies for how it select which broker to route queries to. The order of the strategies matter because as soon as something matches a condition, a broker is selected. + +### timeBoundary + +```json +{ + "type":"timeBoundary" +} +``` + +Including this strategy means all timeBoundary queries are always routed to the highest priority broker. + +### priority + +```json +{ + "type":"priority", + "minPriority":0, + "maxPriority":1 +} +``` + +Queries with a priority set < minPriority are routed to the lowest priority broker. Queries with priority > maxPriority are routed to the highest priority broker. minPriority default is 0, maxPriority default is 1. Using default strategy values, if a query with priority 0 (the default) is sent, it skips the priority selection logic. From ee0e8560297e94997ee7cba9ba512d62d7f11328 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 6 Oct 2014 14:23:23 -0700 Subject: [PATCH 05/69] correct mistakes --- docs/content/Router.md | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/docs/content/Router.md b/docs/content/Router.md index 1bb923e3e91..e3775047335 100644 --- a/docs/content/Router.md +++ b/docs/content/Router.md @@ -5,13 +5,13 @@ layout: doc_page Router Node =========== -You should only ever need the router node if you have a Druid cluster well into the terabyte range. The router node can be used to route queries to different broker instances. By default, the broker routes queries based on how [Rules](Rules.html) are set up. For example, if 1 month of recent data is loaded into a `hot` cluster, queries that fall within the recent month can be routed to a dedicated set of brokers. Queries outside this range are routed to another set of brokers. This set up provides query isolation such that queries for more important data are not impacted by queries for less important data. +You should only ever need the router node if you have a Druid cluster well into the terabyte range. The router node can be used to route queries to different broker nodes. By default, the broker routes queries based on how [Rules](Rules.html) are set up. For example, if 1 month of recent data is loaded into a `hot` cluster, queries that fall within the recent month can be routed to a dedicated set of brokers. Queries outside this range are routed to another set of brokers. This set up provides query isolation such that queries for more important data are not impacted by queries for less important data. Running ------- ``` -io.druid.cli.Main server historical +io.druid.cli.Main server router ``` Example Production Configuration @@ -23,7 +23,6 @@ JVM settings: ``` -server --server -Xmx13g -Xms13g -XX:NewSize=256m @@ -96,7 +95,7 @@ The router module uses several of the default modules in [Configuration](Configu Router Strategies ----------------- -The router has a configurable list of strategies for how it select which broker to route queries to. The order of the strategies matter because as soon as something matches a condition, a broker is selected. +The router has a configurable list of strategies for how it selects which brokers to route queries to. The order of the strategies matter because as soon as a strategy condition is matched, a broker is selected. ### timeBoundary @@ -118,4 +117,4 @@ Including this strategy means all timeBoundary queries are always routed to the } ``` -Queries with a priority set < minPriority are routed to the lowest priority broker. Queries with priority > maxPriority are routed to the highest priority broker. minPriority default is 0, maxPriority default is 1. Using default strategy values, if a query with priority 0 (the default) is sent, it skips the priority selection logic. +Queries with a priority set to less than minPriority are routed to the lowest priority broker. Queries with priority set to greater than maxPriority are routed to the highest priority broker. By default, minPriority is 0 and maxPriority is 1. Using these default values, if a query with priority 0 (the default query priority is 0) is sent, the query skips the priority selection logic. From f78b8f2c3109549cfcd74158741498f03f09f99d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 6 Oct 2014 17:38:13 -0700 Subject: [PATCH 06/69] fix ApproximateHistogram combining factory --- ...ApproximateHistogramAggregatorFactory.java | 26 +------ ...mateHistogramFoldingAggregatorFactory.java | 11 +-- .../aggregation/histogram/Histogram.java | 11 ++- .../ApproximateHistogramGroupByQueryTest.java | 69 +++++++++++++++++-- 4 files changed, 81 insertions(+), 36 deletions(-) diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java index 28a707a9598..c704c72ba2f 100644 --- a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java @@ -84,18 +84,6 @@ public class ApproximateHistogramAggregatorFactory implements AggregatorFactory @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { - ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); - - if (selector == null) { - return Aggregators.noopAggregator(); - } - - final Class classOfObject = selector.classOfObject(); - if (classOfObject != null && (classOfObject.equals(Object.class) - || ApproximateHistogramAggregator.class.isAssignableFrom(classOfObject))) { - System.out.println("here"); - //return new HyperUniquesAggregator(name, selector); - } return new ApproximateHistogramAggregator( name, metricFactory.makeFloatColumnSelector(fieldName), @@ -108,18 +96,6 @@ public class ApproximateHistogramAggregatorFactory implements AggregatorFactory @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { - //ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); - // - //if (selector == null) { - // return Aggregators.noopBufferAggregator(); - //} - // - //final Class classOfObject = selector.classOfObject(); - //if (classOfObject != null && (classOfObject.equals(Object.class) || ApproximateHistogramAggregator.class.isAssignableFrom(classOfObject))) { - // System.out.println("here"); - // //return new HyperUniquesAggregator(name, selector); - //} - return new ApproximateHistogramBufferAggregator( metricFactory.makeFloatColumnSelector(fieldName), resolution, @@ -143,7 +119,7 @@ public class ApproximateHistogramAggregatorFactory implements AggregatorFactory @Override public AggregatorFactory getCombiningFactory() { - return new ApproximateHistogramAggregatorFactory(name, name, resolution, numBuckets, lowerLimit, upperLimit); + return new ApproximateHistogramFoldingAggregatorFactory(name, name, resolution, numBuckets, lowerLimit, upperLimit); } @Override diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java index f84b03ae5b8..50a9e6ba973 100644 --- a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java @@ -76,8 +76,8 @@ public class ApproximateHistogramFoldingAggregatorFactory extends ApproximateHis }; } - final Class classOfObject = selector.classOfObject(); - if (classOfObject.equals(Object.class) || ApproximateHistogram.class.isAssignableFrom(classOfObject)) { + final Class cls = selector.classOfObject(); + if (cls.equals(Object.class) || ApproximateHistogram.class.isAssignableFrom(cls)) { return new ApproximateHistogramFoldingAggregator( name, selector, @@ -90,7 +90,7 @@ public class ApproximateHistogramFoldingAggregatorFactory extends ApproximateHis throw new IAE( "Incompatible type for metric[%s], expected a ApproximateHistogram, got a %s", fieldName, - selector.classOfObject() + cls ); } @@ -118,14 +118,15 @@ public class ApproximateHistogramFoldingAggregatorFactory extends ApproximateHis }; } - if (ApproximateHistogram.class.isAssignableFrom(selector.classOfObject())) { + final Class cls = selector.classOfObject(); + if (cls.equals(Object.class) || ApproximateHistogram.class.isAssignableFrom(cls)) { return new ApproximateHistogramFoldingBufferAggregator(selector, resolution, lowerLimit, upperLimit); } throw new IAE( "Incompatible type for metric[%s], expected a ApproximateHistogram, got a %s", fieldName, - selector.classOfObject() + cls ); } diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/Histogram.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/Histogram.java index 384e6eb1fbf..6785ceb9c10 100644 --- a/histogram/src/main/java/io/druid/query/aggregation/histogram/Histogram.java +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/Histogram.java @@ -84,5 +84,12 @@ public class Histogram return result; } - -} \ No newline at end of file + @Override + public String toString() + { + return "Histogram{" + + "breaks=" + Arrays.toString(breaks) + + ", counts=" + Arrays.toString(counts) + + '}'; + } +} diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index 33ee0b224d9..cd7aa5a70c1 100644 --- a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package io.druid.query.aggregation.histogram; import com.fasterxml.jackson.databind.ObjectMapper; @@ -23,6 +42,8 @@ import io.druid.query.groupby.GroupByQueryEngine; import io.druid.query.groupby.GroupByQueryQueryToolChest; import io.druid.query.groupby.GroupByQueryRunnerFactory; import io.druid.query.groupby.GroupByQueryRunnerTestHelper; +import io.druid.query.groupby.orderby.DefaultLimitSpec; +import io.druid.query.groupby.orderby.OrderByColumnSpec; import io.druid.segment.TestHelper; import org.junit.Before; import org.junit.Test; @@ -130,7 +151,7 @@ public class ApproximateHistogramGroupByQueryTest } @Test - public void testGroupByNWithApproximateHistogramAgg() + public void testGroupByWithApproximateHistogramAgg() { ApproximateHistogramAggregatorFactory aggFactory = new ApproximateHistogramAggregatorFactory( "apphisto", @@ -146,8 +167,18 @@ public class ApproximateHistogramGroupByQueryTest .setGranularity(QueryRunnerTestHelper.allGran) .setDimensions(Arrays.asList(new LegacyDimensionSpec(QueryRunnerTestHelper.providerDimension))) .setInterval(QueryRunnerTestHelper.fullOnInterval) + .setLimitSpec( + new DefaultLimitSpec( + Lists.newArrayList( + new OrderByColumnSpec( + QueryRunnerTestHelper.providerDimension, + OrderByColumnSpec.Direction.DESCENDING + ) + ), 1 + ) + ) .setAggregatorSpecs( - Lists.newArrayList( + Lists.newArrayList( Iterables.concat( QueryRunnerTestHelper.commonAggregators, Lists.newArrayList( @@ -159,7 +190,7 @@ public class ApproximateHistogramGroupByQueryTest ) ) .setPostAggregatorSpecs( - Arrays.asList( + Arrays.asList( QueryRunnerTestHelper.addRowsIndexConstant, QueryRunnerTestHelper.dependentPostAgg, new QuantilePostAggregator("quantile", "apphisto", 0.5f) @@ -167,7 +198,37 @@ public class ApproximateHistogramGroupByQueryTest ) .build(); - List expectedResults = Arrays.asList( + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "1970-01-01T00:00:00.000Z", + "provider", "spot", + "rows", 837L, + "addRowsIndexConstant", 96444.5703125, + "dependentPostAgg", 97282.5703125, + "index", 95606.5703125, + "maxIndex", 277.2735290527344, + "minIndex", 59.02102279663086, + "quantile", 101.78856f, + "uniques", QueryRunnerTestHelper.UNIQUES_9, + "apphisto", + new Histogram( + new float[]{ + 4.457897186279297f, + 59.02102279663086f, + 113.58415222167969f, + 168.14727783203125f, + 222.7104034423828f, + 277.2735290527344f + }, + new double[]{ + 0.0, + 462.4309997558594, + 357.5404968261719, + 15.022850036621094, + 2.0056631565093994 + } + ) + ) ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); From f1d37d569c4736a9f0664b30869f0ca05e2b7004 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 7 Oct 2014 13:01:41 -0700 Subject: [PATCH 07/69] doc fixes --- docs/content/Router.md | 4 ++-- docs/content/toc.textile | 1 + pom.xml | 4 ++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/docs/content/Router.md b/docs/content/Router.md index e3775047335..ce8033836cf 100644 --- a/docs/content/Router.md +++ b/docs/content/Router.md @@ -85,8 +85,8 @@ The router module uses several of the default modules in [Configuration](Configu |Property|Possible Values|Description|Default| |--------|---------------|-----------|-------| -|`druid.router.defaultBrokerServiceName`|Any string.|The default broker to connect to in case service discovery fails.|""| -|`druid.router.tierToBrokerMap`|An ordered JSON map of tiers to broker names. The priority of brokers is based on the ordering.|Queries for a certain tier of data are routed to their appropriate broker.|`highestPriority`| +|`druid.router.defaultBrokerServiceName`|Any string.|The default broker to connect to in case service discovery fails.|"". Must be set.| +|`druid.router.tierToBrokerMap`|An ordered JSON map of tiers to broker names. The priority of brokers is based on the ordering.|Queries for a certain tier of data are routed to their appropriate broker.|{"_default": ""}| |`druid.router.defaultRule`|Any string.|The default rule for all datasources.|"_default"| |`druid.router.rulesEndpoint`|Any string.|The coordinator endpoint to extract rules from.|"/druid/coordinator/v1/rules"| |`druid.router.coordinatorServiceName`|Any string.|The service discovery name of the coordinator.|null. Must be set.| diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 437e383b760..d811e68b55a 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -84,6 +84,7 @@ h2. Experimental * "Geographic Queries":./GeographicQueries.html * "Select Query":./SelectQuery.html * "Approximate Histograms and Quantiles":./ApproxHisto.html +* "Router node":./Router.html h2. Development * "Versioning":./Versioning.html diff --git a/pom.xml b/pom.xml index 1ad6ccb9338..60304fd173c 100644 --- a/pom.xml +++ b/pom.xml @@ -39,9 +39,9 @@ UTF-8 - 0.26.7 + 0.26.8 2.6.0 - 0.2.10 + 0.2.11 From 30183d3b4dafbdd65adf6a0741f77a53023db7b9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 7 Oct 2014 12:40:12 -0700 Subject: [PATCH 08/69] update druid-api for json parsing improvements --- pom.xml | 2 +- .../java/io/druid/data/input/ProtoBufInputRowParser.java | 3 ++- .../io/druid/data/input/ProtoBufInputRowParserTest.java | 3 ++- .../java/io/druid/segment/realtime/FireDepartmentTest.java | 3 ++- .../segment/realtime/plumber/RealtimePlumberSchoolTest.java | 6 +++++- 5 files changed, 12 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index 60304fd173c..54158c11c82 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.8 2.6.0 - 0.2.11 + 0.2.12 diff --git a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java index df8bdb5cef8..4a449efbe82 100644 --- a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java +++ b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java @@ -69,7 +69,8 @@ public class ProtoBufInputRowParser implements ByteBufferInputRowParser if (parseSpec == null) { this.parseSpec = new JSONParseSpec( timestampSpec, - new DimensionsSpec(dimensions, dimensionExclusions, spatialDimensions) + new DimensionsSpec(dimensions, dimensionExclusions, spatialDimensions), + JSONParseSpec.JSON ); } else { this.parseSpec = parseSpec; diff --git a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java index 67d5fdc3574..570045daab0 100644 --- a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java +++ b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java @@ -59,7 +59,8 @@ public class ProtoBufInputRowParserTest ProtoBufInputRowParser parser = new ProtoBufInputRowParser( new JSONParseSpec( new TimestampSpec("timestamp", "iso"), - new DimensionsSpec(Arrays.asList(DIMENSIONS), Arrays.asList(), null) + new DimensionsSpec(Arrays.asList(DIMENSIONS), Arrays.asList(), null), + JSONParseSpec.JSON ), "prototest.desc", null, null, null, null diff --git a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java index 5fca643fc08..d212342abdf 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -61,7 +61,8 @@ public class FireDepartmentTest Arrays.asList("dim1", "dim2"), null, null - ) + ), + JSONParseSpec.JSON ), null, null, null, null ), diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index e610d3c53e7..ff4808f6815 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -123,7 +123,11 @@ public class RealtimePlumberSchoolTest @Override public ParseSpec getParseSpec() { - return new JSONParseSpec(new TimestampSpec("timestamp", "auto"), new DimensionsSpec(null, null, null)); + return new JSONParseSpec( + new TimestampSpec("timestamp", "auto"), + new DimensionsSpec(null, null, null), + JSONParseSpec.JSON + ); } @Override From 19dc32d314fb714aad4b53c24f6b66c93f8efeae Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 7 Oct 2014 13:50:32 -0700 Subject: [PATCH 09/69] add headers for druid vs pages --- docs/content/Druid-vs-Cassandra.md | 5 +++++ docs/content/Druid-vs-Hadoop.md | 4 ++++ docs/content/Druid-vs-Impala-or-Shark.md | 4 ++++ docs/content/Druid-vs-Redshift.md | 4 ++++ docs/content/Druid-vs-Vertica.md | 5 +++++ 5 files changed, 22 insertions(+) diff --git a/docs/content/Druid-vs-Cassandra.md b/docs/content/Druid-vs-Cassandra.md index 4ef29a1ab36..f8d1d59a473 100644 --- a/docs/content/Druid-vs-Cassandra.md +++ b/docs/content/Druid-vs-Cassandra.md @@ -1,6 +1,11 @@ --- layout: doc_page --- + +Druid vs. Cassandra +=================== + + We are not experts on Cassandra, if anything is incorrect about our portrayal, please let us know on the mailing list or via some other means. We will fix this page. Druid is highly optimized for scans and aggregations, it supports arbitrarily deep drill downs into data sets without the need to pre-compute, and it can ingest event streams in real-time and allow users to query events as they come in. Cassandra is a great key-value store and it has some features that allow you to use it to do more interesting things than what you can do with a pure key-value store. But, it is not built for the same use cases that Druid handles, namely regularly scanning over billions of entries per query. diff --git a/docs/content/Druid-vs-Hadoop.md b/docs/content/Druid-vs-Hadoop.md index 3b4ef1df76c..47d03dd7704 100644 --- a/docs/content/Druid-vs-Hadoop.md +++ b/docs/content/Druid-vs-Hadoop.md @@ -2,6 +2,10 @@ layout: doc_page --- +Druid vs Hadoop +=============== + + Hadoop has shown the world that it’s possible to house your data warehouse on commodity hardware for a fraction of the price of typical solutions. As people adopt Hadoop for their data warehousing needs, they find two things. 1. They can now query all of their data in a fairly flexible manner and answer any question they have diff --git a/docs/content/Druid-vs-Impala-or-Shark.md b/docs/content/Druid-vs-Impala-or-Shark.md index cb658c8e087..ab2cb122524 100644 --- a/docs/content/Druid-vs-Impala-or-Shark.md +++ b/docs/content/Druid-vs-Impala-or-Shark.md @@ -1,6 +1,10 @@ --- layout: doc_page --- + +Druid vs Impala or Shark +======================== + The question of Druid versus Impala or Shark basically comes down to your product requirements and what the systems were designed to do. Druid was designed to diff --git a/docs/content/Druid-vs-Redshift.md b/docs/content/Druid-vs-Redshift.md index 4fe06586467..faaaa3f0f33 100644 --- a/docs/content/Druid-vs-Redshift.md +++ b/docs/content/Druid-vs-Redshift.md @@ -1,6 +1,10 @@ --- layout: doc_page --- +Druid vs Redshift +================= + + ###How does Druid compare to Redshift? In terms of drawing a differentiation, Redshift is essentially ParAccel (Actian) which Amazon is licensing. diff --git a/docs/content/Druid-vs-Vertica.md b/docs/content/Druid-vs-Vertica.md index e6971ae03d9..4fe0eb78892 100644 --- a/docs/content/Druid-vs-Vertica.md +++ b/docs/content/Druid-vs-Vertica.md @@ -1,6 +1,11 @@ --- layout: doc_page --- + +Druid vs Vertica +================ + + How does Druid compare to Vertica? Vertica is similar to ParAccel/Redshift ([Druid-vs-Redshift](Druid-vs-Redshift.html)) described above in that it wasn’t built for real-time streaming data ingestion and it supports full SQL. From 2f21b2754cb005e2ce25cbc77c6bb65ef06be0ce Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 7 Oct 2014 13:57:03 -0700 Subject: [PATCH 10/69] fix compile --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 54158c11c82..f79f015c209 100644 --- a/pom.xml +++ b/pom.xml @@ -39,7 +39,7 @@ UTF-8 - 0.26.8 + 0.26.9 2.6.0 0.2.12 From 8f7f497b55b3223229ab478eab41ede9d63a2691 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 7 Oct 2014 14:19:59 -0700 Subject: [PATCH 11/69] jax-rs smile support --- pom.xml | 5 +++++ server/pom.xml | 4 ++++ 2 files changed, 9 insertions(+) diff --git a/pom.xml b/pom.xml index f79f015c209..6dfdc2d8508 100644 --- a/pom.xml +++ b/pom.xml @@ -256,6 +256,11 @@ jackson-jaxrs-json-provider 2.2.3 + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-smile-provider + 2.2.3 + org.codehaus.jackson jackson-core-asl diff --git a/server/pom.xml b/server/pom.xml index 7382aefb237..a8208e048eb 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -77,6 +77,10 @@ com.fasterxml.jackson.jaxrs jackson-jaxrs-json-provider + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-smile-provider + com.fasterxml.jackson.dataformat jackson-dataformat-smile From 1ce67dcfc473f182b0cf4f34556d4da3938a57e2 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 7 Oct 2014 14:27:47 -0700 Subject: [PATCH 12/69] prepare for next release --- docs/content/Examples.md | 4 ++-- docs/content/Kafka-Eight.md | 4 ++-- docs/content/Production-Cluster-Configuration.md | 6 +++--- docs/content/Realtime-Config.md | 4 ++-- docs/content/Router.md | 2 +- docs/content/Simple-Cluster-Configuration.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-1.md | 2 +- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.md | 2 +- 11 files changed, 20 insertions(+), 20 deletions(-) diff --git a/docs/content/Examples.md b/docs/content/Examples.md index 468bdb8f41a..049218da290 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.156 +git checkout druid-0.6.157 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.156-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Kafka-Eight.md b/docs/content/Kafka-Eight.md index d8b3c80232e..c167aea59ba 100644 --- a/docs/content/Kafka-Eight.md +++ b/docs/content/Kafka-Eight.md @@ -8,9 +8,9 @@ The previous examples are for Kafka 7. To support Kafka 8, a couple changes need - Update realtime node's configs for Kafka 8 extensions - e.g. - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.156",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.157",...]` - becomes - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.156",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.157",...]` - Update realtime task config for changed keys - `firehose.type`, `plumber.rejectionPolicyFactory`, and all of `firehose.consumerProps` changes. diff --git a/docs/content/Production-Cluster-Configuration.md b/docs/content/Production-Cluster-Configuration.md index d8ed57ab048..0cf81020510 100644 --- a/docs/content/Production-Cluster-Configuration.md +++ b/docs/content/Production-Cluster-Configuration.md @@ -57,7 +57,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/overlord -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -139,7 +139,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/middlemanager -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.156","io.druid.extensions:druid-kafka-seven:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157","io.druid.extensions:druid-kafka-seven:0.6.157"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -286,7 +286,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/historical -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Realtime-Config.md b/docs/content/Realtime-Config.md index 6cb566fac16..cf5104d550d 100644 --- a/docs/content/Realtime-Config.md +++ b/docs/content/Realtime-Config.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.157"] druid.zk.service.host=localhost @@ -76,7 +76,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/realtime -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.156","io.druid.extensions:druid-kafka-seven:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157","io.druid.extensions:druid-kafka-seven:0.6.157"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Router.md b/docs/content/Router.md index ce8033836cf..b6638fdadd2 100644 --- a/docs/content/Router.md +++ b/docs/content/Router.md @@ -51,7 +51,7 @@ druid.service=druid/prod/router druid.extensions.remoteRepositories=[] druid.extensions.localRepository=lib -druid.extensions.coordinates=["io.druid.extensions:druid-histogram:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-histogram:0.6.157"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Simple-Cluster-Configuration.md b/docs/content/Simple-Cluster-Configuration.md index 788f2d65af8..e873895f097 100644 --- a/docs/content/Simple-Cluster-Configuration.md +++ b/docs/content/Simple-Cluster-Configuration.md @@ -28,7 +28,7 @@ Configuration: -Ddruid.zk.service.host=localhost --Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.156"] +-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.157"] -Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid -Ddruid.db.connector.user=druid diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index a0c89c0fafb..95f53508448 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.156-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.156 +cd druid-services-0.6.157 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md index 399af3cc786..12a5ea405af 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md @@ -91,7 +91,7 @@ druid.service=overlord druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.157"] druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid druid.db.connector.user=druid diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 13483109143..5673a9f8f1f 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes and external depende If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.156-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -240,7 +240,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.156","io.druid.extensions:druid-kafka-seven:0.6.156"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.157","io.druid.extensions:druid-kafka-seven:0.6.157"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index d28dc27923c..83dfd848924 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.156-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.156 +cd druid-services-0.6.157 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.md b/docs/content/Twitter-Tutorial.md index db0eab430c6..c8f2b2adc0e 100644 --- a/docs/content/Twitter-Tutorial.md +++ b/docs/content/Twitter-Tutorial.md @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. # Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.156-bin.tar.gz). +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz). Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: From 96e29c2fdd4cd1b0231e934908c86a46c9ec087c Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 7 Oct 2014 14:30:20 -0700 Subject: [PATCH 13/69] [maven-release-plugin] prepare release druid-0.6.157 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index e07536e0705..8354210a1bd 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/common/pom.xml b/common/pom.xml index 687d67afe92..0a5c46c4206 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/examples/pom.xml b/examples/pom.xml index ed6bfaadae4..10f3b185d70 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 2ab577dd4ef..8bc6c573262 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/histogram/pom.xml b/histogram/pom.xml index e2420e915b3..7c69040227e 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 32336247e92..60d62150584 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index a398851837c..9d5f9bfae1f 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 2251c71c8cd..701c62cb5b0 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 9f4274211d0..1c3277c51f0 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/pom.xml b/pom.xml index 6dfdc2d8508..01c560ad54f 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.157-SNAPSHOT + 0.6.157 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.131-SNAPSHOT + druid-0.6.157 diff --git a/processing/pom.xml b/processing/pom.xml index c00837b2d91..c0f51992ad5 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 2c0590533f9..316fa93c232 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index a512b649ecd..6595ceeb79b 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/server/pom.xml b/server/pom.xml index a8208e048eb..c3dac395c5e 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 diff --git a/services/pom.xml b/services/pom.xml index cfccaaed8de..7f88ab779ff 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.157-SNAPSHOT + 0.6.157 From 8a4b32c482a4821bf82453e67adfdfd88140650a Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 7 Oct 2014 14:30:26 -0700 Subject: [PATCH 14/69] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 8354210a1bd..f976ec19663 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 0a5c46c4206..62a4c7efd0d 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 10f3b185d70..e4cf96e065e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 8bc6c573262..2ef16b75cfe 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index 7c69040227e..dd73d1a7ea3 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 60d62150584..1268759137d 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 9d5f9bfae1f..8031019b137 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 701c62cb5b0..3a30b97b1eb 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 1c3277c51f0..502517995c3 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/pom.xml b/pom.xml index 01c560ad54f..03244fb8736 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.157 + 0.6.158-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.157 + druid-0.6.131-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index c0f51992ad5..a6f09b1f608 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 316fa93c232..291e333eab5 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 6595ceeb79b..eeda5228469 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index c3dac395c5e..0fcf50829bd 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 7f88ab779ff..bd4d5d7a6d2 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.157 + 0.6.158-SNAPSHOT From c25203e1409ed2a2474fd267259e85e1a67e7942 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 7 Oct 2014 14:48:49 -0700 Subject: [PATCH 15/69] remove unused jsonspec content type --- pom.xml | 2 +- .../main/java/io/druid/data/input/ProtoBufInputRowParser.java | 3 +-- .../java/io/druid/data/input/ProtoBufInputRowParserTest.java | 3 +-- .../java/io/druid/segment/realtime/FireDepartmentTest.java | 3 +-- .../segment/realtime/plumber/RealtimePlumberSchoolTest.java | 3 +-- 5 files changed, 5 insertions(+), 9 deletions(-) diff --git a/pom.xml b/pom.xml index 6dfdc2d8508..f14cedd7440 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.9 2.6.0 - 0.2.12 + 0.2.13 diff --git a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java index 4a449efbe82..df8bdb5cef8 100644 --- a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java +++ b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java @@ -69,8 +69,7 @@ public class ProtoBufInputRowParser implements ByteBufferInputRowParser if (parseSpec == null) { this.parseSpec = new JSONParseSpec( timestampSpec, - new DimensionsSpec(dimensions, dimensionExclusions, spatialDimensions), - JSONParseSpec.JSON + new DimensionsSpec(dimensions, dimensionExclusions, spatialDimensions) ); } else { this.parseSpec = parseSpec; diff --git a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java index 570045daab0..67d5fdc3574 100644 --- a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java +++ b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java @@ -59,8 +59,7 @@ public class ProtoBufInputRowParserTest ProtoBufInputRowParser parser = new ProtoBufInputRowParser( new JSONParseSpec( new TimestampSpec("timestamp", "iso"), - new DimensionsSpec(Arrays.asList(DIMENSIONS), Arrays.asList(), null), - JSONParseSpec.JSON + new DimensionsSpec(Arrays.asList(DIMENSIONS), Arrays.asList(), null) ), "prototest.desc", null, null, null, null diff --git a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java index d212342abdf..5fca643fc08 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -61,8 +61,7 @@ public class FireDepartmentTest Arrays.asList("dim1", "dim2"), null, null - ), - JSONParseSpec.JSON + ) ), null, null, null, null ), diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index ff4808f6815..3a9cdd3ebe6 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -125,8 +125,7 @@ public class RealtimePlumberSchoolTest { return new JSONParseSpec( new TimestampSpec("timestamp", "auto"), - new DimensionsSpec(null, null, null), - JSONParseSpec.JSON + new DimensionsSpec(null, null, null) ); } From 8a34a55c0bb6bd9e8e107e9ef35b44972ef1edb3 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 8 Oct 2014 10:51:00 -0700 Subject: [PATCH 16/69] add example hadoop config --- docs/content/Hadoop-Configuration.md | 360 +++++++++++++++++++++++++++ docs/content/toc.textile | 1 + 2 files changed, 361 insertions(+) create mode 100644 docs/content/Hadoop-Configuration.md diff --git a/docs/content/Hadoop-Configuration.md b/docs/content/Hadoop-Configuration.md new file mode 100644 index 00000000000..296558fcee7 --- /dev/null +++ b/docs/content/Hadoop-Configuration.md @@ -0,0 +1,360 @@ +--- +layout: doc_page +--- + +Example Production Hadoop Configuration +======================================= + +The following configuration should work relatively well for Druid indexing and Hadoop. In the example, we are using EC2 cc2.8xlarge nodes. + +### Core-site.xml + +``` + + + + + hadoop.tmp.dir + /mnt/persistent/hadoop + + + + + fs.defaultFS + hdfs://#{IP}:9000 + + + fs.s3.impl + org.apache.hadoop.fs.s3native.NativeS3FileSystem + + + fs.s3.awsAccessKeyId + #{S3_ACCESS_KEY} + + + fs.s3.awsSecretAccessKey + #{S3_SECRET_KEY} + + + fs.s3.buffer.dir + /mnt/persistent/hadoop-s3n + + + fs.s3n.awsAccessKeyId + #{S3N_ACCESS_KEY} + + + fs.s3n.awsSecretAccessKey + #{S3N_SECRET_KEY} + + + + + io.compression.codecs + org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.Lz4Codec,org.apache.hadoop.io.compress.BZip2Codec,org.apache.hadoop.io.compress.SnappyCodec + + + + + io.seqfile.local.dir + /mnt/persistent/hadoop/io/local + + + +``` + +### Mapred-site.xml + +``` + + + + mapreduce.framework.name + yarn + + + + mapreduce.jobtracker.address + #{JT_ADDR}:9001 + + + mapreduce.jobtracker.http.address + #{JT_HTTP_ADDR}:9100 + + + mapreduce.jobhistory.address + #{JH_ADDR}:10020 + + + mapreduce.jobhistory.webapp.address + #{JH_WEBAPP_ADDR}:19888 + + + mapreduce.tasktracker.http.address + #{TT_ADDR}:9103 + + + + + mapreduce.job.reduces + 21 + + + + mapreduce.job.jvm.numtasks + 20 + + + mapreduce.map.memory.mb + 2048 + + + mapreduce.map.java.opts + -server -Xmx1536m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps + + + mapreduce.reduce.memory.mb + 6144 + + + mapreduce.reduce.java.opts + -server -Xmx2560m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps + + + mapreduce.reduce.shuffle.parallelcopies + 50 + + + mapreduce.reduce.shuffle.input.buffer.percent + 0.5 + + + mapreduce.task.io.sort.mb + 256 + + + mapreduce.task.io.sort.factor + 100 + + + mapreduce.jobtracker.handler.count + 64 + + + mapreduce.tasktracker.http.threads + 20 + + + + + mapreduce.cluster.local.dir + /mnt/persistent/hadoop/mapred/local + + + + + mapreduce.jobhistory.recovery.enable + true + + + mapreduce.jobhistory.recovery.store.class + org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService + + + mapreduce.jobhistory.recovery.store.fs.uri + file://${hadoop.tmp.dir}/mapred-jobhistory-state + + + + + + mapreduce.output.fileoutputformat.compress + false + + + mapreduce.map.output.compress + true + + + mapreduce.output.fileoutputformat.compress.type + BLOCK + + + mapreduce.map.output.compress.codec + org.apache.hadoop.io.compress.Lz4Codec + + + mapreduce.output.fileoutputformat.compress.codec + org.apache.hadoop.io.compress.GzipCodec + + + + mapreduce.map.speculative + false + + + mapreduce.reduce.speculative + false + + + + + mapreduce.task.timeout + 1800000 + + + +``` + +### Yarn-site.xml + +``` + + + + yarn.resourcemanager.hostname + #{RM_HOSTNAME} + + + yarn.resourcemanager.scheduler.class + org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler + + + yarn.nodemanager.aux-services + mapreduce_shuffle + + + yarn.log-aggregation-enable + true + + + yarn.log.server.url + http://#{IP_LOG_SERVER}:19888/jobhistory/logs/ + + + yarn.nodemanager.hostname + #{IP_ADDR} + + + yarn.scheduler.minimum-allocation-mb + 512 + + + yarn.nodemanager.resource.memory-mb + 1024 + + + yarn.nodemanager.resource.cpu-vcores + 1 + + + yarn.nodemanager.vmem-check-enabled + false + + + + + yarn.nodemanager.local-dirs + /mnt/persistent/hadoop/nm-local-dir + + + + + yarn.resourcemanager.recovery.enabled + false + + + yarn.resourcemanager.store.class + org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore + + + yarn.resourcemanager.fs.state-store.uri + file://${hadoop.tmp.dir}/yarn-resourcemanager-state + + + + + yarn.resourcemanager.nodes.exclude-path + /mnt/persistent/hadoop/yarn-exclude.txt + + + +``` + +### HDFS-site.xml + +``` + + + + dfs.replication + 3 + + + dfs.namenode.datanode.registration.ip-hostname-check + false + + + dfs.hosts.exclude + /mnt/persistent/hadoop/hdfs-exclude.txt + + + + + dfs.datanode.data.dir + file:///mnt/persistent/hadoop/dfs/data + + + +``` + +### Capacity-scheduler.xml + +``` + + + + yarn.scheduler.capacity.maximum-am-resource-percent + 0.1 + + + yarn.scheduler.capacity.root.queues + default + + + yarn.scheduler.capacity.root.default.capacity + 100 + + + yarn.scheduler.capacity.root.default.user-limit-factor + 1 + + + yarn.scheduler.capacity.root.queues + default + + + yarn.scheduler.capacity.root.default.maximum-capacity + 100 + + + yarn.scheduler.capacity.root.default.state + RUNNING + + + yarn.scheduler.capacity.root.default.acl_submit_applications + * + + + yarn.scheduler.capacity.root.default.acl_administer_queue + * + + + yarn.scheduler.capacity.node-locality-delay + -1 + + + +``` \ No newline at end of file diff --git a/docs/content/toc.textile b/docs/content/toc.textile index d811e68b55a..cc3c86eac99 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -17,6 +17,7 @@ h2. Getting Started h2. Booting a Druid Cluster * "Simple Cluster Configuration":Simple-Cluster-Configuration.html * "Production Cluster Configuration":Production-Cluster-Configuration.html +* "Production Hadoop Configuration":Hadoop-Configuration.html * "Rolling Cluster Updates":Rolling-Updates.html h2. Configuration From 21ac3f2aab1eb7b83533daf9201f0bc2ab0078c3 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 8 Oct 2014 10:56:00 -0700 Subject: [PATCH 17/69] slight clarification about hadoop setup --- docs/content/Hadoop-Configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Hadoop-Configuration.md b/docs/content/Hadoop-Configuration.md index 296558fcee7..ef8c48e167a 100644 --- a/docs/content/Hadoop-Configuration.md +++ b/docs/content/Hadoop-Configuration.md @@ -5,7 +5,7 @@ layout: doc_page Example Production Hadoop Configuration ======================================= -The following configuration should work relatively well for Druid indexing and Hadoop. In the example, we are using EC2 cc2.8xlarge nodes. +The following configuration should work relatively well for Druid indexing and Hadoop. In the example, we are using Hadoop 2.4 with EC2 m1.xlarge nodes for NameNodes and cc2.8xlarge nodes for DataNodes. ### Core-site.xml From 290107ce7b8d1d3401e26c44b860d61b78094d0c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 7 Oct 2014 14:48:49 -0700 Subject: [PATCH 18/69] remove unused jsonspec content type --- pom.xml | 2 +- .../main/java/io/druid/data/input/ProtoBufInputRowParser.java | 3 +-- .../java/io/druid/data/input/ProtoBufInputRowParserTest.java | 3 +-- .../java/io/druid/segment/realtime/FireDepartmentTest.java | 3 +-- .../segment/realtime/plumber/RealtimePlumberSchoolTest.java | 3 +-- 5 files changed, 5 insertions(+), 9 deletions(-) diff --git a/pom.xml b/pom.xml index 03244fb8736..5731b98d72a 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.9 2.6.0 - 0.2.12 + 0.2.13 diff --git a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java index 4a449efbe82..df8bdb5cef8 100644 --- a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java +++ b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java @@ -69,8 +69,7 @@ public class ProtoBufInputRowParser implements ByteBufferInputRowParser if (parseSpec == null) { this.parseSpec = new JSONParseSpec( timestampSpec, - new DimensionsSpec(dimensions, dimensionExclusions, spatialDimensions), - JSONParseSpec.JSON + new DimensionsSpec(dimensions, dimensionExclusions, spatialDimensions) ); } else { this.parseSpec = parseSpec; diff --git a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java index 570045daab0..67d5fdc3574 100644 --- a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java +++ b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java @@ -59,8 +59,7 @@ public class ProtoBufInputRowParserTest ProtoBufInputRowParser parser = new ProtoBufInputRowParser( new JSONParseSpec( new TimestampSpec("timestamp", "iso"), - new DimensionsSpec(Arrays.asList(DIMENSIONS), Arrays.asList(), null), - JSONParseSpec.JSON + new DimensionsSpec(Arrays.asList(DIMENSIONS), Arrays.asList(), null) ), "prototest.desc", null, null, null, null diff --git a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java index d212342abdf..5fca643fc08 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -61,8 +61,7 @@ public class FireDepartmentTest Arrays.asList("dim1", "dim2"), null, null - ), - JSONParseSpec.JSON + ) ), null, null, null, null ), diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index ff4808f6815..3a9cdd3ebe6 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -125,8 +125,7 @@ public class RealtimePlumberSchoolTest { return new JSONParseSpec( new TimestampSpec("timestamp", "auto"), - new DimensionsSpec(null, null, null), - JSONParseSpec.JSON + new DimensionsSpec(null, null, null) ); } From e2e912cdc37e404e8f9c0dd7cf1192e1e8bec716 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 9 Oct 2014 10:26:39 -0700 Subject: [PATCH 19/69] fix broken link, resolved #782 --- docs/content/Querying.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Querying.md b/docs/content/Querying.md index 2b2981b96f7..df516ac4e05 100644 --- a/docs/content/Querying.md +++ b/docs/content/Querying.md @@ -88,7 +88,7 @@ A groupBy also requires the JSON field "aggregations" (See [Aggregations](Aggreg ], ``` -You can also specify postAggregations, which are applied after data has been aggregated for the current granularity and dimensions bucket. See [Post Aggregations](Post Aggregations.html) for a detailed description. In the rand example, an arithmetic type operation (division, as specified by "fn") is performed with the result "name" of "avg_random". The "fields" field specifies the inputs from the aggregation stage to this expression. Note that identifiers corresponding to "name" JSON field inside the type "fieldAccess" are required but not used outside this expression, so they are prefixed with "dummy" for clarity: +You can also specify postAggregations, which are applied after data has been aggregated for the current granularity and dimensions bucket. See [Post Aggregations](Post-aggregations.html) for a detailed description. In the rand example, an arithmetic type operation (division, as specified by "fn") is performed with the result "name" of "avg_random". The "fields" field specifies the inputs from the aggregation stage to this expression. Note that identifiers corresponding to "name" JSON field inside the type "fieldAccess" are required but not used outside this expression, so they are prefixed with "dummy" for clarity: ```javascript "postAggregations": [{ From fa27908ae41f7438cea6a28718aa1f9b42957d9f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 9 Oct 2014 10:28:33 -0700 Subject: [PATCH 20/69] and another one, fixes #782 --- docs/content/Querying.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Querying.md b/docs/content/Querying.md index df516ac4e05..895a1cf2239 100644 --- a/docs/content/Querying.md +++ b/docs/content/Querying.md @@ -127,7 +127,7 @@ Properties shared by all query types |timeseries, topN, groupBy, search|filter|Specifies the filter (the "WHERE" clause in SQL) for the query. See [Filters](Filters.html)|no| |timeseries, topN, groupBy, search|granularity|the timestamp granularity to bucket results into (i.e. "hour"). See [Granularities](Granularities.html) for more information.|no| |timeseries, topN, groupBy|aggregations|aggregations that combine values in a bucket. See [Aggregations](Aggregations.html).|yes| -|timeseries, topN, groupBy|postAggregations|aggregations of aggregations. See [Post Aggregations](Post Aggregations.html).|yes| +|timeseries, topN, groupBy|postAggregations|aggregations of aggregations. See [Post Aggregations](Post-aggregations.html).|yes| |groupBy|dimensions|constrains the groupings; if empty, then one value per time granularity bucket|yes| |search|limit|maximum number of results (default is 1000), a system-level maximum can also be set via `com.metamx.query.search.maxSearchLimit`|no| |search|searchDimensions|Dimensions to apply the search query to. If not specified, it will search through all dimensions.|no| From 0532208c25ad737adadbbe29670e6dc0d160f42d Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 9 Oct 2014 14:21:38 -0700 Subject: [PATCH 21/69] prepare for next release --- docs/content/Examples.md | 4 ++-- docs/content/Kafka-Eight.md | 4 ++-- docs/content/Production-Cluster-Configuration.md | 6 +++--- docs/content/Realtime-Config.md | 4 ++-- docs/content/Router.md | 2 +- docs/content/Simple-Cluster-Configuration.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-1.md | 2 +- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.md | 2 +- pom.xml | 2 +- 12 files changed, 21 insertions(+), 21 deletions(-) diff --git a/docs/content/Examples.md b/docs/content/Examples.md index 049218da290..b61168e20dc 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.157 +git checkout druid-0.6.158 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Kafka-Eight.md b/docs/content/Kafka-Eight.md index c167aea59ba..b7ae4e5c4ab 100644 --- a/docs/content/Kafka-Eight.md +++ b/docs/content/Kafka-Eight.md @@ -8,9 +8,9 @@ The previous examples are for Kafka 7. To support Kafka 8, a couple changes need - Update realtime node's configs for Kafka 8 extensions - e.g. - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.157",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.158",...]` - becomes - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.157",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.158",...]` - Update realtime task config for changed keys - `firehose.type`, `plumber.rejectionPolicyFactory`, and all of `firehose.consumerProps` changes. diff --git a/docs/content/Production-Cluster-Configuration.md b/docs/content/Production-Cluster-Configuration.md index 0cf81020510..70b4cb4036d 100644 --- a/docs/content/Production-Cluster-Configuration.md +++ b/docs/content/Production-Cluster-Configuration.md @@ -57,7 +57,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/overlord -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -139,7 +139,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/middlemanager -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157","io.druid.extensions:druid-kafka-seven:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158","io.druid.extensions:druid-kafka-seven:0.6.158"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -286,7 +286,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/historical -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Realtime-Config.md b/docs/content/Realtime-Config.md index cf5104d550d..c61a22b736d 100644 --- a/docs/content/Realtime-Config.md +++ b/docs/content/Realtime-Config.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.158"] druid.zk.service.host=localhost @@ -76,7 +76,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/realtime -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157","io.druid.extensions:druid-kafka-seven:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158","io.druid.extensions:druid-kafka-seven:0.6.158"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Router.md b/docs/content/Router.md index b6638fdadd2..2a077e944fb 100644 --- a/docs/content/Router.md +++ b/docs/content/Router.md @@ -51,7 +51,7 @@ druid.service=druid/prod/router druid.extensions.remoteRepositories=[] druid.extensions.localRepository=lib -druid.extensions.coordinates=["io.druid.extensions:druid-histogram:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-histogram:0.6.158"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Simple-Cluster-Configuration.md b/docs/content/Simple-Cluster-Configuration.md index e873895f097..98871836233 100644 --- a/docs/content/Simple-Cluster-Configuration.md +++ b/docs/content/Simple-Cluster-Configuration.md @@ -28,7 +28,7 @@ Configuration: -Ddruid.zk.service.host=localhost --Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.157"] +-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.158"] -Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid -Ddruid.db.connector.user=druid diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 95f53508448..7d07d713fb0 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.157 +cd druid-services-0.6.158 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md index 12a5ea405af..e4242e32cbc 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md @@ -91,7 +91,7 @@ druid.service=overlord druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.158"] druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid druid.db.connector.user=druid diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 5673a9f8f1f..51e585e40ba 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes and external depende If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -240,7 +240,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.157","io.druid.extensions:druid-kafka-seven:0.6.157"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.158","io.druid.extensions:druid-kafka-seven:0.6.158"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index 83dfd848924..f7893348ef6 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.157 +cd druid-services-0.6.158 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.md b/docs/content/Twitter-Tutorial.md index c8f2b2adc0e..dd69a952ae9 100644 --- a/docs/content/Twitter-Tutorial.md +++ b/docs/content/Twitter-Tutorial.md @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. # Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.157-bin.tar.gz). +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz). Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: diff --git a/pom.xml b/pom.xml index 5731b98d72a..62408db61aa 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.9 2.6.0 - 0.2.13 + 0.2.14 From dada60a301d3147c5bb483f8f4d0a8eb1e598e74 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 9 Oct 2014 14:24:48 -0700 Subject: [PATCH 22/69] [maven-release-plugin] prepare release druid-0.6.158 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index f976ec19663..24ba056e724 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/common/pom.xml b/common/pom.xml index 62a4c7efd0d..38be89a6821 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/examples/pom.xml b/examples/pom.xml index e4cf96e065e..2c4721fe102 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 2ef16b75cfe..83130405c55 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/histogram/pom.xml b/histogram/pom.xml index dd73d1a7ea3..7b357b03bf6 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 1268759137d..267451d224f 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 8031019b137..4d0f44893f6 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 3a30b97b1eb..86952798fa4 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 502517995c3..fdb82e37fcd 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/pom.xml b/pom.xml index 62408db61aa..2cc2e7754bd 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.158-SNAPSHOT + 0.6.158 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.131-SNAPSHOT + druid-0.6.158 diff --git a/processing/pom.xml b/processing/pom.xml index a6f09b1f608..ee57f65523d 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 291e333eab5..09dc0c7d46d 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index eeda5228469..0b58c6f5661 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/server/pom.xml b/server/pom.xml index 0fcf50829bd..9b6c8952ba0 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 diff --git a/services/pom.xml b/services/pom.xml index bd4d5d7a6d2..03bd30a8695 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.158-SNAPSHOT + 0.6.158 From d0fbd5f729c142f9482360bf41d3b969073e2330 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 9 Oct 2014 14:24:53 -0700 Subject: [PATCH 23/69] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 24ba056e724..795430b3d7a 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 38be89a6821..8b657d43b7b 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 2c4721fe102..e8e39269e2e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 83130405c55..a1fbad02890 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index 7b357b03bf6..2d13ca77b1b 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 267451d224f..7ae737f572f 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 4d0f44893f6..bb18a62d312 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 86952798fa4..d6fa98d745a 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index fdb82e37fcd..66f80d94371 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/pom.xml b/pom.xml index 2cc2e7754bd..f8c672c1c41 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.158 + 0.6.159-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.158 + druid-0.6.131-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index ee57f65523d..adfec8434f8 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 09dc0c7d46d..c7ffa9270ae 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 0b58c6f5661..9fcaf6349dd 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 9b6c8952ba0..fe3b96333e2 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 03bd30a8695..f9c32a441a2 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.158 + 0.6.159-SNAPSHOT From 592611f2ee72e5de037f7c38e7d46927ece32560 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 9 Oct 2014 17:04:36 -0700 Subject: [PATCH 24/69] fix cardinality aggregator serde --- .../CardinalityAggregatorFactory.java | 10 ++++++++-- .../cardinality/CardinalityAggregatorTest.java | 17 ++++++++++++++++- 2 files changed, 24 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java index b1db05a2284..071a91b3c3e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java @@ -63,12 +63,12 @@ public class CardinalityAggregatorFactory implements AggregatorFactory public CardinalityAggregatorFactory( @JsonProperty("name") String name, @JsonProperty("fieldNames") final List fieldNames, - @JsonProperty("byRow") final Boolean byRow + @JsonProperty("byRow") final boolean byRow ) { this.name = name; this.fieldNames = fieldNames; - this.byRow = byRow == null ? false : byRow; + this.byRow = byRow; } @Override @@ -203,6 +203,12 @@ public class CardinalityAggregatorFactory implements AggregatorFactory return fieldNames; } + @JsonProperty + public boolean isByRow() + { + return byRow; + } + @Override public byte[] getCacheKey() { diff --git a/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java index 4d27df07303..9d8ac7c721f 100644 --- a/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java @@ -19,16 +19,20 @@ package io.druid.query.aggregation.cardinality; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.BufferAggregator; import io.druid.segment.DimensionSelector; import io.druid.segment.data.IndexedInts; -import junit.framework.Assert; +import org.junit.Assert; import org.junit.Test; import javax.annotation.Nullable; @@ -378,4 +382,15 @@ public class CardinalityAggregatorTest 0.05 ); } + + @Test + public void testSerde() throws Exception + { + CardinalityAggregatorFactory factory = new CardinalityAggregatorFactory("billy", ImmutableList.of("b", "a", "c"), true); + ObjectMapper objectMapper = new DefaultObjectMapper(); + Assert.assertEquals( + factory, + objectMapper.readValue(objectMapper.writeValueAsString(factory), AggregatorFactory.class) + ); + } } From 830436594831724e676874df901c8f6645dcbd23 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 10 Oct 2014 10:19:33 -0700 Subject: [PATCH 25/69] move thanks page to main site --- docs/content/Thanks.md | 12 ------------ docs/content/toc.textile | 2 +- 2 files changed, 1 insertion(+), 13 deletions(-) delete mode 100644 docs/content/Thanks.md diff --git a/docs/content/Thanks.md b/docs/content/Thanks.md deleted file mode 100644 index 97ec7e0904a..00000000000 --- a/docs/content/Thanks.md +++ /dev/null @@ -1,12 +0,0 @@ ---- -layout: doc_page ---- - -YourKit supports the Druid open source projects with its -full-featured Java Profiler. -YourKit, LLC is the creator of innovative and intelligent tools for profiling -Java and .NET applications. Take a look at YourKit's software products: -YourKit Java -Profiler and -YourKit .NET -Profiler. diff --git a/docs/content/toc.textile b/docs/content/toc.textile index cc3c86eac99..62a9ec0776a 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -93,4 +93,4 @@ h2. Development * "Libraries":./Libraries.html h2. Misc -* "Thanks":./Thanks.html +* "Thanks":/thanks.html From 25cb69446e0d3d3cfdbb9a3647e3facae90409a5 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 10 Oct 2014 14:08:14 -0700 Subject: [PATCH 26/69] Updated druid-api version to always sort dimension names, these tests needed fixing --- .../ApproximateHistogramGroupByQueryTest.java | 15 +- .../ApproximateHistogramTopNQueryTest.java | 8 +- pom.xml | 4 +- .../input/ProtoBufInputRowParserTest.java | 1 + .../io/druid/query/QueryRunnerTestHelper.java | 2 +- .../query/groupby/GroupByQueryRunnerTest.java | 4 +- .../query/search/SearchQueryRunnerTest.java | 24 +-- .../query/select/SelectQueryRunnerTest.java | 8 +- .../timeseries/TimeseriesQueryRunnerTest.java | 48 ++--- .../druid/query/topn/TopNQueryRunnerTest.java | 200 +++++++++--------- .../io/druid/query/topn/TopNQueryTest.java | 4 +- .../druid/query/topn/TopNUnionQueryTest.java | 8 +- .../java/io/druid/segment/AppendTest.java | 38 ++-- .../io/druid/segment/SchemalessTestFull.java | 70 +++--- .../druid/segment/SchemalessTestSimple.java | 12 +- .../test/java/io/druid/segment/TestIndex.java | 4 +- processing/src/test/resources/append.json.1 | 4 +- processing/src/test/resources/append.json.2 | 6 +- processing/src/test/resources/append.json.3 | 4 +- processing/src/test/resources/append.json.4 | 6 +- processing/src/test/resources/append.json.5 | 12 +- processing/src/test/resources/append.json.6 | 4 +- processing/src/test/resources/append.json.7 | 4 +- .../src/test/resources/druid.sample.json | 14 +- .../test/resources/druid.sample.json.bottom | 10 +- .../src/test/resources/druid.sample.json.top | 4 +- 26 files changed, 256 insertions(+), 262 deletions(-) diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index cd7aa5a70c1..73f0a562c06 100644 --- a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -64,13 +64,6 @@ public class ApproximateHistogramGroupByQueryTest { private final QueryRunner runner; private GroupByQueryRunnerFactory factory; - private Supplier configSupplier; - - @Before - public void setUp() throws Exception - { - configSupplier = Suppliers.ofInstance(new GroupByQueryConfig()); - } @Parameterized.Parameters public static Collection constructorFeeder() throws IOException @@ -165,14 +158,14 @@ public class ApproximateHistogramGroupByQueryTest GroupByQuery query = new GroupByQuery.Builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setGranularity(QueryRunnerTestHelper.allGran) - .setDimensions(Arrays.asList(new LegacyDimensionSpec(QueryRunnerTestHelper.providerDimension))) + .setDimensions(Arrays.asList(new LegacyDimensionSpec(QueryRunnerTestHelper.marketDimension))) .setInterval(QueryRunnerTestHelper.fullOnInterval) .setLimitSpec( new DefaultLimitSpec( Lists.newArrayList( new OrderByColumnSpec( - QueryRunnerTestHelper.providerDimension, - OrderByColumnSpec.Direction.DESCENDING + QueryRunnerTestHelper.marketDimension, + OrderByColumnSpec.Direction.ASCENDING ) ), 1 ) @@ -201,7 +194,7 @@ public class ApproximateHistogramGroupByQueryTest List expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01T00:00:00.000Z", - "provider", "spot", + "market", "spot", "rows", 837L, "addRowsIndexConstant", 96444.5703125, "dependentPostAgg", 97282.5703125, diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java index e362b4ad9b2..25f58c064bf 100644 --- a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java @@ -113,7 +113,7 @@ public class ApproximateHistogramTopNQueryTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .metric(QueryRunnerTestHelper.dependentPostAggMetric) .threshold(4) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -144,7 +144,7 @@ public class ApproximateHistogramTopNQueryTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "total_market") + .put(QueryRunnerTestHelper.marketDimension, "total_market") .put("rows", 186L) .put("index", 215679.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D) @@ -175,7 +175,7 @@ public class ApproximateHistogramTopNQueryTest ) .build(), ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "upfront") + .put(QueryRunnerTestHelper.marketDimension, "upfront") .put("rows", 186L) .put("index", 192046.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D) @@ -206,7 +206,7 @@ public class ApproximateHistogramTopNQueryTest ) .build(), ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "spot") + .put(QueryRunnerTestHelper.marketDimension, "spot") .put("rows", 837L) .put("index", 95606.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D) diff --git a/pom.xml b/pom.xml index f8c672c1c41..382fa139925 100644 --- a/pom.xml +++ b/pom.xml @@ -30,7 +30,7 @@ scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.131-SNAPSHOT + druid-0.6.159-SNAPSHOT @@ -41,7 +41,7 @@ UTF-8 0.26.9 2.6.0 - 0.2.14 + 0.2.15 diff --git a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java index 67d5fdc3574..9a099e79366 100644 --- a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java +++ b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java @@ -85,6 +85,7 @@ public class ProtoBufInputRowParserTest InputRow row = parser.parse(ByteBuffer.wrap(out.toByteArray())); System.out.println(row); + Arrays.sort(DIMENSIONS); assertEquals(Arrays.asList(DIMENSIONS), row.getDimensions()); assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch()); diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index deeaff563e8..2027e1bc096 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -82,7 +82,7 @@ public class QueryRunnerTestHelper ); public static final QueryGranularity dayGran = QueryGranularity.DAY; public static final QueryGranularity allGran = QueryGranularity.ALL; - public static final String providerDimension = "proVider"; + public static final String marketDimension = "market"; public static final String qualityDimension = "quality"; public static final String placementDimension = "placement"; public static final String placementishDimension = "placementish"; diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 6beab76793d..5be3210e5ac 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -1291,14 +1291,14 @@ public class GroupByQueryRunnerTest .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) - .setDimFilter(new JavaScriptDimFilter("provider", "function(dim){ return true; }")) + .setDimFilter(new JavaScriptDimFilter("market", "function(dim){ return true; }")) .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, new DoubleSumAggregatorFactory("idx_subagg", "index"), new JavaScriptAggregatorFactory( "js_agg", - Arrays.asList("index", "provider"), + Arrays.asList("index", "market"), "function(current, index, dim){return current + index + dim.length;}", "function(){return 0;}", "function(a,b){return a + b;}" diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java index e5abe9b5d7f..ee833db7385 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java @@ -87,7 +87,7 @@ public class SearchQueryRunnerTest QueryRunnerTestHelper.qualityDimension, Sets.newHashSet("automotive", "mezzanine", "travel", "health", "entertainment") ); - expectedResults.put(QueryRunnerTestHelper.providerDimension.toLowerCase(), Sets.newHashSet("total_market")); + expectedResults.put(QueryRunnerTestHelper.marketDimension.toLowerCase(), Sets.newHashSet("total_market")); expectedResults.put(QueryRunnerTestHelper.placementishDimension, Sets.newHashSet("a")); checkSearchQuery(searchQuery, expectedResults); @@ -137,13 +137,13 @@ public class SearchQueryRunnerTest public void testSearchWithDimensionProvider() { Map> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); - expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet(Arrays.asList("total_market"))); + expectedResults.put(QueryRunnerTestHelper.marketDimension, new HashSet(Arrays.asList("total_market"))); checkSearchQuery( Druids.newSearchQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimensions("provider") + .dimensions("market") .intervals(QueryRunnerTestHelper.fullOnInterval) .query("a") .build(), @@ -163,7 +163,7 @@ public class SearchQueryRunnerTest "automotive", "mezzanine", "travel", "health", "entertainment" ) ), - QueryRunnerTestHelper.providerDimension, + QueryRunnerTestHelper.marketDimension, new HashSet( Arrays.asList("total_market") ) @@ -177,7 +177,7 @@ public class SearchQueryRunnerTest .dimensions( Arrays.asList( QueryRunnerTestHelper.qualityDimension, - QueryRunnerTestHelper.providerDimension + QueryRunnerTestHelper.marketDimension ) ) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -191,7 +191,7 @@ public class SearchQueryRunnerTest public void testSearchWithDimensionsPlacementAndProvider() { Map> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); - expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet(Arrays.asList("total_market"))); + expectedResults.put(QueryRunnerTestHelper.marketDimension, new HashSet(Arrays.asList("total_market"))); checkSearchQuery( Druids.newSearchQueryBuilder() @@ -200,7 +200,7 @@ public class SearchQueryRunnerTest .dimensions( Arrays.asList( QueryRunnerTestHelper.placementishDimension, - QueryRunnerTestHelper.providerDimension + QueryRunnerTestHelper.marketDimension ) ) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -235,15 +235,15 @@ public class SearchQueryRunnerTest public void testSearchWithSingleFilter2() { Map> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); - expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet(Arrays.asList("total_market"))); + expectedResults.put(QueryRunnerTestHelper.marketDimension, new HashSet(Arrays.asList("total_market"))); checkSearchQuery( Druids.newSearchQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .filters(QueryRunnerTestHelper.providerDimension, "total_market") + .filters(QueryRunnerTestHelper.marketDimension, "total_market") .intervals(QueryRunnerTestHelper.fullOnInterval) - .dimensions(QueryRunnerTestHelper.providerDimension) + .dimensions(QueryRunnerTestHelper.marketDimension) .query("a") .build(), expectedResults @@ -260,7 +260,7 @@ public class SearchQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newSelectorDimFilterBuilder() @@ -343,7 +343,7 @@ public class SearchQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("total_market") .build(), Druids.newSelectorDimFilterBuilder() diff --git a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java index b60683c6d38..e4779fb3df5 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java @@ -65,7 +65,7 @@ public class SelectQueryRunnerTest ); } - private static final String providerLowercase = "provider"; + private static final String providerLowercase = "market"; private final QueryRunner runner; @@ -271,7 +271,7 @@ public class SelectQueryRunnerTest SelectQuery query = new SelectQuery( new TableDataSource(QueryRunnerTestHelper.dataSource), new LegacySegmentSpec(new Interval("2011-01-12/2011-01-14")), - new SelectorDimFilter(QueryRunnerTestHelper.providerDimension, "spot"), + new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot"), QueryRunnerTestHelper.dayGran, Lists.newArrayList(QueryRunnerTestHelper.qualityDimension), Lists.newArrayList(QueryRunnerTestHelper.indexMetric), @@ -368,8 +368,8 @@ public class SelectQueryRunnerTest new LegacySegmentSpec(new Interval("2011-01-12/2011-01-14")), new AndDimFilter( Arrays.asList( - new SelectorDimFilter(QueryRunnerTestHelper.providerDimension, "spot"), - new SelectorDimFilter(QueryRunnerTestHelper.providerDimension, "foo") + new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot"), + new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "foo") ) ), QueryRunnerTestHelper.allGran, diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java index c2e3889cd38..d4110c7b266 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -87,7 +87,7 @@ public class TimeseriesQueryRunnerTest QueryGranularity gran = QueryGranularity.DAY; TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(QueryRunnerTestHelper.dayGran) + .granularity(gran) .intervals(QueryRunnerTestHelper.fullOnInterval) .aggregators( Arrays.asList( @@ -193,7 +193,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.dayGran) - .filters(QueryRunnerTestHelper.providerDimension, "upfront") + .filters(QueryRunnerTestHelper.marketDimension, "upfront") .intervals(QueryRunnerTestHelper.fullOnInterval) .aggregators( Arrays.asList( @@ -205,7 +205,7 @@ public class TimeseriesQueryRunnerTest Assert.assertEquals( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("upfront") .build(), query.getDimensionsFilter() @@ -418,7 +418,7 @@ public class TimeseriesQueryRunnerTest { TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .filters(QueryRunnerTestHelper.providerDimension, "spot", "upfront", "total_market") + .filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market") .granularity( new PeriodGranularity( new Period("P7D"), @@ -471,7 +471,7 @@ public class TimeseriesQueryRunnerTest { TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .filters(QueryRunnerTestHelper.providerDimension, "spot", "upfront", "total_market") + .filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market") .granularity(QueryGranularity.HOUR) .intervals( Arrays.asList( @@ -539,7 +539,7 @@ public class TimeseriesQueryRunnerTest { TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .filters(QueryRunnerTestHelper.providerDimension, "spot", "upfront", "total_market") + .filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market") .granularity( new PeriodGranularity( new Period("PT1H"), @@ -586,7 +586,7 @@ public class TimeseriesQueryRunnerTest { TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .filters(QueryRunnerTestHelper.providerDimension, "spot", "upfront", "total_market") + .filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market") .granularity(new PeriodGranularity(new Period("P1M"), null, null)) .intervals( Arrays.asList( @@ -624,7 +624,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query2 = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .filters(QueryRunnerTestHelper.providerDimension, "spot", "upfront", "total_market") + .filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market") .granularity("DAY") .intervals( Arrays.asList( @@ -702,7 +702,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.dayGran) - .filters(QueryRunnerTestHelper.providerDimension, "spot", "upfront", "total_market") + .filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market") .intervals(QueryRunnerTestHelper.firstToThird) .aggregators( Arrays.asList( @@ -752,7 +752,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.dayGran) - .filters(new RegexDimFilter(QueryRunnerTestHelper.providerDimension, "^.p.*$")) // spot and upfront + .filters(new RegexDimFilter(QueryRunnerTestHelper.marketDimension, "^.p.*$")) // spot and upfront .intervals(QueryRunnerTestHelper.firstToThird) .aggregators( Arrays.asList( @@ -802,7 +802,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.dayGran) - .filters(QueryRunnerTestHelper.providerDimension, "spot") + .filters(QueryRunnerTestHelper.marketDimension, "spot") .intervals(QueryRunnerTestHelper.firstToThird) .aggregators( Arrays.asList( @@ -852,7 +852,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.dayGran) - .filters(QueryRunnerTestHelper.providerDimension, "upfront") + .filters(QueryRunnerTestHelper.marketDimension, "upfront") .intervals(QueryRunnerTestHelper.firstToThird) .aggregators( Arrays.asList( @@ -902,7 +902,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.dayGran) - .filters(QueryRunnerTestHelper.providerDimension, "total_market") + .filters(QueryRunnerTestHelper.marketDimension, "total_market") .intervals(QueryRunnerTestHelper.firstToThird) .aggregators( Arrays.asList( @@ -954,7 +954,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newOrDimFilterBuilder() @@ -1011,7 +1011,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newSelectorDimFilterBuilder() @@ -1069,7 +1069,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newSelectorDimFilterBuilder() @@ -1127,7 +1127,7 @@ public class TimeseriesQueryRunnerTest .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.dayGran) .filters( - QueryRunnerTestHelper.providerDimension, + QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market", @@ -1183,7 +1183,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newOrDimFilterBuilder() @@ -1288,7 +1288,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.dayGran) - .filters(QueryRunnerTestHelper.providerDimension, "billy") + .filters(QueryRunnerTestHelper.marketDimension, "billy") .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) @@ -1333,7 +1333,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("billy") .build(), Druids.newSelectorDimFilterBuilder() @@ -1533,7 +1533,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newSelectorDimFilterBuilder() @@ -1556,7 +1556,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newSelectorDimFilterBuilder() @@ -1594,7 +1594,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newOrDimFilterBuilder() @@ -1616,7 +1616,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newOrDimFilterBuilder() diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index 09d383168cf..d155c35f91a 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -104,7 +104,7 @@ public class TopNQueryRunnerTest this.runner = runner; } - private static final String providerDimension = "provider"; + private static final String marketDimension = "market"; @Test public void testFullOnTopN() @@ -112,7 +112,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -136,7 +136,7 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put(providerDimension, "total_market") + .put(marketDimension, "total_market") .put("rows", 186L) .put("index", 215679.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D) @@ -145,7 +145,7 @@ public class TopNQueryRunnerTest .put("minIndex", 792.3260498046875D) .build(), ImmutableMap.builder() - .put(providerDimension, "upfront") + .put(marketDimension, "upfront") .put("rows", 186L) .put("index", 192046.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D) @@ -154,7 +154,7 @@ public class TopNQueryRunnerTest .put("minIndex", 545.9906005859375D) .build(), ImmutableMap.builder() - .put(providerDimension, "spot") + .put(marketDimension, "spot") .put("rows", 837L) .put("index", 95606.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D) @@ -176,7 +176,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.addRowsIndexConstantMetric) .threshold(4) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -200,7 +200,7 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put(providerDimension, "total_market") + .put(marketDimension, "total_market") .put("rows", 186L) .put("index", 215679.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D) @@ -209,7 +209,7 @@ public class TopNQueryRunnerTest .put("minIndex", 792.3260498046875D) .build(), ImmutableMap.builder() - .put(providerDimension, "upfront") + .put(marketDimension, "upfront") .put("rows", 186L) .put("index", 192046.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D) @@ -218,7 +218,7 @@ public class TopNQueryRunnerTest .put("minIndex", 545.9906005859375D) .build(), ImmutableMap.builder() - .put(providerDimension, "spot") + .put(marketDimension, "spot") .put("rows", 837L) .put("index", 95606.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D) @@ -241,7 +241,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.uniqueMetric) .threshold(3) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -265,7 +265,7 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 837L) .put("index", 95606.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D) @@ -274,7 +274,7 @@ public class TopNQueryRunnerTest .put("minIndex", 59.02102279663086D) .build(), ImmutableMap.builder() - .put("provider", "total_market") + .put("market", "total_market") .put("rows", 186L) .put("index", 215679.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D) @@ -283,7 +283,7 @@ public class TopNQueryRunnerTest .put("minIndex", 792.3260498046875D) .build(), ImmutableMap.builder() - .put("provider", "upfront") + .put("market", "upfront") .put("rows", 186L) .put("index", 192046.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D) @@ -306,7 +306,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -321,21 +321,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "spot", + marketDimension, "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, @@ -355,7 +355,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(new NumericTopNMetricSpec("uniques")) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -370,21 +370,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - "provider", "spot", + "market", "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( - "provider", "total_market", + "market", "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - "provider", "upfront", + "market", "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -404,8 +404,8 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .filters(providerDimension, "total_market", "upfront", "spot") - .dimension(providerDimension) + .filters(marketDimension, "total_market", "upfront", "spot") + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -419,21 +419,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "spot", + marketDimension, "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, @@ -453,8 +453,8 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .filters(providerDimension, "total_market", "upfront") - .dimension(providerDimension) + .filters(marketDimension, "total_market", "upfront") + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -468,14 +468,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -495,8 +495,8 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .filters(providerDimension, "upfront") - .dimension(providerDimension) + .filters(marketDimension, "upfront") + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -510,7 +510,7 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -531,7 +531,7 @@ public class TopNQueryRunnerTest .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) .filters(QueryRunnerTestHelper.qualityDimension, "mezzanine") - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -545,21 +545,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 2L, "index", 2591.68359375D, "addRowsIndexConstant", 2594.68359375D, "uniques", QueryRunnerTestHelper.UNIQUES_1 ), ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 2L, "index", 2508.39599609375D, "addRowsIndexConstant", 2511.39599609375D, "uniques", QueryRunnerTestHelper.UNIQUES_1 ), ImmutableMap.of( - providerDimension, "spot", + marketDimension, "spot", "rows", 2L, "index", 220.63774871826172D, "addRowsIndexConstant", 223.63774871826172D, @@ -580,7 +580,7 @@ public class TopNQueryRunnerTest .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) .filters(QueryRunnerTestHelper.qualityDimension, "mezzanine") - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals( @@ -598,21 +598,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 1L, "index", new Float(1447.341160).doubleValue(), "addRowsIndexConstant", new Float(1449.341160).doubleValue(), "uniques", QueryRunnerTestHelper.UNIQUES_1 ), ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 1L, "index", new Float(1314.839715).doubleValue(), "addRowsIndexConstant", new Float(1316.839715).doubleValue(), "uniques", QueryRunnerTestHelper.UNIQUES_1 ), ImmutableMap.of( - providerDimension, "spot", + marketDimension, "spot", "rows", 1L, "index", new Float(109.705815).doubleValue(), "addRowsIndexConstant", new Float(111.705815).doubleValue(), @@ -632,8 +632,8 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .filters(providerDimension, "total_market", "upfront", "billyblank") - .dimension(providerDimension) + .filters(marketDimension, "total_market", "upfront", "billyblank") + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -647,14 +647,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -674,8 +674,8 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .filters(providerDimension, "billyblank") - .dimension(providerDimension) + .filters(marketDimension, "billyblank") + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -701,7 +701,7 @@ public class TopNQueryRunnerTest .fields( Lists.newArrayList( Druids.newSelectorDimFilterBuilder() - .dimension(providerDimension) + .dimension(marketDimension) .value("billyblank") .build(), Druids.newSelectorDimFilterBuilder() @@ -714,7 +714,7 @@ public class TopNQueryRunnerTest .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) .filters(andDimFilter) - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -740,7 +740,7 @@ public class TopNQueryRunnerTest .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) .filters(QueryRunnerTestHelper.placementishDimension, "m") - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -755,7 +755,7 @@ public class TopNQueryRunnerTest .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) .filters(QueryRunnerTestHelper.qualityDimension, "mezzanine") - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -955,7 +955,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(new LexicographicTopNMetricSpec("")) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -969,21 +969,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "spot", + marketDimension, "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -1003,7 +1003,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(new LexicographicTopNMetricSpec("spot")) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -1017,14 +1017,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -1044,7 +1044,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(new LexicographicTopNMetricSpec("t")) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -1058,14 +1058,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -1085,7 +1085,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("upfront"))) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -1099,14 +1099,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "spot", + marketDimension, "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, @@ -1126,7 +1126,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("u"))) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -1140,14 +1140,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "spot", + marketDimension, "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, @@ -1169,7 +1169,7 @@ public class TopNQueryRunnerTest .granularity(QueryRunnerTestHelper.allGran) .dimension( new ExtractionDimensionSpec( - providerDimension, providerDimension, new RegexDimExtractionFn("(.)") + marketDimension, marketDimension, new RegexDimExtractionFn("(.)") ) ) .metric("rows") @@ -1185,21 +1185,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "s", + marketDimension, "s", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( - providerDimension, "t", + marketDimension, "t", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "u", + marketDimension, "u", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -1221,7 +1221,7 @@ public class TopNQueryRunnerTest .granularity(QueryRunnerTestHelper.allGran) .dimension( new ExtractionDimensionSpec( - providerDimension, providerDimension, new RegexDimExtractionFn("(.)") + marketDimension, marketDimension, new RegexDimExtractionFn("(.)") ) ) .metric(new LexicographicTopNMetricSpec(null)) @@ -1237,21 +1237,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "s", + marketDimension, "s", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( - providerDimension, "t", + marketDimension, "t", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "u", + marketDimension, "u", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -1273,7 +1273,7 @@ public class TopNQueryRunnerTest .granularity(QueryRunnerTestHelper.allGran) .dimension( new ExtractionDimensionSpec( - providerDimension, providerDimension, new RegexDimExtractionFn("..(.)") + marketDimension, marketDimension, new RegexDimExtractionFn("..(.)") ) ) .metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec(null))) @@ -1289,21 +1289,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "t", + marketDimension, "t", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "o", + marketDimension, "o", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( - providerDimension, "f", + marketDimension, "f", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -1325,7 +1325,7 @@ public class TopNQueryRunnerTest .granularity(QueryRunnerTestHelper.allGran) .dimension( new ExtractionDimensionSpec( - providerDimension, providerDimension, new RegexDimExtractionFn("(.)") + marketDimension, marketDimension, new RegexDimExtractionFn("(.)") ) ) .metric(new LexicographicTopNMetricSpec("spot")) @@ -1341,14 +1341,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "t", + marketDimension, "t", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "u", + marketDimension, "u", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -1371,7 +1371,7 @@ public class TopNQueryRunnerTest .granularity(QueryRunnerTestHelper.allGran) .dimension( new ExtractionDimensionSpec( - providerDimension, providerDimension, new RegexDimExtractionFn("(.)") + marketDimension, marketDimension, new RegexDimExtractionFn("(.)") ) ) .metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("u"))) @@ -1387,14 +1387,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "t", + marketDimension, "t", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "s", + marketDimension, "s", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, @@ -1416,7 +1416,7 @@ public class TopNQueryRunnerTest .granularity(QueryRunnerTestHelper.allGran) .dimension( new ExtractionDimensionSpec( - providerDimension, providerDimension, new RegexDimExtractionFn("..(.)") + marketDimension, marketDimension, new RegexDimExtractionFn("..(.)") ) ) .metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("p"))) @@ -1432,14 +1432,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "o", + marketDimension, "o", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( - providerDimension, "f", + marketDimension, "f", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -1460,7 +1460,7 @@ public class TopNQueryRunnerTest new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(new InvertedTopNMetricSpec(new NumericTopNMetricSpec(QueryRunnerTestHelper.indexMetric))) .threshold(3) .intervals(QueryRunnerTestHelper.firstToThird) @@ -1474,21 +1474,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "spot", + marketDimension, "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, @@ -1508,7 +1508,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.dependentPostAggMetric) .threshold(4) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -1538,7 +1538,7 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put(providerDimension, "total_market") + .put(marketDimension, "total_market") .put("rows", 186L) .put("index", 215679.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D) @@ -1552,7 +1552,7 @@ public class TopNQueryRunnerTest ) .build(), ImmutableMap.builder() - .put(providerDimension, "upfront") + .put(marketDimension, "upfront") .put("rows", 186L) .put("index", 192046.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D) @@ -1566,7 +1566,7 @@ public class TopNQueryRunnerTest ) .build(), ImmutableMap.builder() - .put(providerDimension, "spot") + .put(marketDimension, "spot") .put("rows", 837L) .put("index", 95606.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D) @@ -1593,7 +1593,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .metric(QueryRunnerTestHelper.dependentPostAggMetric) .threshold(4) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -1619,7 +1619,7 @@ public class TopNQueryRunnerTest TopNResultValue topNResult = new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "total_market") + .put(QueryRunnerTestHelper.marketDimension, "total_market") .put("rows", 186L) .put("index", 215679.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D) @@ -1629,7 +1629,7 @@ public class TopNQueryRunnerTest .put("minIndex", 792.3260498046875D) .build(), ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "upfront") + .put(QueryRunnerTestHelper.marketDimension, "upfront") .put("rows", 186L) .put("index", 192046.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D) @@ -1639,7 +1639,7 @@ public class TopNQueryRunnerTest .put("minIndex", 545.9906005859375D) .build(), ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "spot") + .put(QueryRunnerTestHelper.marketDimension, "spot") .put("rows", 837L) .put("index", 95606.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D) diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java index f2f4ac22f3d..cfe6978e61b 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java @@ -42,7 +42,7 @@ import static io.druid.query.QueryRunnerTestHelper.commonAggregators; import static io.druid.query.QueryRunnerTestHelper.dataSource; import static io.druid.query.QueryRunnerTestHelper.fullOnInterval; import static io.druid.query.QueryRunnerTestHelper.indexMetric; -import static io.druid.query.QueryRunnerTestHelper.providerDimension; +import static io.druid.query.QueryRunnerTestHelper.marketDimension; public class TopNQueryTest { @@ -54,7 +54,7 @@ public class TopNQueryTest Query query = new TopNQueryBuilder() .dataSource(dataSource) .granularity(allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(indexMetric) .threshold(4) .intervals(fullOnInterval) diff --git a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java index 7dc7b645cad..d9975f39153 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java @@ -98,7 +98,7 @@ public class TopNUnionQueryTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.unionDataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .metric(QueryRunnerTestHelper.dependentPostAggMetric) .threshold(4) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -128,7 +128,7 @@ public class TopNUnionQueryTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "total_market") + .put(QueryRunnerTestHelper.marketDimension, "total_market") .put("rows", 744L) .put("index", 862719.3151855469D) .put("addRowsIndexConstant", 863464.3151855469D) @@ -142,7 +142,7 @@ public class TopNUnionQueryTest ) .build(), ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "upfront") + .put(QueryRunnerTestHelper.marketDimension, "upfront") .put("rows", 744L) .put("index", 768184.4240722656D) .put("addRowsIndexConstant", 768929.4240722656D) @@ -156,7 +156,7 @@ public class TopNUnionQueryTest ) .build(), ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "spot") + .put(QueryRunnerTestHelper.marketDimension, "spot") .put("rows", 3348L) .put("index", 382426.28929138184D) .put("addRowsIndexConstant", 385775.28929138184D) diff --git a/processing/src/test/java/io/druid/segment/AppendTest.java b/processing/src/test/java/io/druid/segment/AppendTest.java index 13a37ff2e1a..1ddb8122f94 100644 --- a/processing/src/test/java/io/druid/segment/AppendTest.java +++ b/processing/src/test/java/io/druid/segment/AppendTest.java @@ -79,7 +79,7 @@ public class AppendTest final QueryGranularity allGran = QueryGranularity.ALL; final String dimensionValue = "dimension"; final String valueValue = "value"; - final String providerDimension = "provider"; + final String marketDimension = "market"; final String qualityDimension = "quality"; final String placementDimension = "placement"; final String placementishDimension = "placementish"; @@ -309,7 +309,7 @@ public class AppendTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 3L) .put("index", 300.0D) .put("addRowsIndexConstant", 304.0D) @@ -319,7 +319,7 @@ public class AppendTest .build(), new HashMap() {{ - put("provider", null); + put("market", null); put("rows", 3L); put("index", 200.0D); put("addRowsIndexConstant", 204.0D); @@ -328,7 +328,7 @@ public class AppendTest put("minIndex", 0.0); }}, ImmutableMap.builder() - .put("provider", "total_market") + .put("market", "total_market") .put("rows", 2L) .put("index", 200.0D) .put("addRowsIndexConstant", 203.0D) @@ -355,7 +355,7 @@ public class AppendTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "total_market") + .put("market", "total_market") .put("rows", 3L) .put("index", 300.0D) .put("addRowsIndexConstant", 304.0D) @@ -365,7 +365,7 @@ public class AppendTest .build(), new HashMap() {{ - put("provider", null); + put("market", null); put("rows", 3L); put("index", 100.0D); put("addRowsIndexConstant", 104.0D); @@ -374,7 +374,7 @@ public class AppendTest put("minIndex", 0.0); }}, ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -401,7 +401,7 @@ public class AppendTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -447,7 +447,7 @@ public class AppendTest new SearchHit(placementishDimension, "a"), new SearchHit(qualityDimension, "automotive"), new SearchHit(placementDimension, "mezzanine"), - new SearchHit(providerDimension, "total_market") + new SearchHit(marketDimension, "total_market") ) ) ) @@ -468,7 +468,7 @@ public class AppendTest Arrays.asList( new SearchHit(placementishDimension, "a"), new SearchHit(placementDimension, "mezzanine"), - new SearchHit(providerDimension, "total_market") + new SearchHit(marketDimension, "total_market") ) ) ) @@ -488,7 +488,7 @@ public class AppendTest new SearchResultValue( Arrays.asList( new SearchHit(placementDimension, "mezzanine"), - new SearchHit(providerDimension, "total_market") + new SearchHit(marketDimension, "total_market") ) ) ) @@ -509,7 +509,7 @@ public class AppendTest Arrays.asList( new SearchHit(placementishDimension, "a"), new SearchHit(placementDimension, "mezzanine"), - new SearchHit(providerDimension, "total_market") + new SearchHit(marketDimension, "total_market") ) ) ) @@ -543,7 +543,7 @@ public class AppendTest .dataSource(dataSource) .granularity(allGran) .intervals(fullOnInterval) - .filters(providerDimension, "breakstuff") + .filters(marketDimension, "breakstuff") .aggregators( Lists.newArrayList( Iterables.concat( @@ -593,11 +593,11 @@ public class AppendTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(providerDimension) + .dimension(marketDimension) .value("spot") .build(), Druids.newSelectorDimFilterBuilder() - .dimension(providerDimension) + .dimension(marketDimension) .value("total_market") .build() ) @@ -623,7 +623,7 @@ public class AppendTest return new TopNQueryBuilder() .dataSource(dataSource) .granularity(allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(indexMetric) .threshold(3) .intervals(fullOnInterval) @@ -647,7 +647,7 @@ public class AppendTest return new TopNQueryBuilder() .dataSource(dataSource) .granularity(allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(indexMetric) .threshold(3) .filters( @@ -655,7 +655,7 @@ public class AppendTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(providerDimension) + .dimension(marketDimension) .value("spot") .build(), Druids.newSelectorDimFilterBuilder() @@ -699,7 +699,7 @@ public class AppendTest Druids.newNotDimFilterBuilder() .field( Druids.newSelectorDimFilterBuilder() - .dimension(providerDimension) + .dimension(marketDimension) .value("spot") .build() ).build() diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java index 83f6a62aa99..90b138aca5a 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java @@ -71,7 +71,7 @@ public class SchemalessTestFull final QueryGranularity allGran = QueryGranularity.ALL; final String dimensionValue = "dimension"; final String valueValue = "value"; - final String providerDimension = "provider"; + final String marketDimension = "market"; final String qualityDimension = "quality"; final String placementDimension = "placement"; final String placementishDimension = "placementish"; @@ -133,7 +133,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -142,7 +142,7 @@ public class SchemalessTestFull .put("minIndex", 100.0) .build(), ImmutableMap.builder() - .put("provider", "total_market") + .put("market", "total_market") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -161,7 +161,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -182,7 +182,7 @@ public class SchemalessTestFull new SearchHit(placementishDimension, "a"), new SearchHit(qualityDimension, "automotive"), new SearchHit(placementDimension, "mezzanine"), - new SearchHit(providerDimension, "total_market") + new SearchHit(marketDimension, "total_market") ) ) ) @@ -268,7 +268,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "") + .put("market", "") .put("rows", 2L) .put("index", 200.0D) .put("addRowsIndexConstant", 203.0D) @@ -277,7 +277,7 @@ public class SchemalessTestFull .put("minIndex", 100.0) .build(), ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -296,7 +296,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "") + .put("market", "") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -305,7 +305,7 @@ public class SchemalessTestFull .put("minIndex", 100.0) .build(), ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -407,7 +407,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -417,7 +417,7 @@ public class SchemalessTestFull .build(), new HashMap() {{ - put("provider", null); + put("market", null); put("rows", 1L); put("index", 0.0D); put("addRowsIndexConstant", 2.0D); @@ -436,7 +436,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -540,7 +540,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -549,7 +549,7 @@ public class SchemalessTestFull .put("minIndex", 100.0) .build(), ImmutableMap.builder() - .put("provider", "total_market") + .put("market", "total_market") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -568,7 +568,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -587,7 +587,7 @@ public class SchemalessTestFull new SearchResultValue( Arrays.asList( new SearchHit(qualityDimension, "automotive"), - new SearchHit(providerDimension, "total_market") + new SearchHit(marketDimension, "total_market") ) ) ) @@ -658,7 +658,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 2L) .put("index", 200.0D) .put("addRowsIndexConstant", 203.0D) @@ -755,7 +755,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -765,7 +765,7 @@ public class SchemalessTestFull .build(), new HashMap() {{ - put("provider", null); + put("market", null); put("rows", 1L); put("index", 0.0D); put("addRowsIndexConstant", 2.0D); @@ -784,7 +784,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -942,7 +942,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 200.0D) .put("addRowsIndexConstant", 202.0D) @@ -1042,7 +1042,7 @@ public class SchemalessTestFull Arrays.>asList( new HashMap() {{ - put("provider", null); + put("market", null); put("rows", 2L); put("index", 200.0D); put("addRowsIndexConstant", 203.0D); @@ -1051,7 +1051,7 @@ public class SchemalessTestFull put("minIndex", 100.0); }}, ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -1070,7 +1070,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -1175,7 +1175,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 4L) .put("index", 400.0D) .put("addRowsIndexConstant", 405.0D) @@ -1184,7 +1184,7 @@ public class SchemalessTestFull .put("minIndex", 100.0) .build(), ImmutableMap.builder() - .put("provider", "") + .put("market", "") .put("rows", 3L) .put("index", 200.0D) .put("addRowsIndexConstant", 204.0D) @@ -1193,7 +1193,7 @@ public class SchemalessTestFull .put("minIndex", 0.0) .build(), ImmutableMap.builder() - .put("provider", "total_market") + .put("market", "total_market") .put("rows", 2L) .put("index", 200.0D) .put("addRowsIndexConstant", 203.0D) @@ -1212,7 +1212,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 4L) .put("index", 400.0D) .put("addRowsIndexConstant", 405.0D) @@ -1221,7 +1221,7 @@ public class SchemalessTestFull .put("minIndex", 100.0) .build(), ImmutableMap.builder() - .put("provider", "") + .put("market", "") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -1242,7 +1242,7 @@ public class SchemalessTestFull new SearchHit(placementishDimension, "a"), new SearchHit(qualityDimension, "automotive"), new SearchHit(placementDimension, "mezzanine"), - new SearchHit(providerDimension, "total_market") + new SearchHit(marketDimension, "total_market") ) ) ) @@ -1403,7 +1403,7 @@ public class SchemalessTestFull .dataSource(dataSource) .granularity(allGran) .intervals(fullOnInterval) - .filters(providerDimension, "spot") + .filters(marketDimension, "spot") .aggregators( Lists.newArrayList( Iterables.concat( @@ -1432,7 +1432,7 @@ public class SchemalessTestFull TopNQuery query = new TopNQueryBuilder() .dataSource(dataSource) .granularity(allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(indexMetric) .threshold(3) .intervals(fullOnInterval) @@ -1464,8 +1464,8 @@ public class SchemalessTestFull TopNQuery query = new TopNQueryBuilder() .dataSource(dataSource) .granularity(allGran) - .dimension(providerDimension) - .filters(providerDimension, "spot") + .dimension(marketDimension) + .filters(marketDimension, "spot") .metric(indexMetric) .threshold(3) .intervals(fullOnInterval) @@ -1513,7 +1513,7 @@ public class SchemalessTestFull SearchQuery query = Druids.newSearchQueryBuilder() .dataSource(dataSource) .granularity(allGran) - .filters(providerDimension, "spot") + .filters(marketDimension, "spot") .intervals(fullOnInterval) .query("a") .build(); diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java b/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java index 85226b3256d..524aa5b8668 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java @@ -97,7 +97,7 @@ public class SchemalessTestSimple final QueryGranularity allGran = QueryGranularity.ALL; final String dimensionValue = "dimension"; final String valueValue = "value"; - final String providerDimension = "provider"; + final String marketDimension = "market"; final String qualityDimension = "quality"; final String placementDimension = "placement"; final String placementishDimension = "placementish"; @@ -175,7 +175,7 @@ public class SchemalessTestSimple TopNQuery query = new TopNQueryBuilder() .dataSource(dataSource) .granularity(allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(indexMetric) .threshold(3) .intervals(fullOnInterval) @@ -200,7 +200,7 @@ public class SchemalessTestSimple Arrays.asList( new DimensionAndMetricValueExtractor( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 4L) .put("index", 400.0D) .put("addRowsIndexConstant", 405.0D) @@ -211,7 +211,7 @@ public class SchemalessTestSimple ), new DimensionAndMetricValueExtractor( ImmutableMap.builder() - .put("provider", "") + .put("market", "") .put("rows", 2L) .put("index", 200.0D) .put("addRowsIndexConstant", 203.0D) @@ -222,7 +222,7 @@ public class SchemalessTestSimple ), new DimensionAndMetricValueExtractor( ImmutableMap.builder() - .put("provider", "total_market") + .put("market", "total_market") .put("rows", 2L) .put("index", 200.0D) .put("addRowsIndexConstant", 203.0D) @@ -258,7 +258,7 @@ public class SchemalessTestSimple new SearchHit(placementishDimension, "a"), new SearchHit(qualityDimension, "automotive"), new SearchHit(placementDimension, "mezzanine"), - new SearchHit(providerDimension, "total_market") + new SearchHit(marketDimension, "total_market") ) ) ) diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index bb83dac77ce..bba10185dee 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -59,14 +59,14 @@ public class TestIndex public static final String[] COLUMNS = new String[]{ "ts", - "provider", + "market", "quALIty", "plAcEmEnT", "pLacementish", "iNdEx", "qualiTy_Uniques" }; - public static final String[] DIMENSIONS = new String[]{"provider", "quALIty", "plAcEmEnT", "pLacementish"}; + public static final String[] DIMENSIONS = new String[]{"market", "quALIty", "plAcEmEnT", "pLacementish"}; public static final String[] METRICS = new String[]{"iNdEx"}; private static final Interval DATA_INTERVAL = new Interval("2011-01-12T00:00:00.000Z/2011-05-01T00:00:00.000Z"); private static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ diff --git a/processing/src/test/resources/append.json.1 b/processing/src/test/resources/append.json.1 index aa55a99694a..efce43f58e3 100644 --- a/processing/src/test/resources/append.json.1 +++ b/processing/src/test/resources/append.json.1 @@ -4,7 +4,7 @@ }, { "timestamp":"2011-01-13T00:00:00.000Z", - "provider":"spot", + "market":"spot", "quality":"automotive", "placement":"preferred", "placementish":["a", "preferred"], @@ -12,7 +12,7 @@ }, { "timestamp":"2011-01-14T00:00:00.000Z", - "provider":"spot", + "market":"spot", "quality":"automotive", "index":100.000000 }, diff --git a/processing/src/test/resources/append.json.2 b/processing/src/test/resources/append.json.2 index fd3f916e9ec..88d7501132d 100644 --- a/processing/src/test/resources/append.json.2 +++ b/processing/src/test/resources/append.json.2 @@ -1,14 +1,14 @@ [ { "timestamp":"2011-01-14T22:00:00.000Z", - "provider":"total_market", + "market":"total_market", "placement":"preferred", "placementish":["h", "preferred"], "index":100.000000 }, { "timestamp":"2011-01-14T23:00:00.000Z", - "provider":"total_market", + "market":"total_market", "quality":"business", "placement":"mezzanine", "placementish":["p", "preferred"], @@ -17,7 +17,7 @@ { "timestamp":"2011-01-15T00:00:00.000Z", "placementish":"preferred", - "provider":"spot", + "market":"spot", "index":100.000000 }, { diff --git a/processing/src/test/resources/append.json.3 b/processing/src/test/resources/append.json.3 index 7a963dc4e78..de15d125969 100644 --- a/processing/src/test/resources/append.json.3 +++ b/processing/src/test/resources/append.json.3 @@ -4,7 +4,7 @@ }, { "timestamp":"2011-01-13T00:00:00.000Z", - "provider":"spot", + "market":"spot", "quality":"automotive", "placement":"preferred", "placementish":["a", "preferred"], @@ -12,7 +12,7 @@ }, { "timestamp":"2011-01-14T00:00:00.000Z", - "provider":"total_market", + "market":"total_market", "index":100.000000 }, { diff --git a/processing/src/test/resources/append.json.4 b/processing/src/test/resources/append.json.4 index 5049fdc5e69..1a8d7d5d8d9 100644 --- a/processing/src/test/resources/append.json.4 +++ b/processing/src/test/resources/append.json.4 @@ -1,14 +1,14 @@ [ { "timestamp":"2011-01-13T00:00:00.000Z", - "provider":"total_market", + "market":"total_market", "placement":"preferred", "placementish":["h", "preferred"], "index":100.000000 }, { "timestamp":"2011-01-13T01:00:00.000Z", - "provider":"total_market", + "market":"total_market", "placement":"mezzanine", "placementish":["p", "preferred"], "index":100.000000 @@ -16,7 +16,7 @@ { "timestamp":"2011-01-13T15:00:00.000Z", "placementish":"preferred", - "provider":"spot", + "market":"spot", "index":100.000000 }, { diff --git a/processing/src/test/resources/append.json.5 b/processing/src/test/resources/append.json.5 index 0119fead368..a59d23c13c5 100644 --- a/processing/src/test/resources/append.json.5 +++ b/processing/src/test/resources/append.json.5 @@ -1,32 +1,32 @@ [ { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":"breakstuff", + "market":"breakstuff", "index":100.000000 }, { "timestamp":"2011-01-14T00:00:00.000Z", - "provider":"breakstuff", + "market":"breakstuff", "index":100.000000 }, { "timestamp":"2011-01-16T00:00:00.000Z", - "provider":"breakstuff", + "market":"breakstuff", "index":100.000000 }, { "timestamp":"2011-01-17T00:00:00.000Z", - "provider":"breakstuff", + "market":"breakstuff", "index":100.000000 }, { "timestamp":"2011-01-19T00:00:00.000Z", - "provider":"breakstuff", + "market":"breakstuff", "index":100.000000 }, { "timestamp":"2011-01-21T00:00:00.000Z", - "provider":"breakstuff", + "market":"breakstuff", "index":100.000000 } ] \ No newline at end of file diff --git a/processing/src/test/resources/append.json.6 b/processing/src/test/resources/append.json.6 index 386769ce0ef..56ff8a3faad 100644 --- a/processing/src/test/resources/append.json.6 +++ b/processing/src/test/resources/append.json.6 @@ -1,12 +1,12 @@ [ { "timestamp":"2011-01-13T00:00:00.000Z", - "provider":"breakstuff", + "market":"breakstuff", "index":100.000000 }, { "timestamp":"2011-01-15T00:00:00.000Z", - "provider":"spot", + "market":"spot", "index":100.000000 } ] \ No newline at end of file diff --git a/processing/src/test/resources/append.json.7 b/processing/src/test/resources/append.json.7 index 8645df52b8f..cc8089f8981 100644 --- a/processing/src/test/resources/append.json.7 +++ b/processing/src/test/resources/append.json.7 @@ -1,12 +1,12 @@ [ { "timestamp":"2011-01-18T00:00:00.000Z", - "provider":"spot", + "market":"spot", "index":100.000000 }, { "timestamp":"2011-01-20T00:00:00.000Z", - "provider":"spot", + "market":"spot", "index":100.000000 } ] \ No newline at end of file diff --git a/processing/src/test/resources/druid.sample.json b/processing/src/test/resources/druid.sample.json index e188e7be089..a3cddb46ff0 100644 --- a/processing/src/test/resources/druid.sample.json +++ b/processing/src/test/resources/druid.sample.json @@ -4,7 +4,7 @@ }, { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":"spot", + "market":"spot", "quality":"automotive", "placement":"preferred", "placementish":["a", "preferred"], @@ -12,7 +12,7 @@ }, { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":"spot", + "market":"spot", "quality":"automotive", "index":100.000000 }, @@ -23,14 +23,14 @@ }, { "timestamp":"2011-01-13T00:00:00.000Z", - "provider":"total_market", + "market":"total_market", "placementish":["h", "preferred"], "placement":"preferred", "index":100.000000 }, { "timestamp":"2011-01-13T00:00:00.000Z", - "provider":"total_market", + "market":"total_market", "placementish":["p", "preferred"], "quality":"business", "placement":"mezzanine", @@ -39,7 +39,7 @@ { "timestamp":"2011-01-12T00:00:00.000Z", "placementish":"preferred", - "provider":"spot", + "market":"spot", "index":100.000000 }, { @@ -55,12 +55,12 @@ }, { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":"", + "market":"", "index":100.000000 }, { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":["", "spot"], + "market":["", "spot"], "index":100.000000 } ] \ No newline at end of file diff --git a/processing/src/test/resources/druid.sample.json.bottom b/processing/src/test/resources/druid.sample.json.bottom index a7b3b648a2a..75ea3502c6e 100644 --- a/processing/src/test/resources/druid.sample.json.bottom +++ b/processing/src/test/resources/druid.sample.json.bottom @@ -6,14 +6,14 @@ }, { "timestamp":"2011-01-13T00:00:00.000Z", - "provider":"total_market", + "market":"total_market", "placement":"preferred", "placementish":["h", "preferred"], "index":100.000000 }, { "timestamp":"2011-01-13T00:00:00.000Z", - "provider":"total_market", + "market":"total_market", "quality":"business", "placement":"mezzanine", "placementish":["p", "preferred"], @@ -22,7 +22,7 @@ { "timestamp":"2011-01-12T00:00:00.000Z", "placementish":"preferred", - "provider":"spot", + "market":"spot", "index":100.000000 }, { @@ -38,12 +38,12 @@ }, { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":"", + "market":"", "index":100.000000 }, { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":["", "spot"], + "market":["", "spot"], "index":100.000000 } ] \ No newline at end of file diff --git a/processing/src/test/resources/druid.sample.json.top b/processing/src/test/resources/druid.sample.json.top index 8439291578d..4d1834ca27a 100644 --- a/processing/src/test/resources/druid.sample.json.top +++ b/processing/src/test/resources/druid.sample.json.top @@ -4,7 +4,7 @@ }, { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":"spot", + "market":"spot", "quality":"automotive", "placement":"preferred", "placementish":["a", "preferred"], @@ -12,7 +12,7 @@ }, { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":"spot", + "market":"spot", "quality":"automotive", "index":100.000000 } From e1fedbe7415aedc092d624abcef1c91cff3af194 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Sun, 12 Oct 2014 11:26:46 -0700 Subject: [PATCH 27/69] RemoteTaskRunner should respect worker version changes (fixes #787). --- .../indexing/overlord/ImmutableZkWorker.java | 2 +- .../indexing/overlord/RemoteTaskRunner.java | 28 ++++++++++++++++- .../io/druid/indexing/overlord/ZkWorker.java | 28 ++++++++--------- .../overlord/RemoteTaskRunnerTest.java | 31 +++++++++++++++++++ 4 files changed, 72 insertions(+), 17 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ImmutableZkWorker.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ImmutableZkWorker.java index f657b6a2435..f78f576de67 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ImmutableZkWorker.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ImmutableZkWorker.java @@ -32,7 +32,7 @@ public class ImmutableZkWorker { private final Worker worker; private final int currCapacityUsed; - private final Set availabilityGroups; + private final ImmutableSet availabilityGroups; public ImmutableZkWorker(Worker worker, int currCapacityUsed, Set availabilityGroups) { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index e9bba479c64..e382c6dee3c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -164,7 +164,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer @Override public void childEvent(CuratorFramework client, final PathChildrenCacheEvent event) throws Exception { - Worker worker; + final Worker worker; switch (event.getType()) { case CHILD_ADDED: worker = jsonMapper.readValue( @@ -198,6 +198,14 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer } ); break; + case CHILD_UPDATED: + worker = jsonMapper.readValue( + event.getData().getData(), + Worker.class + ); + updateWorker(worker); + break; + case CHILD_REMOVED: worker = jsonMapper.readValue( event.getData().getData(), @@ -745,6 +753,24 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer } } + /** + * We allow workers to change their own capacities and versions. They cannot change their own hosts or ips without + * dropping themselves and re-announcing. + */ + private void updateWorker(final Worker worker) + { + final ZkWorker zkWorker = zkWorkers.get(worker.getHost()); + if (zkWorker != null) { + log.info("Worker[%s] updated its announcement from[%s] to[%s].", worker.getHost(), zkWorker.getWorker(), worker); + zkWorker.setWorker(worker); + } else { + log.warn( + "WTF, worker[%s] updated its announcement but we didn't have a ZkWorker for it. Ignoring.", + worker.getHost() + ); + } + } + /** * When a ephemeral worker node disappears from ZK, incomplete running tasks will be retried by * the logic in the status listener. We still have to make sure there are no tasks assigned diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java index abc4da0ad57..54b09da2b29 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java @@ -22,11 +22,11 @@ package io.druid.indexing.overlord; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; +import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import io.druid.indexing.common.task.Task; import io.druid.indexing.worker.TaskAnnouncement; import io.druid.indexing.worker.Worker; import org.apache.curator.framework.recipes.cache.ChildData; @@ -46,15 +46,15 @@ import java.util.concurrent.atomic.AtomicReference; */ public class ZkWorker implements Closeable { - private final Worker worker; private final PathChildrenCache statusCache; private final Function cacheConverter; + private AtomicReference worker; private AtomicReference lastCompletedTaskTime = new AtomicReference(new DateTime()); public ZkWorker(Worker worker, PathChildrenCache statusCache, final ObjectMapper jsonMapper) { - this.worker = worker; + this.worker = new AtomicReference<>(worker); this.statusCache = statusCache; this.cacheConverter = new Function() { @@ -84,7 +84,7 @@ public class ZkWorker implements Closeable @JsonProperty("worker") public Worker getWorker() { - return worker; + return worker.get(); } @JsonProperty("runningTasks") @@ -137,30 +137,28 @@ public class ZkWorker implements Closeable return getRunningTasks().containsKey(taskId); } - public boolean isAtCapacity() - { - return getCurrCapacityUsed() >= worker.getCapacity(); - } - public boolean isValidVersion(String minVersion) { - return worker.getVersion().compareTo(minVersion) >= 0; + return worker.get().getVersion().compareTo(minVersion) >= 0; } - public boolean canRunTask(Task task) + public void setWorker(Worker newWorker) { - return (worker.getCapacity() - getCurrCapacityUsed() >= task.getTaskResource().getRequiredCapacity() - && !getAvailabilityGroups().contains(task.getTaskResource().getAvailabilityGroup())); + final Worker oldWorker = worker.get(); + Preconditions.checkArgument(newWorker.getHost().equals(oldWorker.getHost()), "Cannot change Worker host"); + Preconditions.checkArgument(newWorker.getIp().equals(oldWorker.getIp()), "Cannot change Worker ip"); + + worker.set(newWorker); } public void setLastCompletedTaskTime(DateTime completedTaskTime) { - lastCompletedTaskTime.getAndSet(completedTaskTime); + lastCompletedTaskTime.set(completedTaskTime); } public ImmutableZkWorker toImmutable() { - return new ImmutableZkWorker(worker, getCurrCapacityUsed(), getAvailabilityGroups()); + return new ImmutableZkWorker(worker.get(), getCurrCapacityUsed(), getAvailabilityGroups()); } @Override diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java index 11c7c85c639..b5f86ea7494 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java @@ -361,6 +361,29 @@ public class RemoteTaskRunnerTest Assert.assertEquals(TaskStatus.Status.FAILED, status.getStatusCode()); } + @Test + public void testWorkerDisabled() throws Exception + { + doSetup(); + final ListenableFuture result = remoteTaskRunner.run(task); + + Assert.assertTrue(taskAnnounced(task.getId())); + mockWorkerRunningTask(task); + Assert.assertTrue(workerRunningTask(task.getId())); + + // Disable while task running + disableWorker(); + + // Continue test + mockWorkerCompleteSuccessfulTask(task); + Assert.assertTrue(workerCompletedTask(result)); + Assert.assertEquals(task.getId(), result.get().getId()); + Assert.assertEquals(TaskStatus.Status.SUCCESS, result.get().getStatusCode()); + + // Confirm RTR thinks the worker is disabled. + Assert.assertEquals("", Iterables.getOnlyElement(remoteTaskRunner.getWorkers()).getWorker().getVersion()); + } + private void doSetup() throws Exception { makeWorker(); @@ -405,6 +428,14 @@ public class RemoteTaskRunnerTest ); } + private void disableWorker() throws Exception + { + cf.setData().forPath( + announcementsPath, + jsonMapper.writeValueAsBytes(new Worker(worker.getHost(), worker.getIp(), worker.getCapacity(), "")) + ); + } + private boolean taskAnnounced(final String taskId) { return pathExists(joiner.join(tasksPath, taskId)); From 282e40e07ce7ebb6de4d03f199e1dd9b1e89176f Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 13 Oct 2014 11:10:17 -0700 Subject: [PATCH 28/69] update bytebuffer collections dep to fix issue with empty spatial concise sets --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index f8c672c1c41..d1caceabeac 100644 --- a/pom.xml +++ b/pom.xml @@ -89,7 +89,7 @@ com.metamx bytebuffer-collections - 0.0.2 + 0.0.4 com.metamx @@ -194,7 +194,7 @@ it.uniroma3.mat extendedset - 1.3.4 + 1.3.7 com.google.guava From d4f5a71dedb93a7f9ce82a25a6139d87dd9a3cbc Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 13 Oct 2014 12:48:12 -0700 Subject: [PATCH 29/69] prepare for next release --- docs/content/Examples.md | 4 ++-- docs/content/Kafka-Eight.md | 4 ++-- docs/content/Production-Cluster-Configuration.md | 6 +++--- docs/content/Realtime-Config.md | 4 ++-- docs/content/Router.md | 2 +- docs/content/Simple-Cluster-Configuration.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-1.md | 2 +- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.md | 2 +- 11 files changed, 20 insertions(+), 20 deletions(-) diff --git a/docs/content/Examples.md b/docs/content/Examples.md index b61168e20dc..dfd44ffe927 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.158 +git checkout druid-0.6.159 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.159-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Kafka-Eight.md b/docs/content/Kafka-Eight.md index b7ae4e5c4ab..5819a931b2a 100644 --- a/docs/content/Kafka-Eight.md +++ b/docs/content/Kafka-Eight.md @@ -8,9 +8,9 @@ The previous examples are for Kafka 7. To support Kafka 8, a couple changes need - Update realtime node's configs for Kafka 8 extensions - e.g. - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.158",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.159",...]` - becomes - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.158",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.159",...]` - Update realtime task config for changed keys - `firehose.type`, `plumber.rejectionPolicyFactory`, and all of `firehose.consumerProps` changes. diff --git a/docs/content/Production-Cluster-Configuration.md b/docs/content/Production-Cluster-Configuration.md index 70b4cb4036d..6fc319712de 100644 --- a/docs/content/Production-Cluster-Configuration.md +++ b/docs/content/Production-Cluster-Configuration.md @@ -57,7 +57,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/overlord -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.159"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -139,7 +139,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/middlemanager -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158","io.druid.extensions:druid-kafka-seven:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.159","io.druid.extensions:druid-kafka-seven:0.6.159"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -286,7 +286,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/historical -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.159"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Realtime-Config.md b/docs/content/Realtime-Config.md index c61a22b736d..12ab55ff8e3 100644 --- a/docs/content/Realtime-Config.md +++ b/docs/content/Realtime-Config.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.159"] druid.zk.service.host=localhost @@ -76,7 +76,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/realtime -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158","io.druid.extensions:druid-kafka-seven:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.159","io.druid.extensions:druid-kafka-seven:0.6.159"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Router.md b/docs/content/Router.md index 2a077e944fb..73849164216 100644 --- a/docs/content/Router.md +++ b/docs/content/Router.md @@ -51,7 +51,7 @@ druid.service=druid/prod/router druid.extensions.remoteRepositories=[] druid.extensions.localRepository=lib -druid.extensions.coordinates=["io.druid.extensions:druid-histogram:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-histogram:0.6.159"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Simple-Cluster-Configuration.md b/docs/content/Simple-Cluster-Configuration.md index 98871836233..8bfade9ab9a 100644 --- a/docs/content/Simple-Cluster-Configuration.md +++ b/docs/content/Simple-Cluster-Configuration.md @@ -28,7 +28,7 @@ Configuration: -Ddruid.zk.service.host=localhost --Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.158"] +-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.159"] -Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid -Ddruid.db.connector.user=druid diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 7d07d713fb0..4a74b4f5b20 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.159-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.158 +cd druid-services-0.6.159 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md index e4242e32cbc..7ef8648cc0f 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md @@ -91,7 +91,7 @@ druid.service=overlord druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.159"] druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid druid.db.connector.user=druid diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 51e585e40ba..3797746ca20 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes and external depende If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.159-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.159"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -240,7 +240,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.158","io.druid.extensions:druid-kafka-seven:0.6.158"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.159","io.druid.extensions:druid-kafka-seven:0.6.159"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index f7893348ef6..47ebec7e049 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.159-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.158 +cd druid-services-0.6.159 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.md b/docs/content/Twitter-Tutorial.md index dd69a952ae9..95993732449 100644 --- a/docs/content/Twitter-Tutorial.md +++ b/docs/content/Twitter-Tutorial.md @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. # Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.158-bin.tar.gz). +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.159-bin.tar.gz). Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: From 7fd1747ffa8eade09704e32eaf3a6d80a19c33d6 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 13 Oct 2014 12:50:41 -0700 Subject: [PATCH 30/69] [maven-release-plugin] prepare release druid-0.6.159 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 795430b3d7a..74863effc38 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/common/pom.xml b/common/pom.xml index 8b657d43b7b..d48011e3434 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/examples/pom.xml b/examples/pom.xml index e8e39269e2e..c59a68a7772 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index a1fbad02890..546a3ec9a27 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/histogram/pom.xml b/histogram/pom.xml index 2d13ca77b1b..5159554cea2 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 7ae737f572f..32a00aa3a37 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index bb18a62d312..4a085554c10 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index d6fa98d745a..aa519cb387c 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 66f80d94371..9e491f6548c 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/pom.xml b/pom.xml index d1caceabeac..5c597957197 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.159-SNAPSHOT + 0.6.159 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.131-SNAPSHOT + druid-0.6.159 diff --git a/processing/pom.xml b/processing/pom.xml index adfec8434f8..f001fad2c83 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index c7ffa9270ae..13946216979 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 9fcaf6349dd..c5ce3df8727 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/server/pom.xml b/server/pom.xml index fe3b96333e2..be472dc473a 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 diff --git a/services/pom.xml b/services/pom.xml index f9c32a441a2..c7dfe120611 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.159-SNAPSHOT + 0.6.159 From a4c8f0440949c474bed0b7d4862aafd04380185f Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 13 Oct 2014 12:50:45 -0700 Subject: [PATCH 31/69] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 74863effc38..5682138b49d 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index d48011e3434..da437e99e1f 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index c59a68a7772..fb2597326e7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 546a3ec9a27..6cb8f84483e 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index 5159554cea2..a26ce5f5601 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 32a00aa3a37..cfd2e8da529 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 4a085554c10..5fe0ebae569 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index aa519cb387c..be87d5b4d79 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 9e491f6548c..a26e940b725 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/pom.xml b/pom.xml index 5c597957197..8c1202d7b79 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.159 + 0.6.160-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.159 + druid-0.6.131-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index f001fad2c83..867b720ca57 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 13946216979..745f10e88ac 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index c5ce3df8727..e4914713493 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index be472dc473a..740e06d56ad 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index c7dfe120611..9968d0b6146 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.159 + 0.6.160-SNAPSHOT From 35716e8deef7428a4933c587453585bc1532269d Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 13 Oct 2014 14:33:45 -0700 Subject: [PATCH 32/69] add a best practices doc --- docs/content/Best-Practices.md | 23 +++++++++++++++++++++++ docs/content/toc.textile | 1 + 2 files changed, 24 insertions(+) create mode 100644 docs/content/Best-Practices.md diff --git a/docs/content/Best-Practices.md b/docs/content/Best-Practices.md new file mode 100644 index 00000000000..f4cd8f32595 --- /dev/null +++ b/docs/content/Best-Practices.md @@ -0,0 +1,23 @@ +--- +layout: doc_page +--- + +Best Practices +============== + +# Use UTC Timezone + +We recommend using UTC timezone for all your events and across on your nodes, not just for Druid, but for all data infrastructure. This can greatly mitigate potential query problems with inconsistent timezones. + +# Use Lowercase Strings for Column Names + +Druid is not perfect in how it handles mix-cased dimension and metric names. This will hopefully change very soon but for the time being, lower casing your column names is recommended. + +# SSDs + +SSDs are highly recommended for historical and real-time nodes if you are not running a cluster that is entirely in memory. SSDs can greatly mitigate the time required to page data in and out of memory. + +# Provide Columns Names in Lexicographic Order for Best Results + +Although Druid supports schemaless ingestion of dimensions, because of https://github.com/metamx/druid/issues/658, you may sometimes get bigger segments than necessary. To ensure segments are as compact as possible, providing dimension names in lexicographic order is recommended. This may require some ETL processing on your data however. + diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 62a9ec0776a..21f867bca36 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -19,6 +19,7 @@ h2. Booting a Druid Cluster * "Production Cluster Configuration":Production-Cluster-Configuration.html * "Production Hadoop Configuration":Hadoop-Configuration.html * "Rolling Cluster Updates":Rolling-Updates.html +* "Best Practices":Best-Practices.html h2. Configuration * "Common Configuration":Configuration.html From 62f77fabc46f51870588c93ae9de6a10876c506b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 13 Oct 2014 17:45:34 -0400 Subject: [PATCH 33/69] simplify test for approx histogram --- .../ApproximateHistogramGroupByQueryTest.java | 60 +++++++------------ 1 file changed, 23 insertions(+), 37 deletions(-) diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index cd7aa5a70c1..6d6aade79cb 100644 --- a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -30,12 +30,9 @@ import io.druid.data.input.Row; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; -import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.MaxAggregatorFactory; -import io.druid.query.aggregation.MinAggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; -import io.druid.query.dimension.LegacyDimensionSpec; import io.druid.query.groupby.GroupByQuery; import io.druid.query.groupby.GroupByQueryConfig; import io.druid.query.groupby.GroupByQueryEngine; @@ -165,13 +162,20 @@ public class ApproximateHistogramGroupByQueryTest GroupByQuery query = new GroupByQuery.Builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setGranularity(QueryRunnerTestHelper.allGran) - .setDimensions(Arrays.asList(new LegacyDimensionSpec(QueryRunnerTestHelper.providerDimension))) + .setDimensions( + Arrays.asList( + new DefaultDimensionSpec( + QueryRunnerTestHelper.providerDimension, + "provider" + ) + ) + ) .setInterval(QueryRunnerTestHelper.fullOnInterval) .setLimitSpec( new DefaultLimitSpec( Lists.newArrayList( new OrderByColumnSpec( - QueryRunnerTestHelper.providerDimension, + "provider", OrderByColumnSpec.Direction.DESCENDING ) ), 1 @@ -179,20 +183,12 @@ public class ApproximateHistogramGroupByQueryTest ) .setAggregatorSpecs( Lists.newArrayList( - Iterables.concat( - QueryRunnerTestHelper.commonAggregators, - Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index"), - aggFactory - ) - ) + QueryRunnerTestHelper.rowsCount, + aggFactory ) ) .setPostAggregatorSpecs( - Arrays.asList( - QueryRunnerTestHelper.addRowsIndexConstant, - QueryRunnerTestHelper.dependentPostAgg, + Arrays.asList( new QuantilePostAggregator("quantile", "apphisto", 0.5f) ) ) @@ -201,31 +197,21 @@ public class ApproximateHistogramGroupByQueryTest List expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01T00:00:00.000Z", - "provider", "spot", - "rows", 837L, - "addRowsIndexConstant", 96444.5703125, - "dependentPostAgg", 97282.5703125, - "index", 95606.5703125, - "maxIndex", 277.2735290527344, - "minIndex", 59.02102279663086, - "quantile", 101.78856f, - "uniques", QueryRunnerTestHelper.UNIQUES_9, + "provider", "upfront", + "rows", 186L, + "quantile", 880.9881f, "apphisto", new Histogram( new float[]{ - 4.457897186279297f, - 59.02102279663086f, - 113.58415222167969f, - 168.14727783203125f, - 222.7104034423828f, - 277.2735290527344f + 214.97299194335938f, + 545.9906005859375f, + 877.0081787109375f, + 1208.0257568359375f, + 1539.0433349609375f, + 1870.06103515625f }, new double[]{ - 0.0, - 462.4309997558594, - 357.5404968261719, - 15.022850036621094, - 2.0056631565093994 + 0.0, 67.53287506103516, 72.22068786621094, 31.984678268432617, 14.261756896972656 } ) ) From c18f156cd8ab710eb8ceaabce99aa283815efcc1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 13 Oct 2014 17:53:24 -0400 Subject: [PATCH 34/69] remove unused test code --- .../histogram/ApproximateHistogramGroupByQueryTest.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index 6d6aade79cb..cfd1b37835e 100644 --- a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -42,7 +42,6 @@ import io.druid.query.groupby.GroupByQueryRunnerTestHelper; import io.druid.query.groupby.orderby.DefaultLimitSpec; import io.druid.query.groupby.orderby.OrderByColumnSpec; import io.druid.segment.TestHelper; -import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -61,13 +60,6 @@ public class ApproximateHistogramGroupByQueryTest { private final QueryRunner runner; private GroupByQueryRunnerFactory factory; - private Supplier configSupplier; - - @Before - public void setUp() throws Exception - { - configSupplier = Suppliers.ofInstance(new GroupByQueryConfig()); - } @Parameterized.Parameters public static Collection constructorFeeder() throws IOException From 5b194ee5fe154bfa462fc8d65a03ce361f919cc9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 14 Oct 2014 09:22:19 -0400 Subject: [PATCH 35/69] fix groupBy ordering casing bug --- .../ApproximateHistogramGroupByQueryTest.java | 6 +-- .../groupby/orderby/DefaultLimitSpec.java | 2 +- .../query/groupby/GroupByQueryRunnerTest.java | 42 +++++++++++++++++++ 3 files changed, 46 insertions(+), 4 deletions(-) diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index cfd1b37835e..d623b9b6649 100644 --- a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -158,7 +158,7 @@ public class ApproximateHistogramGroupByQueryTest Arrays.asList( new DefaultDimensionSpec( QueryRunnerTestHelper.providerDimension, - "provider" + "proViderAlias" ) ) ) @@ -167,7 +167,7 @@ public class ApproximateHistogramGroupByQueryTest new DefaultLimitSpec( Lists.newArrayList( new OrderByColumnSpec( - "provider", + "proViderAlias", OrderByColumnSpec.Direction.DESCENDING ) ), 1 @@ -189,7 +189,7 @@ public class ApproximateHistogramGroupByQueryTest List expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01T00:00:00.000Z", - "provider", "upfront", + "provideralias", "upfront", "rows", 186L, "quantile", 880.9881f, "apphisto", diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java index bf6fc138ad4..ad96280a925 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java @@ -173,7 +173,7 @@ public class DefaultLimitSpec implements LimitSpec public String apply(Row input) { // Multi-value dimensions have all been flattened at this point; - final List dimList = input.getDimension(dimension); + final List dimList = input.getDimension(dimension.toLowerCase()); return dimList.isEmpty() ? null : dimList.get(0); } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 6beab76793d..1456df7fcb9 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -733,6 +733,48 @@ public class GroupByQueryRunnerTest ); } + @Test + public void testGroupByWithMixedCasingOrdering() + { + GroupByQuery query = new GroupByQuery.Builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setGranularity(QueryRunnerTestHelper.allGran) + .setDimensions( + Arrays.asList( + new DefaultDimensionSpec( + QueryRunnerTestHelper.providerDimension, + "ProviderAlias" + ) + ) + ) + .setInterval(QueryRunnerTestHelper.fullOnInterval) + .setLimitSpec( + new DefaultLimitSpec( + Lists.newArrayList( + new OrderByColumnSpec( + "providerALIAS", + OrderByColumnSpec.Direction.DESCENDING + ) + ), 3 + ) + ) + .setAggregatorSpecs( + Lists.newArrayList( + QueryRunnerTestHelper.rowsCount + ) + ) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "provideralias", "upfront", "rows", 186L), + GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "provideralias", "total_market", "rows", 186L), + GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "provideralias", "spot", "rows", 837L) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, "order-limit"); + } + @Test public void testHavingSpec() { From bb9043286238f0e67537ab8b6cd610b720455df8 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 10 Oct 2014 14:08:14 -0700 Subject: [PATCH 36/69] Updated druid-api version to always sort dimension names, these tests needed fixing --- .../ApproximateHistogramGroupByQueryTest.java | 8 +- .../ApproximateHistogramTopNQueryTest.java | 8 +- pom.xml | 4 +- .../input/ProtoBufInputRowParserTest.java | 1 + .../io/druid/query/QueryRunnerTestHelper.java | 2 +- .../query/groupby/GroupByQueryRunnerTest.java | 16 +- .../query/search/SearchQueryRunnerTest.java | 24 +-- .../query/select/SelectQueryRunnerTest.java | 8 +- .../timeseries/TimeseriesQueryRunnerTest.java | 48 ++--- .../druid/query/topn/TopNQueryRunnerTest.java | 200 +++++++++--------- .../io/druid/query/topn/TopNQueryTest.java | 4 +- .../druid/query/topn/TopNUnionQueryTest.java | 8 +- .../java/io/druid/segment/AppendTest.java | 38 ++-- .../io/druid/segment/SchemalessTestFull.java | 70 +++--- .../druid/segment/SchemalessTestSimple.java | 12 +- .../test/java/io/druid/segment/TestIndex.java | 4 +- processing/src/test/resources/append.json.1 | 4 +- processing/src/test/resources/append.json.2 | 6 +- processing/src/test/resources/append.json.3 | 4 +- processing/src/test/resources/append.json.4 | 6 +- processing/src/test/resources/append.json.5 | 12 +- processing/src/test/resources/append.json.6 | 4 +- processing/src/test/resources/append.json.7 | 4 +- .../src/test/resources/druid.sample.json | 14 +- .../test/resources/druid.sample.json.bottom | 10 +- .../src/test/resources/druid.sample.json.top | 4 +- 26 files changed, 262 insertions(+), 261 deletions(-) diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index d623b9b6649..1c7b23b3a3f 100644 --- a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -157,8 +157,8 @@ public class ApproximateHistogramGroupByQueryTest .setDimensions( Arrays.asList( new DefaultDimensionSpec( - QueryRunnerTestHelper.providerDimension, - "proViderAlias" + QueryRunnerTestHelper.marketDimension, + "marKetAlias" ) ) ) @@ -167,7 +167,7 @@ public class ApproximateHistogramGroupByQueryTest new DefaultLimitSpec( Lists.newArrayList( new OrderByColumnSpec( - "proViderAlias", + "marKetAlias", OrderByColumnSpec.Direction.DESCENDING ) ), 1 @@ -189,7 +189,7 @@ public class ApproximateHistogramGroupByQueryTest List expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01T00:00:00.000Z", - "provideralias", "upfront", + "marketalias", "upfront", "rows", 186L, "quantile", 880.9881f, "apphisto", diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java index e362b4ad9b2..25f58c064bf 100644 --- a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java @@ -113,7 +113,7 @@ public class ApproximateHistogramTopNQueryTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .metric(QueryRunnerTestHelper.dependentPostAggMetric) .threshold(4) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -144,7 +144,7 @@ public class ApproximateHistogramTopNQueryTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "total_market") + .put(QueryRunnerTestHelper.marketDimension, "total_market") .put("rows", 186L) .put("index", 215679.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D) @@ -175,7 +175,7 @@ public class ApproximateHistogramTopNQueryTest ) .build(), ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "upfront") + .put(QueryRunnerTestHelper.marketDimension, "upfront") .put("rows", 186L) .put("index", 192046.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D) @@ -206,7 +206,7 @@ public class ApproximateHistogramTopNQueryTest ) .build(), ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "spot") + .put(QueryRunnerTestHelper.marketDimension, "spot") .put("rows", 837L) .put("index", 95606.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D) diff --git a/pom.xml b/pom.xml index 8c1202d7b79..0c6a4b34aa8 100644 --- a/pom.xml +++ b/pom.xml @@ -30,7 +30,7 @@ scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.131-SNAPSHOT + druid-0.6.159-SNAPSHOT @@ -41,7 +41,7 @@ UTF-8 0.26.9 2.6.0 - 0.2.14 + 0.2.15 diff --git a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java index 67d5fdc3574..9a099e79366 100644 --- a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java +++ b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java @@ -85,6 +85,7 @@ public class ProtoBufInputRowParserTest InputRow row = parser.parse(ByteBuffer.wrap(out.toByteArray())); System.out.println(row); + Arrays.sort(DIMENSIONS); assertEquals(Arrays.asList(DIMENSIONS), row.getDimensions()); assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch()); diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index deeaff563e8..bb0556f4757 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -82,7 +82,7 @@ public class QueryRunnerTestHelper ); public static final QueryGranularity dayGran = QueryGranularity.DAY; public static final QueryGranularity allGran = QueryGranularity.ALL; - public static final String providerDimension = "proVider"; + public static final String marketDimension = "marKet"; public static final String qualityDimension = "quality"; public static final String placementDimension = "placement"; public static final String placementishDimension = "placementish"; diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 1456df7fcb9..772d3b0d18d 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -742,8 +742,8 @@ public class GroupByQueryRunnerTest .setDimensions( Arrays.asList( new DefaultDimensionSpec( - QueryRunnerTestHelper.providerDimension, - "ProviderAlias" + QueryRunnerTestHelper.marketDimension, + "MarketAlias" ) ) ) @@ -752,7 +752,7 @@ public class GroupByQueryRunnerTest new DefaultLimitSpec( Lists.newArrayList( new OrderByColumnSpec( - "providerALIAS", + "marketALIAS", OrderByColumnSpec.Direction.DESCENDING ) ), 3 @@ -766,9 +766,9 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "provideralias", "upfront", "rows", 186L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "provideralias", "total_market", "rows", 186L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "provideralias", "spot", "rows", 837L) + GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "marketalias", "upfront", "rows", 186L), + GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "marketalias", "total_market", "rows", 186L), + GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "marketalias", "spot", "rows", 837L) ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -1333,14 +1333,14 @@ public class GroupByQueryRunnerTest .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) - .setDimFilter(new JavaScriptDimFilter("provider", "function(dim){ return true; }")) + .setDimFilter(new JavaScriptDimFilter("market", "function(dim){ return true; }")) .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, new DoubleSumAggregatorFactory("idx_subagg", "index"), new JavaScriptAggregatorFactory( "js_agg", - Arrays.asList("index", "provider"), + Arrays.asList("index", "market"), "function(current, index, dim){return current + index + dim.length;}", "function(){return 0;}", "function(a,b){return a + b;}" diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java index e5abe9b5d7f..ee833db7385 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java @@ -87,7 +87,7 @@ public class SearchQueryRunnerTest QueryRunnerTestHelper.qualityDimension, Sets.newHashSet("automotive", "mezzanine", "travel", "health", "entertainment") ); - expectedResults.put(QueryRunnerTestHelper.providerDimension.toLowerCase(), Sets.newHashSet("total_market")); + expectedResults.put(QueryRunnerTestHelper.marketDimension.toLowerCase(), Sets.newHashSet("total_market")); expectedResults.put(QueryRunnerTestHelper.placementishDimension, Sets.newHashSet("a")); checkSearchQuery(searchQuery, expectedResults); @@ -137,13 +137,13 @@ public class SearchQueryRunnerTest public void testSearchWithDimensionProvider() { Map> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); - expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet(Arrays.asList("total_market"))); + expectedResults.put(QueryRunnerTestHelper.marketDimension, new HashSet(Arrays.asList("total_market"))); checkSearchQuery( Druids.newSearchQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimensions("provider") + .dimensions("market") .intervals(QueryRunnerTestHelper.fullOnInterval) .query("a") .build(), @@ -163,7 +163,7 @@ public class SearchQueryRunnerTest "automotive", "mezzanine", "travel", "health", "entertainment" ) ), - QueryRunnerTestHelper.providerDimension, + QueryRunnerTestHelper.marketDimension, new HashSet( Arrays.asList("total_market") ) @@ -177,7 +177,7 @@ public class SearchQueryRunnerTest .dimensions( Arrays.asList( QueryRunnerTestHelper.qualityDimension, - QueryRunnerTestHelper.providerDimension + QueryRunnerTestHelper.marketDimension ) ) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -191,7 +191,7 @@ public class SearchQueryRunnerTest public void testSearchWithDimensionsPlacementAndProvider() { Map> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); - expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet(Arrays.asList("total_market"))); + expectedResults.put(QueryRunnerTestHelper.marketDimension, new HashSet(Arrays.asList("total_market"))); checkSearchQuery( Druids.newSearchQueryBuilder() @@ -200,7 +200,7 @@ public class SearchQueryRunnerTest .dimensions( Arrays.asList( QueryRunnerTestHelper.placementishDimension, - QueryRunnerTestHelper.providerDimension + QueryRunnerTestHelper.marketDimension ) ) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -235,15 +235,15 @@ public class SearchQueryRunnerTest public void testSearchWithSingleFilter2() { Map> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); - expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet(Arrays.asList("total_market"))); + expectedResults.put(QueryRunnerTestHelper.marketDimension, new HashSet(Arrays.asList("total_market"))); checkSearchQuery( Druids.newSearchQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .filters(QueryRunnerTestHelper.providerDimension, "total_market") + .filters(QueryRunnerTestHelper.marketDimension, "total_market") .intervals(QueryRunnerTestHelper.fullOnInterval) - .dimensions(QueryRunnerTestHelper.providerDimension) + .dimensions(QueryRunnerTestHelper.marketDimension) .query("a") .build(), expectedResults @@ -260,7 +260,7 @@ public class SearchQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newSelectorDimFilterBuilder() @@ -343,7 +343,7 @@ public class SearchQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("total_market") .build(), Druids.newSelectorDimFilterBuilder() diff --git a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java index b60683c6d38..e4779fb3df5 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java @@ -65,7 +65,7 @@ public class SelectQueryRunnerTest ); } - private static final String providerLowercase = "provider"; + private static final String providerLowercase = "market"; private final QueryRunner runner; @@ -271,7 +271,7 @@ public class SelectQueryRunnerTest SelectQuery query = new SelectQuery( new TableDataSource(QueryRunnerTestHelper.dataSource), new LegacySegmentSpec(new Interval("2011-01-12/2011-01-14")), - new SelectorDimFilter(QueryRunnerTestHelper.providerDimension, "spot"), + new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot"), QueryRunnerTestHelper.dayGran, Lists.newArrayList(QueryRunnerTestHelper.qualityDimension), Lists.newArrayList(QueryRunnerTestHelper.indexMetric), @@ -368,8 +368,8 @@ public class SelectQueryRunnerTest new LegacySegmentSpec(new Interval("2011-01-12/2011-01-14")), new AndDimFilter( Arrays.asList( - new SelectorDimFilter(QueryRunnerTestHelper.providerDimension, "spot"), - new SelectorDimFilter(QueryRunnerTestHelper.providerDimension, "foo") + new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot"), + new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "foo") ) ), QueryRunnerTestHelper.allGran, diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java index c2e3889cd38..d4110c7b266 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -87,7 +87,7 @@ public class TimeseriesQueryRunnerTest QueryGranularity gran = QueryGranularity.DAY; TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(QueryRunnerTestHelper.dayGran) + .granularity(gran) .intervals(QueryRunnerTestHelper.fullOnInterval) .aggregators( Arrays.asList( @@ -193,7 +193,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.dayGran) - .filters(QueryRunnerTestHelper.providerDimension, "upfront") + .filters(QueryRunnerTestHelper.marketDimension, "upfront") .intervals(QueryRunnerTestHelper.fullOnInterval) .aggregators( Arrays.asList( @@ -205,7 +205,7 @@ public class TimeseriesQueryRunnerTest Assert.assertEquals( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("upfront") .build(), query.getDimensionsFilter() @@ -418,7 +418,7 @@ public class TimeseriesQueryRunnerTest { TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .filters(QueryRunnerTestHelper.providerDimension, "spot", "upfront", "total_market") + .filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market") .granularity( new PeriodGranularity( new Period("P7D"), @@ -471,7 +471,7 @@ public class TimeseriesQueryRunnerTest { TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .filters(QueryRunnerTestHelper.providerDimension, "spot", "upfront", "total_market") + .filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market") .granularity(QueryGranularity.HOUR) .intervals( Arrays.asList( @@ -539,7 +539,7 @@ public class TimeseriesQueryRunnerTest { TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .filters(QueryRunnerTestHelper.providerDimension, "spot", "upfront", "total_market") + .filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market") .granularity( new PeriodGranularity( new Period("PT1H"), @@ -586,7 +586,7 @@ public class TimeseriesQueryRunnerTest { TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .filters(QueryRunnerTestHelper.providerDimension, "spot", "upfront", "total_market") + .filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market") .granularity(new PeriodGranularity(new Period("P1M"), null, null)) .intervals( Arrays.asList( @@ -624,7 +624,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query2 = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .filters(QueryRunnerTestHelper.providerDimension, "spot", "upfront", "total_market") + .filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market") .granularity("DAY") .intervals( Arrays.asList( @@ -702,7 +702,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.dayGran) - .filters(QueryRunnerTestHelper.providerDimension, "spot", "upfront", "total_market") + .filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market") .intervals(QueryRunnerTestHelper.firstToThird) .aggregators( Arrays.asList( @@ -752,7 +752,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.dayGran) - .filters(new RegexDimFilter(QueryRunnerTestHelper.providerDimension, "^.p.*$")) // spot and upfront + .filters(new RegexDimFilter(QueryRunnerTestHelper.marketDimension, "^.p.*$")) // spot and upfront .intervals(QueryRunnerTestHelper.firstToThird) .aggregators( Arrays.asList( @@ -802,7 +802,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.dayGran) - .filters(QueryRunnerTestHelper.providerDimension, "spot") + .filters(QueryRunnerTestHelper.marketDimension, "spot") .intervals(QueryRunnerTestHelper.firstToThird) .aggregators( Arrays.asList( @@ -852,7 +852,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.dayGran) - .filters(QueryRunnerTestHelper.providerDimension, "upfront") + .filters(QueryRunnerTestHelper.marketDimension, "upfront") .intervals(QueryRunnerTestHelper.firstToThird) .aggregators( Arrays.asList( @@ -902,7 +902,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.dayGran) - .filters(QueryRunnerTestHelper.providerDimension, "total_market") + .filters(QueryRunnerTestHelper.marketDimension, "total_market") .intervals(QueryRunnerTestHelper.firstToThird) .aggregators( Arrays.asList( @@ -954,7 +954,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newOrDimFilterBuilder() @@ -1011,7 +1011,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newSelectorDimFilterBuilder() @@ -1069,7 +1069,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newSelectorDimFilterBuilder() @@ -1127,7 +1127,7 @@ public class TimeseriesQueryRunnerTest .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.dayGran) .filters( - QueryRunnerTestHelper.providerDimension, + QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market", @@ -1183,7 +1183,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newOrDimFilterBuilder() @@ -1288,7 +1288,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.dayGran) - .filters(QueryRunnerTestHelper.providerDimension, "billy") + .filters(QueryRunnerTestHelper.marketDimension, "billy") .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) @@ -1333,7 +1333,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("billy") .build(), Druids.newSelectorDimFilterBuilder() @@ -1533,7 +1533,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newSelectorDimFilterBuilder() @@ -1556,7 +1556,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newSelectorDimFilterBuilder() @@ -1594,7 +1594,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newOrDimFilterBuilder() @@ -1616,7 +1616,7 @@ public class TimeseriesQueryRunnerTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .value("spot") .build(), Druids.newOrDimFilterBuilder() diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index 09d383168cf..d155c35f91a 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -104,7 +104,7 @@ public class TopNQueryRunnerTest this.runner = runner; } - private static final String providerDimension = "provider"; + private static final String marketDimension = "market"; @Test public void testFullOnTopN() @@ -112,7 +112,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -136,7 +136,7 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put(providerDimension, "total_market") + .put(marketDimension, "total_market") .put("rows", 186L) .put("index", 215679.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D) @@ -145,7 +145,7 @@ public class TopNQueryRunnerTest .put("minIndex", 792.3260498046875D) .build(), ImmutableMap.builder() - .put(providerDimension, "upfront") + .put(marketDimension, "upfront") .put("rows", 186L) .put("index", 192046.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D) @@ -154,7 +154,7 @@ public class TopNQueryRunnerTest .put("minIndex", 545.9906005859375D) .build(), ImmutableMap.builder() - .put(providerDimension, "spot") + .put(marketDimension, "spot") .put("rows", 837L) .put("index", 95606.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D) @@ -176,7 +176,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.addRowsIndexConstantMetric) .threshold(4) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -200,7 +200,7 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put(providerDimension, "total_market") + .put(marketDimension, "total_market") .put("rows", 186L) .put("index", 215679.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D) @@ -209,7 +209,7 @@ public class TopNQueryRunnerTest .put("minIndex", 792.3260498046875D) .build(), ImmutableMap.builder() - .put(providerDimension, "upfront") + .put(marketDimension, "upfront") .put("rows", 186L) .put("index", 192046.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D) @@ -218,7 +218,7 @@ public class TopNQueryRunnerTest .put("minIndex", 545.9906005859375D) .build(), ImmutableMap.builder() - .put(providerDimension, "spot") + .put(marketDimension, "spot") .put("rows", 837L) .put("index", 95606.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D) @@ -241,7 +241,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.uniqueMetric) .threshold(3) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -265,7 +265,7 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 837L) .put("index", 95606.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D) @@ -274,7 +274,7 @@ public class TopNQueryRunnerTest .put("minIndex", 59.02102279663086D) .build(), ImmutableMap.builder() - .put("provider", "total_market") + .put("market", "total_market") .put("rows", 186L) .put("index", 215679.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D) @@ -283,7 +283,7 @@ public class TopNQueryRunnerTest .put("minIndex", 792.3260498046875D) .build(), ImmutableMap.builder() - .put("provider", "upfront") + .put("market", "upfront") .put("rows", 186L) .put("index", 192046.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D) @@ -306,7 +306,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -321,21 +321,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "spot", + marketDimension, "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, @@ -355,7 +355,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(new NumericTopNMetricSpec("uniques")) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -370,21 +370,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - "provider", "spot", + "market", "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( - "provider", "total_market", + "market", "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - "provider", "upfront", + "market", "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -404,8 +404,8 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .filters(providerDimension, "total_market", "upfront", "spot") - .dimension(providerDimension) + .filters(marketDimension, "total_market", "upfront", "spot") + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -419,21 +419,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "spot", + marketDimension, "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, @@ -453,8 +453,8 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .filters(providerDimension, "total_market", "upfront") - .dimension(providerDimension) + .filters(marketDimension, "total_market", "upfront") + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -468,14 +468,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -495,8 +495,8 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .filters(providerDimension, "upfront") - .dimension(providerDimension) + .filters(marketDimension, "upfront") + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -510,7 +510,7 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -531,7 +531,7 @@ public class TopNQueryRunnerTest .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) .filters(QueryRunnerTestHelper.qualityDimension, "mezzanine") - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -545,21 +545,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 2L, "index", 2591.68359375D, "addRowsIndexConstant", 2594.68359375D, "uniques", QueryRunnerTestHelper.UNIQUES_1 ), ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 2L, "index", 2508.39599609375D, "addRowsIndexConstant", 2511.39599609375D, "uniques", QueryRunnerTestHelper.UNIQUES_1 ), ImmutableMap.of( - providerDimension, "spot", + marketDimension, "spot", "rows", 2L, "index", 220.63774871826172D, "addRowsIndexConstant", 223.63774871826172D, @@ -580,7 +580,7 @@ public class TopNQueryRunnerTest .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) .filters(QueryRunnerTestHelper.qualityDimension, "mezzanine") - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals( @@ -598,21 +598,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 1L, "index", new Float(1447.341160).doubleValue(), "addRowsIndexConstant", new Float(1449.341160).doubleValue(), "uniques", QueryRunnerTestHelper.UNIQUES_1 ), ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 1L, "index", new Float(1314.839715).doubleValue(), "addRowsIndexConstant", new Float(1316.839715).doubleValue(), "uniques", QueryRunnerTestHelper.UNIQUES_1 ), ImmutableMap.of( - providerDimension, "spot", + marketDimension, "spot", "rows", 1L, "index", new Float(109.705815).doubleValue(), "addRowsIndexConstant", new Float(111.705815).doubleValue(), @@ -632,8 +632,8 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .filters(providerDimension, "total_market", "upfront", "billyblank") - .dimension(providerDimension) + .filters(marketDimension, "total_market", "upfront", "billyblank") + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -647,14 +647,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -674,8 +674,8 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .filters(providerDimension, "billyblank") - .dimension(providerDimension) + .filters(marketDimension, "billyblank") + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -701,7 +701,7 @@ public class TopNQueryRunnerTest .fields( Lists.newArrayList( Druids.newSelectorDimFilterBuilder() - .dimension(providerDimension) + .dimension(marketDimension) .value("billyblank") .build(), Druids.newSelectorDimFilterBuilder() @@ -714,7 +714,7 @@ public class TopNQueryRunnerTest .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) .filters(andDimFilter) - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -740,7 +740,7 @@ public class TopNQueryRunnerTest .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) .filters(QueryRunnerTestHelper.placementishDimension, "m") - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -755,7 +755,7 @@ public class TopNQueryRunnerTest .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) .filters(QueryRunnerTestHelper.qualityDimension, "mezzanine") - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -955,7 +955,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(new LexicographicTopNMetricSpec("")) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -969,21 +969,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "spot", + marketDimension, "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -1003,7 +1003,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(new LexicographicTopNMetricSpec("spot")) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -1017,14 +1017,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -1044,7 +1044,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(new LexicographicTopNMetricSpec("t")) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -1058,14 +1058,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -1085,7 +1085,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("upfront"))) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -1099,14 +1099,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "spot", + marketDimension, "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, @@ -1126,7 +1126,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("u"))) .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) @@ -1140,14 +1140,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "spot", + marketDimension, "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, @@ -1169,7 +1169,7 @@ public class TopNQueryRunnerTest .granularity(QueryRunnerTestHelper.allGran) .dimension( new ExtractionDimensionSpec( - providerDimension, providerDimension, new RegexDimExtractionFn("(.)") + marketDimension, marketDimension, new RegexDimExtractionFn("(.)") ) ) .metric("rows") @@ -1185,21 +1185,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "s", + marketDimension, "s", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( - providerDimension, "t", + marketDimension, "t", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "u", + marketDimension, "u", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -1221,7 +1221,7 @@ public class TopNQueryRunnerTest .granularity(QueryRunnerTestHelper.allGran) .dimension( new ExtractionDimensionSpec( - providerDimension, providerDimension, new RegexDimExtractionFn("(.)") + marketDimension, marketDimension, new RegexDimExtractionFn("(.)") ) ) .metric(new LexicographicTopNMetricSpec(null)) @@ -1237,21 +1237,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "s", + marketDimension, "s", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( - providerDimension, "t", + marketDimension, "t", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "u", + marketDimension, "u", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -1273,7 +1273,7 @@ public class TopNQueryRunnerTest .granularity(QueryRunnerTestHelper.allGran) .dimension( new ExtractionDimensionSpec( - providerDimension, providerDimension, new RegexDimExtractionFn("..(.)") + marketDimension, marketDimension, new RegexDimExtractionFn("..(.)") ) ) .metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec(null))) @@ -1289,21 +1289,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "t", + marketDimension, "t", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "o", + marketDimension, "o", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( - providerDimension, "f", + marketDimension, "f", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -1325,7 +1325,7 @@ public class TopNQueryRunnerTest .granularity(QueryRunnerTestHelper.allGran) .dimension( new ExtractionDimensionSpec( - providerDimension, providerDimension, new RegexDimExtractionFn("(.)") + marketDimension, marketDimension, new RegexDimExtractionFn("(.)") ) ) .metric(new LexicographicTopNMetricSpec("spot")) @@ -1341,14 +1341,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "t", + marketDimension, "t", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "u", + marketDimension, "u", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -1371,7 +1371,7 @@ public class TopNQueryRunnerTest .granularity(QueryRunnerTestHelper.allGran) .dimension( new ExtractionDimensionSpec( - providerDimension, providerDimension, new RegexDimExtractionFn("(.)") + marketDimension, marketDimension, new RegexDimExtractionFn("(.)") ) ) .metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("u"))) @@ -1387,14 +1387,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "t", + marketDimension, "t", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "s", + marketDimension, "s", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, @@ -1416,7 +1416,7 @@ public class TopNQueryRunnerTest .granularity(QueryRunnerTestHelper.allGran) .dimension( new ExtractionDimensionSpec( - providerDimension, providerDimension, new RegexDimExtractionFn("..(.)") + marketDimension, marketDimension, new RegexDimExtractionFn("..(.)") ) ) .metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("p"))) @@ -1432,14 +1432,14 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "o", + marketDimension, "o", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( - providerDimension, "f", + marketDimension, "f", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, @@ -1460,7 +1460,7 @@ public class TopNQueryRunnerTest new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(new InvertedTopNMetricSpec(new NumericTopNMetricSpec(QueryRunnerTestHelper.indexMetric))) .threshold(3) .intervals(QueryRunnerTestHelper.firstToThird) @@ -1474,21 +1474,21 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - providerDimension, "spot", + marketDimension, "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D, "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( - providerDimension, "upfront", + marketDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D, "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( - providerDimension, "total_market", + marketDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D, @@ -1508,7 +1508,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(QueryRunnerTestHelper.dependentPostAggMetric) .threshold(4) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -1538,7 +1538,7 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put(providerDimension, "total_market") + .put(marketDimension, "total_market") .put("rows", 186L) .put("index", 215679.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D) @@ -1552,7 +1552,7 @@ public class TopNQueryRunnerTest ) .build(), ImmutableMap.builder() - .put(providerDimension, "upfront") + .put(marketDimension, "upfront") .put("rows", 186L) .put("index", 192046.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D) @@ -1566,7 +1566,7 @@ public class TopNQueryRunnerTest ) .build(), ImmutableMap.builder() - .put(providerDimension, "spot") + .put(marketDimension, "spot") .put("rows", 837L) .put("index", 95606.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D) @@ -1593,7 +1593,7 @@ public class TopNQueryRunnerTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .metric(QueryRunnerTestHelper.dependentPostAggMetric) .threshold(4) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -1619,7 +1619,7 @@ public class TopNQueryRunnerTest TopNResultValue topNResult = new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "total_market") + .put(QueryRunnerTestHelper.marketDimension, "total_market") .put("rows", 186L) .put("index", 215679.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D) @@ -1629,7 +1629,7 @@ public class TopNQueryRunnerTest .put("minIndex", 792.3260498046875D) .build(), ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "upfront") + .put(QueryRunnerTestHelper.marketDimension, "upfront") .put("rows", 186L) .put("index", 192046.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D) @@ -1639,7 +1639,7 @@ public class TopNQueryRunnerTest .put("minIndex", 545.9906005859375D) .build(), ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "spot") + .put(QueryRunnerTestHelper.marketDimension, "spot") .put("rows", 837L) .put("index", 95606.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D) diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java index f2f4ac22f3d..cfe6978e61b 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java @@ -42,7 +42,7 @@ import static io.druid.query.QueryRunnerTestHelper.commonAggregators; import static io.druid.query.QueryRunnerTestHelper.dataSource; import static io.druid.query.QueryRunnerTestHelper.fullOnInterval; import static io.druid.query.QueryRunnerTestHelper.indexMetric; -import static io.druid.query.QueryRunnerTestHelper.providerDimension; +import static io.druid.query.QueryRunnerTestHelper.marketDimension; public class TopNQueryTest { @@ -54,7 +54,7 @@ public class TopNQueryTest Query query = new TopNQueryBuilder() .dataSource(dataSource) .granularity(allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(indexMetric) .threshold(4) .intervals(fullOnInterval) diff --git a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java index 7dc7b645cad..d9975f39153 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java @@ -98,7 +98,7 @@ public class TopNUnionQueryTest TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.unionDataSource) .granularity(QueryRunnerTestHelper.allGran) - .dimension(QueryRunnerTestHelper.providerDimension) + .dimension(QueryRunnerTestHelper.marketDimension) .metric(QueryRunnerTestHelper.dependentPostAggMetric) .threshold(4) .intervals(QueryRunnerTestHelper.fullOnInterval) @@ -128,7 +128,7 @@ public class TopNUnionQueryTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "total_market") + .put(QueryRunnerTestHelper.marketDimension, "total_market") .put("rows", 744L) .put("index", 862719.3151855469D) .put("addRowsIndexConstant", 863464.3151855469D) @@ -142,7 +142,7 @@ public class TopNUnionQueryTest ) .build(), ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "upfront") + .put(QueryRunnerTestHelper.marketDimension, "upfront") .put("rows", 744L) .put("index", 768184.4240722656D) .put("addRowsIndexConstant", 768929.4240722656D) @@ -156,7 +156,7 @@ public class TopNUnionQueryTest ) .build(), ImmutableMap.builder() - .put(QueryRunnerTestHelper.providerDimension, "spot") + .put(QueryRunnerTestHelper.marketDimension, "spot") .put("rows", 3348L) .put("index", 382426.28929138184D) .put("addRowsIndexConstant", 385775.28929138184D) diff --git a/processing/src/test/java/io/druid/segment/AppendTest.java b/processing/src/test/java/io/druid/segment/AppendTest.java index 13a37ff2e1a..1ddb8122f94 100644 --- a/processing/src/test/java/io/druid/segment/AppendTest.java +++ b/processing/src/test/java/io/druid/segment/AppendTest.java @@ -79,7 +79,7 @@ public class AppendTest final QueryGranularity allGran = QueryGranularity.ALL; final String dimensionValue = "dimension"; final String valueValue = "value"; - final String providerDimension = "provider"; + final String marketDimension = "market"; final String qualityDimension = "quality"; final String placementDimension = "placement"; final String placementishDimension = "placementish"; @@ -309,7 +309,7 @@ public class AppendTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 3L) .put("index", 300.0D) .put("addRowsIndexConstant", 304.0D) @@ -319,7 +319,7 @@ public class AppendTest .build(), new HashMap() {{ - put("provider", null); + put("market", null); put("rows", 3L); put("index", 200.0D); put("addRowsIndexConstant", 204.0D); @@ -328,7 +328,7 @@ public class AppendTest put("minIndex", 0.0); }}, ImmutableMap.builder() - .put("provider", "total_market") + .put("market", "total_market") .put("rows", 2L) .put("index", 200.0D) .put("addRowsIndexConstant", 203.0D) @@ -355,7 +355,7 @@ public class AppendTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "total_market") + .put("market", "total_market") .put("rows", 3L) .put("index", 300.0D) .put("addRowsIndexConstant", 304.0D) @@ -365,7 +365,7 @@ public class AppendTest .build(), new HashMap() {{ - put("provider", null); + put("market", null); put("rows", 3L); put("index", 100.0D); put("addRowsIndexConstant", 104.0D); @@ -374,7 +374,7 @@ public class AppendTest put("minIndex", 0.0); }}, ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -401,7 +401,7 @@ public class AppendTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -447,7 +447,7 @@ public class AppendTest new SearchHit(placementishDimension, "a"), new SearchHit(qualityDimension, "automotive"), new SearchHit(placementDimension, "mezzanine"), - new SearchHit(providerDimension, "total_market") + new SearchHit(marketDimension, "total_market") ) ) ) @@ -468,7 +468,7 @@ public class AppendTest Arrays.asList( new SearchHit(placementishDimension, "a"), new SearchHit(placementDimension, "mezzanine"), - new SearchHit(providerDimension, "total_market") + new SearchHit(marketDimension, "total_market") ) ) ) @@ -488,7 +488,7 @@ public class AppendTest new SearchResultValue( Arrays.asList( new SearchHit(placementDimension, "mezzanine"), - new SearchHit(providerDimension, "total_market") + new SearchHit(marketDimension, "total_market") ) ) ) @@ -509,7 +509,7 @@ public class AppendTest Arrays.asList( new SearchHit(placementishDimension, "a"), new SearchHit(placementDimension, "mezzanine"), - new SearchHit(providerDimension, "total_market") + new SearchHit(marketDimension, "total_market") ) ) ) @@ -543,7 +543,7 @@ public class AppendTest .dataSource(dataSource) .granularity(allGran) .intervals(fullOnInterval) - .filters(providerDimension, "breakstuff") + .filters(marketDimension, "breakstuff") .aggregators( Lists.newArrayList( Iterables.concat( @@ -593,11 +593,11 @@ public class AppendTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(providerDimension) + .dimension(marketDimension) .value("spot") .build(), Druids.newSelectorDimFilterBuilder() - .dimension(providerDimension) + .dimension(marketDimension) .value("total_market") .build() ) @@ -623,7 +623,7 @@ public class AppendTest return new TopNQueryBuilder() .dataSource(dataSource) .granularity(allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(indexMetric) .threshold(3) .intervals(fullOnInterval) @@ -647,7 +647,7 @@ public class AppendTest return new TopNQueryBuilder() .dataSource(dataSource) .granularity(allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(indexMetric) .threshold(3) .filters( @@ -655,7 +655,7 @@ public class AppendTest .fields( Arrays.asList( Druids.newSelectorDimFilterBuilder() - .dimension(providerDimension) + .dimension(marketDimension) .value("spot") .build(), Druids.newSelectorDimFilterBuilder() @@ -699,7 +699,7 @@ public class AppendTest Druids.newNotDimFilterBuilder() .field( Druids.newSelectorDimFilterBuilder() - .dimension(providerDimension) + .dimension(marketDimension) .value("spot") .build() ).build() diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java index 83f6a62aa99..90b138aca5a 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java @@ -71,7 +71,7 @@ public class SchemalessTestFull final QueryGranularity allGran = QueryGranularity.ALL; final String dimensionValue = "dimension"; final String valueValue = "value"; - final String providerDimension = "provider"; + final String marketDimension = "market"; final String qualityDimension = "quality"; final String placementDimension = "placement"; final String placementishDimension = "placementish"; @@ -133,7 +133,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -142,7 +142,7 @@ public class SchemalessTestFull .put("minIndex", 100.0) .build(), ImmutableMap.builder() - .put("provider", "total_market") + .put("market", "total_market") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -161,7 +161,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -182,7 +182,7 @@ public class SchemalessTestFull new SearchHit(placementishDimension, "a"), new SearchHit(qualityDimension, "automotive"), new SearchHit(placementDimension, "mezzanine"), - new SearchHit(providerDimension, "total_market") + new SearchHit(marketDimension, "total_market") ) ) ) @@ -268,7 +268,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "") + .put("market", "") .put("rows", 2L) .put("index", 200.0D) .put("addRowsIndexConstant", 203.0D) @@ -277,7 +277,7 @@ public class SchemalessTestFull .put("minIndex", 100.0) .build(), ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -296,7 +296,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "") + .put("market", "") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -305,7 +305,7 @@ public class SchemalessTestFull .put("minIndex", 100.0) .build(), ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -407,7 +407,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -417,7 +417,7 @@ public class SchemalessTestFull .build(), new HashMap() {{ - put("provider", null); + put("market", null); put("rows", 1L); put("index", 0.0D); put("addRowsIndexConstant", 2.0D); @@ -436,7 +436,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -540,7 +540,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -549,7 +549,7 @@ public class SchemalessTestFull .put("minIndex", 100.0) .build(), ImmutableMap.builder() - .put("provider", "total_market") + .put("market", "total_market") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -568,7 +568,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -587,7 +587,7 @@ public class SchemalessTestFull new SearchResultValue( Arrays.asList( new SearchHit(qualityDimension, "automotive"), - new SearchHit(providerDimension, "total_market") + new SearchHit(marketDimension, "total_market") ) ) ) @@ -658,7 +658,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 2L) .put("index", 200.0D) .put("addRowsIndexConstant", 203.0D) @@ -755,7 +755,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -765,7 +765,7 @@ public class SchemalessTestFull .build(), new HashMap() {{ - put("provider", null); + put("market", null); put("rows", 1L); put("index", 0.0D); put("addRowsIndexConstant", 2.0D); @@ -784,7 +784,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -942,7 +942,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 200.0D) .put("addRowsIndexConstant", 202.0D) @@ -1042,7 +1042,7 @@ public class SchemalessTestFull Arrays.>asList( new HashMap() {{ - put("provider", null); + put("market", null); put("rows", 2L); put("index", 200.0D); put("addRowsIndexConstant", 203.0D); @@ -1051,7 +1051,7 @@ public class SchemalessTestFull put("minIndex", 100.0); }}, ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -1070,7 +1070,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -1175,7 +1175,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 4L) .put("index", 400.0D) .put("addRowsIndexConstant", 405.0D) @@ -1184,7 +1184,7 @@ public class SchemalessTestFull .put("minIndex", 100.0) .build(), ImmutableMap.builder() - .put("provider", "") + .put("market", "") .put("rows", 3L) .put("index", 200.0D) .put("addRowsIndexConstant", 204.0D) @@ -1193,7 +1193,7 @@ public class SchemalessTestFull .put("minIndex", 0.0) .build(), ImmutableMap.builder() - .put("provider", "total_market") + .put("market", "total_market") .put("rows", 2L) .put("index", 200.0D) .put("addRowsIndexConstant", 203.0D) @@ -1212,7 +1212,7 @@ public class SchemalessTestFull new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 4L) .put("index", 400.0D) .put("addRowsIndexConstant", 405.0D) @@ -1221,7 +1221,7 @@ public class SchemalessTestFull .put("minIndex", 100.0) .build(), ImmutableMap.builder() - .put("provider", "") + .put("market", "") .put("rows", 1L) .put("index", 100.0D) .put("addRowsIndexConstant", 102.0D) @@ -1242,7 +1242,7 @@ public class SchemalessTestFull new SearchHit(placementishDimension, "a"), new SearchHit(qualityDimension, "automotive"), new SearchHit(placementDimension, "mezzanine"), - new SearchHit(providerDimension, "total_market") + new SearchHit(marketDimension, "total_market") ) ) ) @@ -1403,7 +1403,7 @@ public class SchemalessTestFull .dataSource(dataSource) .granularity(allGran) .intervals(fullOnInterval) - .filters(providerDimension, "spot") + .filters(marketDimension, "spot") .aggregators( Lists.newArrayList( Iterables.concat( @@ -1432,7 +1432,7 @@ public class SchemalessTestFull TopNQuery query = new TopNQueryBuilder() .dataSource(dataSource) .granularity(allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(indexMetric) .threshold(3) .intervals(fullOnInterval) @@ -1464,8 +1464,8 @@ public class SchemalessTestFull TopNQuery query = new TopNQueryBuilder() .dataSource(dataSource) .granularity(allGran) - .dimension(providerDimension) - .filters(providerDimension, "spot") + .dimension(marketDimension) + .filters(marketDimension, "spot") .metric(indexMetric) .threshold(3) .intervals(fullOnInterval) @@ -1513,7 +1513,7 @@ public class SchemalessTestFull SearchQuery query = Druids.newSearchQueryBuilder() .dataSource(dataSource) .granularity(allGran) - .filters(providerDimension, "spot") + .filters(marketDimension, "spot") .intervals(fullOnInterval) .query("a") .build(); diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java b/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java index 85226b3256d..524aa5b8668 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java @@ -97,7 +97,7 @@ public class SchemalessTestSimple final QueryGranularity allGran = QueryGranularity.ALL; final String dimensionValue = "dimension"; final String valueValue = "value"; - final String providerDimension = "provider"; + final String marketDimension = "market"; final String qualityDimension = "quality"; final String placementDimension = "placement"; final String placementishDimension = "placementish"; @@ -175,7 +175,7 @@ public class SchemalessTestSimple TopNQuery query = new TopNQueryBuilder() .dataSource(dataSource) .granularity(allGran) - .dimension(providerDimension) + .dimension(marketDimension) .metric(indexMetric) .threshold(3) .intervals(fullOnInterval) @@ -200,7 +200,7 @@ public class SchemalessTestSimple Arrays.asList( new DimensionAndMetricValueExtractor( ImmutableMap.builder() - .put("provider", "spot") + .put("market", "spot") .put("rows", 4L) .put("index", 400.0D) .put("addRowsIndexConstant", 405.0D) @@ -211,7 +211,7 @@ public class SchemalessTestSimple ), new DimensionAndMetricValueExtractor( ImmutableMap.builder() - .put("provider", "") + .put("market", "") .put("rows", 2L) .put("index", 200.0D) .put("addRowsIndexConstant", 203.0D) @@ -222,7 +222,7 @@ public class SchemalessTestSimple ), new DimensionAndMetricValueExtractor( ImmutableMap.builder() - .put("provider", "total_market") + .put("market", "total_market") .put("rows", 2L) .put("index", 200.0D) .put("addRowsIndexConstant", 203.0D) @@ -258,7 +258,7 @@ public class SchemalessTestSimple new SearchHit(placementishDimension, "a"), new SearchHit(qualityDimension, "automotive"), new SearchHit(placementDimension, "mezzanine"), - new SearchHit(providerDimension, "total_market") + new SearchHit(marketDimension, "total_market") ) ) ) diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index bb83dac77ce..bba10185dee 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -59,14 +59,14 @@ public class TestIndex public static final String[] COLUMNS = new String[]{ "ts", - "provider", + "market", "quALIty", "plAcEmEnT", "pLacementish", "iNdEx", "qualiTy_Uniques" }; - public static final String[] DIMENSIONS = new String[]{"provider", "quALIty", "plAcEmEnT", "pLacementish"}; + public static final String[] DIMENSIONS = new String[]{"market", "quALIty", "plAcEmEnT", "pLacementish"}; public static final String[] METRICS = new String[]{"iNdEx"}; private static final Interval DATA_INTERVAL = new Interval("2011-01-12T00:00:00.000Z/2011-05-01T00:00:00.000Z"); private static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ diff --git a/processing/src/test/resources/append.json.1 b/processing/src/test/resources/append.json.1 index aa55a99694a..efce43f58e3 100644 --- a/processing/src/test/resources/append.json.1 +++ b/processing/src/test/resources/append.json.1 @@ -4,7 +4,7 @@ }, { "timestamp":"2011-01-13T00:00:00.000Z", - "provider":"spot", + "market":"spot", "quality":"automotive", "placement":"preferred", "placementish":["a", "preferred"], @@ -12,7 +12,7 @@ }, { "timestamp":"2011-01-14T00:00:00.000Z", - "provider":"spot", + "market":"spot", "quality":"automotive", "index":100.000000 }, diff --git a/processing/src/test/resources/append.json.2 b/processing/src/test/resources/append.json.2 index fd3f916e9ec..88d7501132d 100644 --- a/processing/src/test/resources/append.json.2 +++ b/processing/src/test/resources/append.json.2 @@ -1,14 +1,14 @@ [ { "timestamp":"2011-01-14T22:00:00.000Z", - "provider":"total_market", + "market":"total_market", "placement":"preferred", "placementish":["h", "preferred"], "index":100.000000 }, { "timestamp":"2011-01-14T23:00:00.000Z", - "provider":"total_market", + "market":"total_market", "quality":"business", "placement":"mezzanine", "placementish":["p", "preferred"], @@ -17,7 +17,7 @@ { "timestamp":"2011-01-15T00:00:00.000Z", "placementish":"preferred", - "provider":"spot", + "market":"spot", "index":100.000000 }, { diff --git a/processing/src/test/resources/append.json.3 b/processing/src/test/resources/append.json.3 index 7a963dc4e78..de15d125969 100644 --- a/processing/src/test/resources/append.json.3 +++ b/processing/src/test/resources/append.json.3 @@ -4,7 +4,7 @@ }, { "timestamp":"2011-01-13T00:00:00.000Z", - "provider":"spot", + "market":"spot", "quality":"automotive", "placement":"preferred", "placementish":["a", "preferred"], @@ -12,7 +12,7 @@ }, { "timestamp":"2011-01-14T00:00:00.000Z", - "provider":"total_market", + "market":"total_market", "index":100.000000 }, { diff --git a/processing/src/test/resources/append.json.4 b/processing/src/test/resources/append.json.4 index 5049fdc5e69..1a8d7d5d8d9 100644 --- a/processing/src/test/resources/append.json.4 +++ b/processing/src/test/resources/append.json.4 @@ -1,14 +1,14 @@ [ { "timestamp":"2011-01-13T00:00:00.000Z", - "provider":"total_market", + "market":"total_market", "placement":"preferred", "placementish":["h", "preferred"], "index":100.000000 }, { "timestamp":"2011-01-13T01:00:00.000Z", - "provider":"total_market", + "market":"total_market", "placement":"mezzanine", "placementish":["p", "preferred"], "index":100.000000 @@ -16,7 +16,7 @@ { "timestamp":"2011-01-13T15:00:00.000Z", "placementish":"preferred", - "provider":"spot", + "market":"spot", "index":100.000000 }, { diff --git a/processing/src/test/resources/append.json.5 b/processing/src/test/resources/append.json.5 index 0119fead368..a59d23c13c5 100644 --- a/processing/src/test/resources/append.json.5 +++ b/processing/src/test/resources/append.json.5 @@ -1,32 +1,32 @@ [ { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":"breakstuff", + "market":"breakstuff", "index":100.000000 }, { "timestamp":"2011-01-14T00:00:00.000Z", - "provider":"breakstuff", + "market":"breakstuff", "index":100.000000 }, { "timestamp":"2011-01-16T00:00:00.000Z", - "provider":"breakstuff", + "market":"breakstuff", "index":100.000000 }, { "timestamp":"2011-01-17T00:00:00.000Z", - "provider":"breakstuff", + "market":"breakstuff", "index":100.000000 }, { "timestamp":"2011-01-19T00:00:00.000Z", - "provider":"breakstuff", + "market":"breakstuff", "index":100.000000 }, { "timestamp":"2011-01-21T00:00:00.000Z", - "provider":"breakstuff", + "market":"breakstuff", "index":100.000000 } ] \ No newline at end of file diff --git a/processing/src/test/resources/append.json.6 b/processing/src/test/resources/append.json.6 index 386769ce0ef..56ff8a3faad 100644 --- a/processing/src/test/resources/append.json.6 +++ b/processing/src/test/resources/append.json.6 @@ -1,12 +1,12 @@ [ { "timestamp":"2011-01-13T00:00:00.000Z", - "provider":"breakstuff", + "market":"breakstuff", "index":100.000000 }, { "timestamp":"2011-01-15T00:00:00.000Z", - "provider":"spot", + "market":"spot", "index":100.000000 } ] \ No newline at end of file diff --git a/processing/src/test/resources/append.json.7 b/processing/src/test/resources/append.json.7 index 8645df52b8f..cc8089f8981 100644 --- a/processing/src/test/resources/append.json.7 +++ b/processing/src/test/resources/append.json.7 @@ -1,12 +1,12 @@ [ { "timestamp":"2011-01-18T00:00:00.000Z", - "provider":"spot", + "market":"spot", "index":100.000000 }, { "timestamp":"2011-01-20T00:00:00.000Z", - "provider":"spot", + "market":"spot", "index":100.000000 } ] \ No newline at end of file diff --git a/processing/src/test/resources/druid.sample.json b/processing/src/test/resources/druid.sample.json index e188e7be089..a3cddb46ff0 100644 --- a/processing/src/test/resources/druid.sample.json +++ b/processing/src/test/resources/druid.sample.json @@ -4,7 +4,7 @@ }, { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":"spot", + "market":"spot", "quality":"automotive", "placement":"preferred", "placementish":["a", "preferred"], @@ -12,7 +12,7 @@ }, { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":"spot", + "market":"spot", "quality":"automotive", "index":100.000000 }, @@ -23,14 +23,14 @@ }, { "timestamp":"2011-01-13T00:00:00.000Z", - "provider":"total_market", + "market":"total_market", "placementish":["h", "preferred"], "placement":"preferred", "index":100.000000 }, { "timestamp":"2011-01-13T00:00:00.000Z", - "provider":"total_market", + "market":"total_market", "placementish":["p", "preferred"], "quality":"business", "placement":"mezzanine", @@ -39,7 +39,7 @@ { "timestamp":"2011-01-12T00:00:00.000Z", "placementish":"preferred", - "provider":"spot", + "market":"spot", "index":100.000000 }, { @@ -55,12 +55,12 @@ }, { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":"", + "market":"", "index":100.000000 }, { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":["", "spot"], + "market":["", "spot"], "index":100.000000 } ] \ No newline at end of file diff --git a/processing/src/test/resources/druid.sample.json.bottom b/processing/src/test/resources/druid.sample.json.bottom index a7b3b648a2a..75ea3502c6e 100644 --- a/processing/src/test/resources/druid.sample.json.bottom +++ b/processing/src/test/resources/druid.sample.json.bottom @@ -6,14 +6,14 @@ }, { "timestamp":"2011-01-13T00:00:00.000Z", - "provider":"total_market", + "market":"total_market", "placement":"preferred", "placementish":["h", "preferred"], "index":100.000000 }, { "timestamp":"2011-01-13T00:00:00.000Z", - "provider":"total_market", + "market":"total_market", "quality":"business", "placement":"mezzanine", "placementish":["p", "preferred"], @@ -22,7 +22,7 @@ { "timestamp":"2011-01-12T00:00:00.000Z", "placementish":"preferred", - "provider":"spot", + "market":"spot", "index":100.000000 }, { @@ -38,12 +38,12 @@ }, { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":"", + "market":"", "index":100.000000 }, { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":["", "spot"], + "market":["", "spot"], "index":100.000000 } ] \ No newline at end of file diff --git a/processing/src/test/resources/druid.sample.json.top b/processing/src/test/resources/druid.sample.json.top index 8439291578d..4d1834ca27a 100644 --- a/processing/src/test/resources/druid.sample.json.top +++ b/processing/src/test/resources/druid.sample.json.top @@ -4,7 +4,7 @@ }, { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":"spot", + "market":"spot", "quality":"automotive", "placement":"preferred", "placementish":["a", "preferred"], @@ -12,7 +12,7 @@ }, { "timestamp":"2011-01-12T00:00:00.000Z", - "provider":"spot", + "market":"spot", "quality":"automotive", "index":100.000000 } From f4a97aebbc2605798637285c123a709008a9f527 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 15 Oct 2014 22:32:56 +0530 Subject: [PATCH 37/69] fix rollup for hashed partitions truncate timestamp while calculating the partitionNumber --- .../io/druid/indexer/HadoopDruidIndexerConfig.java | 5 ++++- .../io/druid/indexing/common/task/IndexTask.java | 12 +++++++----- pom.xml | 2 +- .../partition/HashBasedNumberedShardSpec.java | 12 ++++++------ .../io/druid/timeline/partition/LinearShardSpec.java | 4 ++-- .../druid/timeline/partition/NumberedShardSpec.java | 4 ++-- .../timeline/partition/SingleDimensionShardSpec.java | 6 +++--- .../server/shard/HashBasedNumberedShardSpecTest.java | 5 +++-- .../server/shard/SingleDimensionShardSpecTest.java | 2 +- 9 files changed, 29 insertions(+), 23 deletions(-) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index e6402221e45..335c8c1858b 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -41,6 +41,7 @@ import com.metamx.common.logger.Logger; import io.druid.common.utils.JodaUtils; import io.druid.data.input.InputRow; import io.druid.data.input.impl.StringInputRowParser; +import io.druid.granularity.QueryGranularity; import io.druid.guice.GuiceInjectors; import io.druid.guice.JsonConfigProvider; import io.druid.guice.annotations.Self; @@ -172,6 +173,7 @@ public class HadoopDruidIndexerConfig private volatile PathSpec pathSpec; private volatile Map shardSpecLookups = Maps.newHashMap(); private volatile Map hadoopShardSpecLookup = Maps.newHashMap(); + private final QueryGranularity rollupGran; @JsonCreator public HadoopDruidIndexerConfig( @@ -203,6 +205,7 @@ public class HadoopDruidIndexerConfig hadoopShardSpecLookup.put(hadoopyShardSpec.getActualSpec(), hadoopyShardSpec); } } + this.rollupGran = schema.getDataSchema().getGranularitySpec().getQueryGranularity(); } @JsonProperty @@ -326,7 +329,7 @@ public class HadoopDruidIndexerConfig return Optional.absent(); } - final ShardSpec actualSpec = shardSpecLookups.get(timeBucket.get().getStart()).getShardSpec(inputRow); + final ShardSpec actualSpec = shardSpecLookups.get(timeBucket.get().getStart()).getShardSpec(rollupGran.truncate(inputRow.getTimestampFromEpoch()), inputRow); final HadoopyShardSpec hadoopyShardSpec = hadoopShardSpecLookup.get(actualSpec); return Optional.of( diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index b7496c70782..f04736a66e3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -407,14 +407,14 @@ public class IndexTask extends AbstractFixedIntervalTask final int myRowFlushBoundary = rowFlushBoundary > 0 ? rowFlushBoundary : toolbox.getConfig().getDefaultRowFlushBoundary(); - + final QueryGranularity rollupGran = ingestionSchema.getDataSchema().getGranularitySpec().getQueryGranularity(); try { plumber.startJob(); while (firehose.hasMore()) { final InputRow inputRow = firehose.nextRow(); - if (shouldIndex(shardSpec, interval, inputRow)) { + if (shouldIndex(shardSpec, interval, inputRow, rollupGran)) { int numRows = plumber.add(inputRow); if (numRows == -1) { throw new ISE( @@ -469,13 +469,15 @@ public class IndexTask extends AbstractFixedIntervalTask * * @return true or false */ - private boolean shouldIndex( + private static boolean shouldIndex( final ShardSpec shardSpec, final Interval interval, - final InputRow inputRow + final InputRow inputRow, + final QueryGranularity rollupGran ) { - return interval.contains(inputRow.getTimestampFromEpoch()) && shardSpec.isInChunk(inputRow); + return interval.contains(inputRow.getTimestampFromEpoch()) + && shardSpec.isInChunk(rollupGran.truncate(inputRow.getTimestampFromEpoch()), inputRow); } public static class IndexIngestionSpec extends IngestionSpec diff --git a/pom.xml b/pom.xml index 8c1202d7b79..c8eca1e8738 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.9 2.6.0 - 0.2.14 + 0.2.14.1 diff --git a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java index be640a03545..56ff5159c62 100644 --- a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java @@ -49,14 +49,14 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec } @Override - public boolean isInChunk(InputRow inputRow) + public boolean isInChunk(long timestamp, InputRow inputRow) { - return (((long) hash(inputRow)) - getPartitionNum()) % getPartitions() == 0; + return (((long) hash(timestamp, inputRow)) - getPartitionNum()) % getPartitions() == 0; } - protected int hash(InputRow inputRow) + protected int hash(long timestamp, InputRow inputRow) { - final List groupKey = Rows.toGroupKey(inputRow.getTimestampFromEpoch(), inputRow); + final List groupKey = Rows.toGroupKey(timestamp, inputRow); try { return hashFunction.hashBytes(jsonMapper.writeValueAsBytes(groupKey)).asInt(); } @@ -80,9 +80,9 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec return new ShardSpecLookup() { @Override - public ShardSpec getShardSpec(InputRow row) + public ShardSpec getShardSpec(long timestamp, InputRow row) { - int index = Math.abs(hash(row) % getPartitions()); + int index = Math.abs(hash(timestamp, row) % getPartitions()); return shardSpecs.get(index); } }; diff --git a/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java b/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java index d3ab608a328..b095f7135c4 100644 --- a/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java @@ -50,7 +50,7 @@ public class LinearShardSpec implements ShardSpec return new ShardSpecLookup() { @Override - public ShardSpec getShardSpec(InputRow row) + public ShardSpec getShardSpec(long timestamp, InputRow row) { return shardSpecs.get(0); } @@ -63,7 +63,7 @@ public class LinearShardSpec implements ShardSpec } @Override - public boolean isInChunk(InputRow inputRow) { + public boolean isInChunk(long timestamp, InputRow inputRow) { return true; } diff --git a/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java b/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java index 65399f1009f..7c90e66c711 100644 --- a/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java @@ -60,7 +60,7 @@ public class NumberedShardSpec implements ShardSpec return new ShardSpecLookup() { @Override - public ShardSpec getShardSpec(InputRow row) + public ShardSpec getShardSpec(long timestamp, InputRow row) { return shardSpecs.get(0); } @@ -80,7 +80,7 @@ public class NumberedShardSpec implements ShardSpec } @Override - public boolean isInChunk(InputRow inputRow) + public boolean isInChunk(long timestamp, InputRow inputRow) { return true; } diff --git a/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java b/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java index 1cf232a4b29..2350dc559ef 100644 --- a/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java @@ -100,10 +100,10 @@ public class SingleDimensionShardSpec implements ShardSpec return new ShardSpecLookup() { @Override - public ShardSpec getShardSpec(InputRow row) + public ShardSpec getShardSpec(long timestamp, InputRow row) { for (ShardSpec spec : shardSpecs) { - if (spec.isInChunk(row)) { + if (spec.isInChunk(timestamp, row)) { return spec; } } @@ -124,7 +124,7 @@ public class SingleDimensionShardSpec implements ShardSpec } @Override - public boolean isInChunk(InputRow inputRow) + public boolean isInChunk(long timestamp, InputRow inputRow) { final List values = inputRow.getDimension(dimension); diff --git a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java index a5e3ff1181d..47c78724010 100644 --- a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java +++ b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java @@ -127,7 +127,7 @@ public class HashBasedNumberedShardSpecTest public boolean assertExistsInOneSpec(List specs, InputRow row) { for (ShardSpec spec : specs) { - if (spec.isInChunk(row)) { + if (spec.isInChunk(row.getTimestampFromEpoch(), row)) { return true; } } @@ -145,7 +145,7 @@ public class HashBasedNumberedShardSpecTest } @Override - protected int hash(InputRow inputRow) + protected int hash(long timestamp, InputRow inputRow) { return inputRow.hashCode(); } @@ -208,4 +208,5 @@ public class HashBasedNumberedShardSpecTest return 0; } } + } diff --git a/server/src/test/java/io/druid/server/shard/SingleDimensionShardSpecTest.java b/server/src/test/java/io/druid/server/shard/SingleDimensionShardSpecTest.java index d07b52d9fcb..5bb49b38e5c 100644 --- a/server/src/test/java/io/druid/server/shard/SingleDimensionShardSpecTest.java +++ b/server/src/test/java/io/druid/server/shard/SingleDimensionShardSpecTest.java @@ -111,7 +111,7 @@ public class SingleDimensionShardSpecTest } ) ); - Assert.assertEquals(String.format("spec[%s], row[%s]", spec, inputRow), pair.lhs, spec.isInChunk(inputRow)); + Assert.assertEquals(String.format("spec[%s], row[%s]", spec, inputRow), pair.lhs, spec.isInChunk(inputRow.getTimestampFromEpoch(), inputRow)); } } } From 41e88baecaec5a97f34655a3c93a61283e90f3a0 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 15 Oct 2014 23:09:28 +0530 Subject: [PATCH 38/69] Add test for bucket selection --- .../indexer/HadoopDruidIndexerConfigTest.java | 83 +++++++++++++++++++ 1 file changed, 83 insertions(+) diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java index ee8fa9315c5..1fe6068f2ad 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -20,15 +20,34 @@ package io.druid.indexer; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.client.util.Lists; +import com.google.common.base.Optional; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.metamx.common.Granularity; +import io.druid.data.input.InputRow; +import io.druid.data.input.MapBasedInputRow; +import io.druid.data.input.impl.JSONDataSpec; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.indexer.rollup.DataRollupSpec; import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import io.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.joda.time.DateTime; +import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; +import java.util.Arrays; +import java.util.List; + /** */ public class HadoopDruidIndexerConfigTest @@ -125,4 +144,68 @@ public class HadoopDruidIndexerConfigTest ); } + + @Test + public void testHashedBucketSelection() { + List specs = Lists.newArrayList(); + final int partitionCount = 10; + for (int i = 0; i < partitionCount; i++) { + specs.add(new HadoopyShardSpec(new HashBasedNumberedShardSpec(i, partitionCount, new DefaultObjectMapper()), i)); + } + HadoopIngestionSpec spec = new HadoopIngestionSpec( + null, null, null, + "foo", + new TimestampSpec("timestamp", "auto"), + new JSONDataSpec(ImmutableList.of("foo"), null), + new UniformGranularitySpec( + Granularity.HOUR, + QueryGranularity.MINUTE, + ImmutableList.of(new Interval("2010-01-01/P1D")), + Granularity.HOUR + ), + null, + null, + null, + null, + null, + false, + true, + ImmutableMap.of(new DateTime("2010-01-01T01:00:00"), specs), + false, + new DataRollupSpec(ImmutableList.of(), QueryGranularity.MINUTE), + null, + false, + ImmutableMap.of("foo", "bar"), + false, + null, + null, + null, + null, + null, + null + ); + HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromSchema(spec); + final List dims = Arrays.asList("diM1", "dIM2"); + final ImmutableMap values = ImmutableMap.of( + "Dim1", + "1", + "DiM2", + "2", + "dim1", + "3", + "dim2", + "4" + ); + final long timestamp = new DateTime("2010-01-01T01:00:01").getMillis(); + final Bucket expectedBucket = config.getBucket(new MapBasedInputRow(timestamp, dims, values)).get(); + final long nextBucketTimestamp = QueryGranularity.MINUTE.next(QueryGranularity.MINUTE.truncate(timestamp)); + // check that all rows having same set of dims and truncated timestamp hash to same bucket + for (int i = 0; timestamp + i < nextBucketTimestamp; i++) { + Assert.assertEquals( + expectedBucket.partitionNum, + config.getBucket(new MapBasedInputRow(timestamp + i, dims, values)).get().partitionNum + ); + } + + } } From 20d29666af5d984d85de04ce18abb48410e346c6 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 16 Oct 2014 13:17:04 -0700 Subject: [PATCH 39/69] New PartitionsSpec docs. --- docs/content/Batch-ingestion.md | 61 ++++++++++++++++++++++----------- 1 file changed, 41 insertions(+), 20 deletions(-) diff --git a/docs/content/Batch-ingestion.md b/docs/content/Batch-ingestion.md index 2f53eb48b4e..0326d1e1a66 100644 --- a/docs/content/Batch-ingestion.md +++ b/docs/content/Batch-ingestion.md @@ -162,37 +162,58 @@ The indexing process has the ability to roll data up as it processes the incomin ### Partitioning specification -Segments are always partitioned based on timestamp (according to the granularitySpec) and may be further partitioned in some other way depending on partition type. -Druid supports two types of partitions spec - singleDimension and hashed. +Segments are always partitioned based on timestamp (according to the granularitySpec) and may be further partitioned in +some other way depending on partition type. Druid supports two types of partitioning strategies: "hashed" (based on the +hash of all dimensions in each row), and "dimension" (based on ranges of a single dimension). -In SingleDimension partition type data is partitioned based on the values in that dimension. -For example, data for a day may be split by the dimension "last\_name" into two segments: one with all values from A-M and one with all values from N-Z. +Hashed partitioning is recommended in most cases, as it will improve indexing performance and create more uniformly +sized data segments relative to single-dimension partitioning. -In hashed partition type, the number of partitions is determined based on the targetPartitionSize and cardinality of input set and the data is partitioned based on the hashcode of the row. - -It is recommended to use Hashed partition as it is more efficient than singleDimension since it does not need to determine the dimension for creating partitions. -Hashing also gives better distribution of data resulting in equal sized partitions and improving query performance - -To use this druid to automatically determine optimal partitions indexer must be given a target partition size. It can then find a good set of partition ranges on its own. - -#### Configuration for disabling auto-sharding and creating Fixed number of partitions - Druid can be configured to NOT run determine partitions and create a fixed number of shards by specifying numShards in hashed partitionsSpec. - e.g This configuration will skip determining optimal partitions and always create 4 shards for every segment granular interval +#### Hash-based partitioning ```json "partitionsSpec": { - "type": "hashed" - "numShards": 4 + "type": "hashed", + "targetPartitionSize": 5000000 } ``` +Hashed partitioning works by first selecting a number of segments, and then partitioning rows across those segments +according to the hash of all dimensions in each row. The number of segments is determined automatically based on the +cardinality of the input set and a target partition size. + +The configuration options are: + |property|description|required?| |--------|-----------|---------| -|type|type of partitionSpec to be used |no, default : singleDimension| -|targetPartitionSize|target number of rows to include in a partition, should be a number that targets segments of 700MB\~1GB.|yes| +|type|type of partitionSpec to be used |"hashed"| +|targetPartitionSize|target number of rows to include in a partition, should be a number that targets segments of 500MB\~1GB.|either this or numShards| +|numShards|specify the number of partitions directly, instead of a target partition size. Ingestion will run faster, since it can skip the step necessary to select a number of partitions automatically.|either this or targetPartitionSize| + +#### Single-dimension partitioning + +```json + "partitionsSpec": { + "type": "dimension", + "targetPartitionSize": 5000000 + } +``` + +Single-dimension partitioning works by first selecting a dimension to partition on, and then separating that dimension +into contiguous ranges. Each segment will contain all rows with values of that dimension in that range. For example, +your segments may be partitioned on the dimension "host" using the ranges "a.example.com" to "f.example.com" and +"f.example.com" to "z.example.com". By default, the dimension to use is determined automatically, although you can +override it with a specific dimension. + +The configuration options are: + +|property|description|required?| +|--------|-----------|---------| +|type|type of partitionSpec to be used |"dimension"| +|targetPartitionSize|target number of rows to include in a partition, should be a number that targets segments of 500MB\~1GB.|yes| +|maxPartitionSize|maximum number of rows to include in a partition. Defaults to 50% larger than the targetPartitionSize.|no| |partitionDimension|the dimension to partition on. Leave blank to select a dimension automatically.|no| -|assumeGrouped|assume input data has already been grouped on time and dimensions. This is faster, but can choose suboptimal partitions if the assumption is violated.|no| -|numShards|provides a way to manually override druid-auto sharding and specify the number of shards to create for each segment granular interval.It is only supported by hashed partitionSpec and targetPartitionSize must be set to -1|no| +|assumeGrouped|assume input data has already been grouped on time and dimensions. Ingestion will run faster, but can choose suboptimal partitions if the assumption is violated.|no| ### Updater job spec From 3d09edaae661e0139c9cca112ab588f770db68f4 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 17 Oct 2014 11:31:18 -0400 Subject: [PATCH 40/69] fix rabbit serde and add tests; fixes #794 --- rabbitmq/pom.xml | 8 ++- .../JacksonifiedConnectionFactory.java | 14 +++- .../rabbitmq/RabbitMQDruidModule.java | 2 +- .../rabbitmq/RabbitMQFirehoseConfig.java | 18 +++-- .../rabbitmq/RabbitMQFirehoseFactory.java | 57 +++++++++------ .../rabbitmq/RabbitMQFirehoseFactoryTest.java | 69 +++++++++++++++++++ 6 files changed, 139 insertions(+), 29 deletions(-) create mode 100644 rabbitmq/src/test/java/io/druid/examples/rabbitmq/RabbitMQFirehoseFactoryTest.java diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 745f10e88ac..e0d2e693414 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -1,5 +1,6 @@ - + 4.0.0 io.druid.extensions druid-rabbitmq @@ -17,6 +18,11 @@ io.druid druid-api + + io.druid + druid-processing + ${project.parent.version} + com.rabbitmq amqp-client diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java index 132fe3b6179..9f6e6f67ec8 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java @@ -20,8 +20,10 @@ package io.druid.firehose.rabbitmq; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Maps; import com.rabbitmq.client.ConnectionFactory; +import javax.annotation.Nullable; import java.net.URISyntaxException; import java.security.KeyManagementException; import java.security.NoSuchAlgorithmException; @@ -161,7 +163,17 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory @JsonProperty public Map getClientProperties() { - return super.getClientProperties(); + return Maps.transformEntries( + super.getClientProperties(), + new Maps.EntryTransformer() + { + @Override + public Object transformEntry(String key, Object value) + { + return value.toString(); + } + } + ); } @Override diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQDruidModule.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQDruidModule.java index 548cbcc1d1a..8ca79bbea99 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQDruidModule.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQDruidModule.java @@ -29,7 +29,7 @@ import java.util.List; /** */ -public class RabbitMQDruidModule implements DruidModule +public class RabbitMQDruidModule implements DruidModule { @Override public List getJacksonModules() diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseConfig.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseConfig.java index 7bae291c8a3..0b17e533144 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseConfig.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseConfig.java @@ -105,29 +105,35 @@ public class RabbitMQFirehoseConfig } @JsonProperty - public int getMaxRetries() { + public int getMaxRetries() + { return maxRetries; } - public void setMaxRetries(int maxRetries) { + public void setMaxRetries(int maxRetries) + { this.maxRetries = maxRetries; } @JsonProperty - public int getRetryIntervalSeconds() { + public int getRetryIntervalSeconds() + { return retryIntervalSeconds; } - public void setRetryIntervalSeconds(int retryIntervalSeconds) { + public void setRetryIntervalSeconds(int retryIntervalSeconds) + { this.retryIntervalSeconds = retryIntervalSeconds; } @JsonProperty - public long getMaxDurationSeconds() { + public long getMaxDurationSeconds() + { return maxDurationSeconds; } - public void setMaxDurationSeconds(int maxDurationSeconds) { + public void setMaxDurationSeconds(int maxDurationSeconds) + { this.maxDurationSeconds = maxDurationSeconds; } } diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java index 8504c70ed9d..ec650829e5a 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java @@ -50,14 +50,14 @@ import java.util.concurrent.LinkedBlockingQueue; /** * A FirehoseFactory for RabbitMQ. - * + *

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

* File: realtime.spec *

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

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

* For more information on RabbitMQ high availability please see: * http://www.rabbitmq.com/ha.html. */ @@ -105,14 +105,9 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory _queue; - public QueueingConsumer(Channel ch) { + public QueueingConsumer(Channel ch) + { this(ch, new LinkedBlockingQueue()); } - public QueueingConsumer(Channel ch, BlockingQueue q) { + public QueueingConsumer(Channel ch, BlockingQueue q) + { super(ch); this._queue = q; } - @Override public void handleShutdownSignal(String consumerTag, ShutdownSignalException sig) { + @Override + public void handleShutdownSignal(String consumerTag, ShutdownSignalException sig) + { _queue.clear(); } - @Override public void handleCancel(String consumerTag) throws IOException { + @Override + public void handleCancel(String consumerTag) throws IOException + { _queue.clear(); } - @Override public void handleDelivery(String consumerTag, - Envelope envelope, - AMQP.BasicProperties properties, - byte[] body) - throws IOException + @Override + public void handleDelivery( + String consumerTag, + Envelope envelope, + AMQP.BasicProperties properties, + byte[] body + ) + throws IOException { this._queue.add(new Delivery(envelope, properties, body)); } public Delivery nextDelivery() - throws InterruptedException, ShutdownSignalException, ConsumerCancelledException + throws InterruptedException, ShutdownSignalException, ConsumerCancelledException { return _queue.take(); } diff --git a/rabbitmq/src/test/java/io/druid/examples/rabbitmq/RabbitMQFirehoseFactoryTest.java b/rabbitmq/src/test/java/io/druid/examples/rabbitmq/RabbitMQFirehoseFactoryTest.java new file mode 100644 index 00000000000..dc52cc30c37 --- /dev/null +++ b/rabbitmq/src/test/java/io/druid/examples/rabbitmq/RabbitMQFirehoseFactoryTest.java @@ -0,0 +1,69 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.examples.rabbitmq; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.JSONParseSpec; +import io.druid.data.input.impl.SpatialDimensionSchema; +import io.druid.data.input.impl.StringInputRowParser; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.firehose.rabbitmq.JacksonifiedConnectionFactory; +import io.druid.firehose.rabbitmq.RabbitMQFirehoseConfig; +import io.druid.firehose.rabbitmq.RabbitMQFirehoseFactory; +import io.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; + +/** + */ +public class RabbitMQFirehoseFactoryTest +{ + private static final ObjectMapper mapper = new DefaultObjectMapper(); + + @Test + public void testSerde() throws Exception + { + RabbitMQFirehoseFactory factory = new RabbitMQFirehoseFactory( + new JacksonifiedConnectionFactory(), + new RabbitMQFirehoseConfig(), + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "auto"), + new DimensionsSpec( + Arrays.asList("dim"), + Lists.newArrayList(), + Lists.newArrayList() + ) + ), + null, null, null, null + ) + ); + + byte[] bytes = mapper.writeValueAsBytes(factory); + RabbitMQFirehoseFactory factory2 = mapper.readValue(bytes, RabbitMQFirehoseFactory.class); + byte[] bytes2 = mapper.writeValueAsBytes(factory2); + + Assert.assertArrayEquals(bytes, bytes2); + } +} From bf3d31e5cc350779823001cf3e0edf68fc1285bc Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 17 Oct 2014 12:32:37 -0400 Subject: [PATCH 41/69] address cr --- .../JacksonifiedConnectionFactory.java | 209 +++++++++++------- .../rabbitmq/RabbitMQFirehoseConfig.java | 151 ++++++++----- .../rabbitmq/RabbitMQFirehoseFactory.java | 9 +- .../rabbitmq/RabbitMQFirehoseFactoryTest.java | 36 ++- 4 files changed, 272 insertions(+), 133 deletions(-) diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java index 9f6e6f67ec8..4bd8ccb24a4 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java @@ -19,14 +19,11 @@ package io.druid.firehose.rabbitmq; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Maps; import com.rabbitmq.client.ConnectionFactory; -import javax.annotation.Nullable; -import java.net.URISyntaxException; -import java.security.KeyManagementException; -import java.security.NoSuchAlgorithmException; import java.util.Map; /** @@ -35,133 +32,137 @@ import java.util.Map; */ public class JacksonifiedConnectionFactory extends ConnectionFactory { + public static JacksonifiedConnectionFactory makeDefaultConnectionFactory() throws Exception + { + return new JacksonifiedConnectionFactory(null, 0, null, null, null, null, 0, 0, 0, 0, null); + } + + private final String host; + private final int port; + private final String username; + private final String password; + private final String virtualHost; + private final String uri; + private final int requestedChannelMax; + private final int requestedFrameMax; + private final int requestedHeartbeat; + private final int connectionTimeout; + private final Map clientProperties; + + @JsonCreator + public JacksonifiedConnectionFactory( + @JsonProperty("host") String host, + @JsonProperty("port") int port, + @JsonProperty("username") String username, + @JsonProperty("password") String password, + @JsonProperty("virtualHost") String virtualHost, + @JsonProperty("uri") String uri, + @JsonProperty("requestedChannelMax") int requestedChannelMax, + @JsonProperty("requestedFrameMax") int requestedFrameMax, + @JsonProperty("requestedHeartbeat") int requestedHeartbeat, + @JsonProperty("connectionTimeout") int connectionTimeout, + @JsonProperty("clientProperties") Map clientProperties + ) throws Exception + { + this.host = host == null ? super.getHost() : host; + this.port = port == 0 ? super.getPort() : port; + this.username = username == null ? super.getUsername() : username; + this.password = password == null ? super.getPassword() : password; + this.virtualHost = virtualHost == null ? super.getVirtualHost() : virtualHost; + this.uri = uri; + this.requestedChannelMax = requestedChannelMax == 0 ? super.getRequestedChannelMax() : requestedChannelMax; + this.requestedFrameMax = requestedFrameMax == 0 ? super.getRequestedFrameMax() : requestedFrameMax; + this.requestedHeartbeat = requestedHeartbeat == 0 ? super.getRequestedHeartbeat() : requestedHeartbeat; + this.connectionTimeout = connectionTimeout == 0 ? super.getConnectionTimeout() : connectionTimeout; + this.clientProperties = clientProperties == null ? super.getClientProperties() : clientProperties; + + super.setHost(this.host); + super.setPort(this.port); + super.setUsername(this.username); + super.setPassword(this.password); + super.setVirtualHost(this.virtualHost); + if (this.uri != null) { + super.setUri(this.uri); + } + super.setRequestedChannelMax(this.requestedChannelMax); + super.setRequestedFrameMax(this.requestedFrameMax); + super.setRequestedHeartbeat(this.requestedHeartbeat); + super.setConnectionTimeout(this.connectionTimeout); + super.setClientProperties(this.clientProperties); + } + @Override @JsonProperty public String getHost() { - return super.getHost(); - } - - @Override - public void setHost(String host) - { - super.setHost(host); + return host; } @Override @JsonProperty public int getPort() { - return super.getPort(); + return port; } - @Override - public void setPort(int port) - { - super.setPort(port); - } @Override @JsonProperty public String getUsername() { - return super.getUsername(); - } - - @Override - public void setUsername(String username) - { - super.setUsername(username); + return username; } @Override @JsonProperty public String getPassword() { - return super.getPassword(); - } - - @Override - public void setPassword(String password) - { - super.setPassword(password); + return password; } @Override @JsonProperty public String getVirtualHost() { - return super.getVirtualHost(); + return virtualHost; } - @Override - public void setVirtualHost(String virtualHost) - { - super.setVirtualHost(virtualHost); - } - - @Override @JsonProperty - public void setUri(String uriString) throws URISyntaxException, NoSuchAlgorithmException, KeyManagementException + public String getUri() { - super.setUri(uriString); + return uri; } @Override @JsonProperty public int getRequestedChannelMax() { - return super.getRequestedChannelMax(); - } - - @Override - public void setRequestedChannelMax(int requestedChannelMax) - { - super.setRequestedChannelMax(requestedChannelMax); + return requestedChannelMax; } @Override @JsonProperty public int getRequestedFrameMax() { - return super.getRequestedFrameMax(); - } - - @Override - public void setRequestedFrameMax(int requestedFrameMax) - { - super.setRequestedFrameMax(requestedFrameMax); + return requestedFrameMax; } @Override @JsonProperty public int getRequestedHeartbeat() { - return super.getRequestedHeartbeat(); - } - - @Override - public void setConnectionTimeout(int connectionTimeout) - { - super.setConnectionTimeout(connectionTimeout); + return requestedHeartbeat; } @Override @JsonProperty public int getConnectionTimeout() { - return super.getConnectionTimeout(); + return connectionTimeout; } - @Override - public void setRequestedHeartbeat(int requestedHeartbeat) - { - super.setRequestedHeartbeat(requestedHeartbeat); - } - - @Override - @JsonProperty - public Map getClientProperties() + @JsonProperty("clientProperties") + public Map getClientPropertiesForSerde() { return Maps.transformEntries( super.getClientProperties(), @@ -177,8 +178,68 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory } @Override - public void setClientProperties(Map clientProperties) + public boolean equals(Object o) { - super.setClientProperties(clientProperties); + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + JacksonifiedConnectionFactory that = (JacksonifiedConnectionFactory) o; + + if (connectionTimeout != that.connectionTimeout) { + return false; + } + if (port != that.port) { + return false; + } + if (requestedChannelMax != that.requestedChannelMax) { + return false; + } + if (requestedFrameMax != that.requestedFrameMax) { + return false; + } + if (requestedHeartbeat != that.requestedHeartbeat) { + return false; + } + if (clientProperties != null ? !clientProperties.equals(that.clientProperties) : that.clientProperties != null) { + return false; + } + if (host != null ? !host.equals(that.host) : that.host != null) { + return false; + } + if (password != null ? !password.equals(that.password) : that.password != null) { + return false; + } + if (uri != null ? !uri.equals(that.uri) : that.uri != null) { + return false; + } + if (username != null ? !username.equals(that.username) : that.username != null) { + return false; + } + if (virtualHost != null ? !virtualHost.equals(that.virtualHost) : that.virtualHost != null) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = host != null ? host.hashCode() : 0; + result = 31 * result + port; + result = 31 * result + (username != null ? username.hashCode() : 0); + result = 31 * result + (password != null ? password.hashCode() : 0); + result = 31 * result + (virtualHost != null ? virtualHost.hashCode() : 0); + result = 31 * result + (uri != null ? uri.hashCode() : 0); + result = 31 * result + requestedChannelMax; + result = 31 * result + requestedFrameMax; + result = 31 * result + requestedHeartbeat; + result = 31 * result + connectionTimeout; + result = 31 * result + (clientProperties != null ? clientProperties.hashCode() : 0); + return result; } } diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseConfig.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseConfig.java index 0b17e533144..325bc41cd92 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseConfig.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseConfig.java @@ -19,6 +19,7 @@ package io.druid.firehose.rabbitmq; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; /** @@ -26,17 +27,50 @@ import com.fasterxml.jackson.annotation.JsonProperty; */ public class RabbitMQFirehoseConfig { - private String queue = null; - private String exchange = null; - private String routingKey = null; - private boolean durable = false; - private boolean exclusive = false; - private boolean autoDelete = false; - // Lyra (auto reconnect) properties - private int maxRetries = 100; - private int retryIntervalSeconds = 2; - private long maxDurationSeconds = 5 * 60; + private static final int defaultMaxRetries = 100; + private static final int defaultRetryIntervalSeconds = 2; + private static final long defaultMaxDurationSeconds = 5 * 60; + + public static RabbitMQFirehoseConfig makeDefaultConfig() + { + return new RabbitMQFirehoseConfig(null, null, null, false, false, false, 0, 0, 0); + } + + private final String queue; + private final String exchange; + private final String routingKey; + private final boolean durable; + private final boolean exclusive; + private final boolean autoDelete; + private final int maxRetries; + private final int retryIntervalSeconds; + private final long maxDurationSeconds; + + @JsonCreator + public RabbitMQFirehoseConfig( + @JsonProperty("queue") String queue, + @JsonProperty("exchange") String exchange, + @JsonProperty("routingKey") String routingKey, + @JsonProperty("durable") boolean durable, + @JsonProperty("exclusive") boolean exclusive, + @JsonProperty("autoDelete") boolean autoDelete, + @JsonProperty("maxRetries") int maxRetries, + @JsonProperty("retryIntervalSeconds") int retryIntervalSeconds, + @JsonProperty("maxDurationSeconds") long maxDurationSeconds + ) + { + this.queue = queue; + this.exchange = exchange; + this.routingKey = routingKey; + this.durable = durable; + this.exclusive = exclusive; + this.autoDelete = autoDelete; + + this.maxRetries = maxRetries == 0 ? defaultMaxRetries : maxRetries; + this.retryIntervalSeconds = retryIntervalSeconds == 0 ? defaultRetryIntervalSeconds : retryIntervalSeconds; + this.maxDurationSeconds = maxDurationSeconds == 0 ? defaultMaxDurationSeconds : maxDurationSeconds; + } @JsonProperty public String getQueue() @@ -44,96 +78,109 @@ public class RabbitMQFirehoseConfig return queue; } - public void setQueue(String queue) - { - this.queue = queue; - } - @JsonProperty public String getExchange() { return exchange; } - public void setExchange(String exchange) - { - this.exchange = exchange; - } - @JsonProperty public String getRoutingKey() { return routingKey; } - public void setRoutingKey(String routingKey) - { - this.routingKey = routingKey; - } - @JsonProperty public boolean isDurable() { return durable; } - public void setDurable(boolean durable) - { - this.durable = durable; - } - @JsonProperty public boolean isExclusive() { return exclusive; } - public void setExclusive(boolean exclusive) - { - this.exclusive = exclusive; - } - @JsonProperty public boolean isAutoDelete() { return autoDelete; } - public void setAutoDelete(boolean autoDelete) - { - this.autoDelete = autoDelete; - } - @JsonProperty public int getMaxRetries() { return maxRetries; } - public void setMaxRetries(int maxRetries) - { - this.maxRetries = maxRetries; - } - @JsonProperty public int getRetryIntervalSeconds() { return retryIntervalSeconds; } - public void setRetryIntervalSeconds(int retryIntervalSeconds) - { - this.retryIntervalSeconds = retryIntervalSeconds; - } - @JsonProperty public long getMaxDurationSeconds() { return maxDurationSeconds; } - public void setMaxDurationSeconds(int maxDurationSeconds) + @Override + public boolean equals(Object o) { - this.maxDurationSeconds = maxDurationSeconds; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + RabbitMQFirehoseConfig that = (RabbitMQFirehoseConfig) o; + + if (autoDelete != that.autoDelete) { + return false; + } + if (durable != that.durable) { + return false; + } + if (exclusive != that.exclusive) { + return false; + } + if (maxDurationSeconds != that.maxDurationSeconds) { + return false; + } + if (maxRetries != that.maxRetries) { + return false; + } + if (retryIntervalSeconds != that.retryIntervalSeconds) { + return false; + } + if (exchange != null ? !exchange.equals(that.exchange) : that.exchange != null) { + return false; + } + if (queue != null ? !queue.equals(that.queue) : that.queue != null) { + return false; + } + if (routingKey != null ? !routingKey.equals(that.routingKey) : that.routingKey != null) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = queue != null ? queue.hashCode() : 0; + result = 31 * result + (exchange != null ? exchange.hashCode() : 0); + result = 31 * result + (routingKey != null ? routingKey.hashCode() : 0); + result = 31 * result + (durable ? 1 : 0); + result = 31 * result + (exclusive ? 1 : 0); + result = 31 * result + (autoDelete ? 1 : 0); + result = 31 * result + maxRetries; + result = 31 * result + retryIntervalSeconds; + result = 31 * result + (int) (maxDurationSeconds ^ (maxDurationSeconds >>> 32)); + return result; } } diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java index ec650829e5a..f38398bcc84 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java @@ -26,7 +26,6 @@ import com.metamx.common.logger.Logger; import com.rabbitmq.client.AMQP; import com.rabbitmq.client.Channel; import com.rabbitmq.client.Connection; -import com.rabbitmq.client.ConnectionFactory; import com.rabbitmq.client.ConsumerCancelledException; import com.rabbitmq.client.DefaultConsumer; import com.rabbitmq.client.Envelope; @@ -114,10 +113,12 @@ public class RabbitMQFirehoseFactory implements FirehoseFactoryof("hi", "bye") + ); + RabbitMQFirehoseFactory factory = new RabbitMQFirehoseFactory( - new JacksonifiedConnectionFactory(), - new RabbitMQFirehoseConfig(), + connectionFactory, + config, new StringInputRowParser( new JSONParseSpec( new TimestampSpec("timestamp", "auto"), new DimensionsSpec( - Arrays.asList("dim"), + Arrays.asList("dim"), Lists.newArrayList(), Lists.newArrayList() ) @@ -65,5 +92,8 @@ public class RabbitMQFirehoseFactoryTest byte[] bytes2 = mapper.writeValueAsBytes(factory2); Assert.assertArrayEquals(bytes, bytes2); + + Assert.assertEquals(factory.getConfig(), factory2.getConfig()); + Assert.assertEquals(factory.getConnectionFactory(), factory2.getConnectionFactory()); } } From 56b8ef4629073451cd6a0fd0bb10b65900dbb5a8 Mon Sep 17 00:00:00 2001 From: fjy Date: Sun, 19 Oct 2014 16:04:52 -0700 Subject: [PATCH 42/69] one more comment --- .../druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java index 4bd8ccb24a4..56cb040f89f 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java @@ -64,6 +64,8 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory @JsonProperty("clientProperties") Map clientProperties ) throws Exception { + super(); + this.host = host == null ? super.getHost() : host; this.port = port == 0 ? super.getPort() : port; this.username = username == null ? super.getUsername() : username; From cc18bdeb0d157f3cffdb223236d22488cc323a33 Mon Sep 17 00:00:00 2001 From: fjy Date: Sun, 19 Oct 2014 16:14:26 -0700 Subject: [PATCH 43/69] another test for defaults --- .../JacksonifiedConnectionFactory.java | 4 +- .../rabbitmq/RabbitMQFirehoseFactoryTest.java | 40 +++++++++++++++++++ 2 files changed, 42 insertions(+), 2 deletions(-) diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java index 56cb040f89f..a4a39bcc9cd 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java @@ -65,7 +65,7 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory ) throws Exception { super(); - + this.host = host == null ? super.getHost() : host; this.port = port == 0 ? super.getPort() : port; this.username = username == null ? super.getUsername() : username; @@ -206,7 +206,7 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory if (requestedHeartbeat != that.requestedHeartbeat) { return false; } - if (clientProperties != null ? !clientProperties.equals(that.clientProperties) : that.clientProperties != null) { + if (clientProperties != null ? Maps.difference(clientProperties, that.clientProperties).areEqual() : that.clientProperties != null) { return false; } if (host != null ? !host.equals(that.host) : that.host != null) { diff --git a/rabbitmq/src/test/java/io/druid/examples/rabbitmq/RabbitMQFirehoseFactoryTest.java b/rabbitmq/src/test/java/io/druid/examples/rabbitmq/RabbitMQFirehoseFactoryTest.java index 09bd205df25..eca04717fc2 100644 --- a/rabbitmq/src/test/java/io/druid/examples/rabbitmq/RabbitMQFirehoseFactoryTest.java +++ b/rabbitmq/src/test/java/io/druid/examples/rabbitmq/RabbitMQFirehoseFactoryTest.java @@ -22,6 +22,7 @@ package io.druid.examples.rabbitmq; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.rabbitmq.client.ConnectionFactory; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.SpatialDimensionSchema; @@ -96,4 +97,43 @@ public class RabbitMQFirehoseFactoryTest Assert.assertEquals(factory.getConfig(), factory2.getConfig()); Assert.assertEquals(factory.getConnectionFactory(), factory2.getConnectionFactory()); } + + @Test + public void testDefaultSerde() throws Exception + { + RabbitMQFirehoseConfig config = RabbitMQFirehoseConfig.makeDefaultConfig(); + + JacksonifiedConnectionFactory connectionFactory = JacksonifiedConnectionFactory.makeDefaultConnectionFactory(); + + RabbitMQFirehoseFactory factory = new RabbitMQFirehoseFactory( + connectionFactory, + config, + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "auto"), + new DimensionsSpec( + Arrays.asList("dim"), + Lists.newArrayList(), + Lists.newArrayList() + ) + ), + null, null, null, null + ) + ); + + byte[] bytes = mapper.writeValueAsBytes(factory); + RabbitMQFirehoseFactory factory2 = mapper.readValue(bytes, RabbitMQFirehoseFactory.class); + byte[] bytes2 = mapper.writeValueAsBytes(factory2); + + Assert.assertArrayEquals(bytes, bytes2); + + Assert.assertEquals(factory.getConfig(), factory2.getConfig()); + Assert.assertEquals(factory.getConnectionFactory(), factory2.getConnectionFactory()); + + Assert.assertEquals(300, factory2.getConfig().getMaxDurationSeconds()); + + Assert.assertEquals(ConnectionFactory.DEFAULT_HOST, factory2.getConnectionFactory().getHost()); + Assert.assertEquals(ConnectionFactory.DEFAULT_USER, factory2.getConnectionFactory().getUsername()); + Assert.assertEquals(ConnectionFactory.DEFAULT_AMQP_PORT, factory2.getConnectionFactory().getPort()); + } } From 9f75df63f34d469ae27ab965425f0f0cebc6b984 Mon Sep 17 00:00:00 2001 From: fjy Date: Sun, 19 Oct 2014 16:38:21 -0700 Subject: [PATCH 44/69] fix more things --- rabbitmq/pom.xml | 11 ++--- .../JacksonifiedConnectionFactory.java | 40 +++++++++++++------ 2 files changed, 33 insertions(+), 18 deletions(-) diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index e0d2e693414..a35f3b16c15 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -18,11 +18,6 @@ io.druid druid-api - - io.druid - druid-processing - ${project.parent.version} - com.rabbitmq amqp-client @@ -45,5 +40,11 @@ commons-cli test + + io.druid + druid-processing + ${project.parent.version} + test + \ No newline at end of file diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java index a4a39bcc9cd..28f1d0d14d1 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/JacksonifiedConnectionFactory.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Maps; import com.rabbitmq.client.ConnectionFactory; +import com.rabbitmq.client.LongString; import java.util.Map; @@ -37,6 +38,24 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory return new JacksonifiedConnectionFactory(null, 0, null, null, null, null, 0, 0, 0, 0, null); } + private static Map getSerializableClientProperties(final Map clientProperties) + { + return Maps.transformEntries( + clientProperties, + new Maps.EntryTransformer() + { + @Override + public Object transformEntry(String key, Object value) + { + if (value instanceof LongString) { + return value.toString(); + } + return value; + } + } + ); + } + private final String host; private final int port; private final String username; @@ -164,19 +183,9 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory } @JsonProperty("clientProperties") - public Map getClientPropertiesForSerde() + public Map getSerializableClientProperties() { - return Maps.transformEntries( - super.getClientProperties(), - new Maps.EntryTransformer() - { - @Override - public Object transformEntry(String key, Object value) - { - return value.toString(); - } - } - ); + return getSerializableClientProperties(clientProperties); } @Override @@ -206,7 +215,12 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory if (requestedHeartbeat != that.requestedHeartbeat) { return false; } - if (clientProperties != null ? Maps.difference(clientProperties, that.clientProperties).areEqual() : that.clientProperties != null) { + if (clientProperties != null + ? !Maps.difference( + getSerializableClientProperties(clientProperties), + getSerializableClientProperties(that.clientProperties) + ).areEqual() + : that.clientProperties != null) { return false; } if (host != null ? !host.equals(that.host) : that.host != null) { From 63a4f2dc055598aab2bf36bc4d55769c9ec13a49 Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Sun, 19 Oct 2014 18:29:34 -0700 Subject: [PATCH 45/69] fixed broken internal link --- docs/content/Querying.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Querying.md b/docs/content/Querying.md index 895a1cf2239..8fc926b426d 100644 --- a/docs/content/Querying.md +++ b/docs/content/Querying.md @@ -133,7 +133,7 @@ Properties shared by all query types |search|searchDimensions|Dimensions to apply the search query to. If not specified, it will search through all dimensions.|no| |search|query|The query portion of the search query. This is essentially a predicate that specifies if something matches.|yes| -Query Context +Query Context ------------- |property |default | description | From b7c4983a918edfc1dde39add84f4e35376dcb698 Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Sun, 19 Oct 2014 18:38:20 -0700 Subject: [PATCH 46/69] added syntax highlighting --- docs/content/Granularities.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/content/Granularities.md b/docs/content/Granularities.md index d4f3ea73141..e8a3e4fc2bf 100644 --- a/docs/content/Granularities.md +++ b/docs/content/Granularities.md @@ -21,13 +21,13 @@ Duration granularities are specified as an exact duration in milliseconds and ti They also support specifying an optional origin, which defines where to start counting time buckets from (defaults to 1970-01-01T00:00:00Z). -``` +```javascript {"type": "duration", "duration": "7200000"} ``` This chunks up every 2 hours. -``` +```javascript {"type": "duration", "duration": "3600000", "origin": "2012-01-01T00:30:00Z"} ``` @@ -39,13 +39,13 @@ Period granularities are specified as arbitrary period combinations of years, mo Time zone is optional (defaults to UTC). Origin is optional (defaults to 1970-01-01T00:00:00 in the given time zone). -``` +```javascript {"type": "period", "period": "P2D", "timeZone": "America/Los_Angeles"} ``` This will bucket by two-day chunks in the Pacific timezone. -``` +```javascript {"type": "period", "period": "P3M", "timeZone": "America/Los_Angeles", "origin": "2012-02-01T00:00:00-08:00"} ``` From 1dd9aeba4f22db24d12984b27976d846ddc978f0 Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Sun, 19 Oct 2014 21:22:10 -0700 Subject: [PATCH 47/69] removed link with no target --- docs/content/Querying.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Querying.md b/docs/content/Querying.md index 8fc926b426d..dd556b2740f 100644 --- a/docs/content/Querying.md +++ b/docs/content/Querying.md @@ -66,7 +66,7 @@ The dataSource JSON field shown next identifies where to apply the query. In thi "dataSource": "randSeq", ``` -The granularity JSON field specifies the bucket size for values. It could be a built-in time interval like "second", "minute", "fifteen_minute", "thirty_minute", "hour" or "day". It can also be an expression like `{"type": "period", "period":"PT6m"}` meaning "6 minute buckets". See [Granularities](Granularities.html) for more information on the different options for this field. In this example, it is set to the special value "all" which means [bucket all data points together into the same time bucket]() +The granularity JSON field specifies the bucket size for values. It could be a built-in time interval like "second", "minute", "fifteen_minute", "thirty_minute", "hour" or "day". It can also be an expression like `{"type": "period", "period":"PT6m"}` meaning "6 minute buckets". See [Granularities](Granularities.html) for more information on the different options for this field. In this example, it is set to the special value "all" which means bucket all data points together into the same time bucket. ```javascript "granularity": "all", From 38cb73e0cccc61f00b43ef4bb4944c0c1550787c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 20 Oct 2014 13:19:21 -0700 Subject: [PATCH 48/69] javacript tiered broker selector strategy --- ...avascriptTieredBrokerSelectorStrategy.java | 60 ++++++++++ .../router/TieredBrokerSelectorStrategy.java | 3 +- ...criptTieredBrokerSelectorStrategyTest.java | 103 ++++++++++++++++++ 3 files changed, 165 insertions(+), 1 deletion(-) create mode 100644 server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java create mode 100644 server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java diff --git a/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java b/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java new file mode 100644 index 00000000000..cce7fb25554 --- /dev/null +++ b/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java @@ -0,0 +1,60 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.server.router; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.api.client.repackaged.com.google.common.base.Throwables; +import com.google.common.base.Optional; +import io.druid.query.Query; + +import javax.script.Compilable; +import javax.script.Invocable; +import javax.script.ScriptEngine; +import javax.script.ScriptEngineManager; +import javax.script.ScriptException; + +public class JavascriptTieredBrokerSelectorStrategy implements TieredBrokerSelectorStrategy +{ + private final SelectorFunction function; + + public JavascriptTieredBrokerSelectorStrategy(@JsonProperty("function") String function) + { + final ScriptEngine engine = new ScriptEngineManager().getEngineByName("javascript"); + try { + ((Compilable)engine).compile("var apply = " + function).eval(); + } catch(ScriptException e) { + Throwables.propagate(e); + } + this.function = ((Invocable)engine).getInterface(SelectorFunction.class); + } + + @Override + public Optional getBrokerServiceName( + TieredBrokerConfig config, Query query + ) + { + return Optional.fromNullable(function.apply(config, query)); + } + + private static interface SelectorFunction + { + public String apply(TieredBrokerConfig config, Query query); + } +} diff --git a/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java b/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java index 40a7714d870..74c1091397e 100644 --- a/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java +++ b/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java @@ -29,7 +29,8 @@ import io.druid.query.Query; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = "timeBoundary", value = TimeBoundaryTieredBrokerSelectorStrategy.class), - @JsonSubTypes.Type(name = "priority", value = PriorityTieredBrokerSelectorStrategy.class) + @JsonSubTypes.Type(name = "priority", value = PriorityTieredBrokerSelectorStrategy.class), + @JsonSubTypes.Type(name = "javascript", value = JavascriptTieredBrokerSelectorStrategy.class) }) public interface TieredBrokerSelectorStrategy diff --git a/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java b/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java new file mode 100644 index 00000000000..78a6db3c59d --- /dev/null +++ b/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java @@ -0,0 +1,103 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.server.router; + +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import io.druid.query.Druids; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import org.junit.Assert; +import org.junit.Test; + +import java.util.LinkedHashMap; + +public class JavascriptTieredBrokerSelectorStrategyTest +{ + + @Test + public void testGetBrokerServiceName() throws Exception + { + TieredBrokerSelectorStrategy jsStrategy = new JavascriptTieredBrokerSelectorStrategy( + "function (config, query) { if (config.getTierToBrokerMap().values().size() > 0 && query.getAggregatorSpecs && query.getAggregatorSpecs().size() <= 2) { return config.getTierToBrokerMap().values().toArray()[0] } else { return config.getDefaultBrokerServiceName() } }" + ); + + final LinkedHashMap tierBrokerMap = new LinkedHashMap<>(); + tierBrokerMap.put("fast", "druid/fastBroker"); + tierBrokerMap.put("slow", "druid/broker"); + + final TieredBrokerConfig tieredBrokerConfig = new TieredBrokerConfig() + { + @Override + public String getDefaultBrokerServiceName() + { + return "druid/broker"; + } + + @Override + public LinkedHashMap getTierToBrokerMap() + { + return tierBrokerMap; + } + }; + + final Druids.TimeseriesQueryBuilder queryBuilder = Druids.newTimeseriesQueryBuilder().dataSource("test") + .intervals("2014/2015") + .aggregators( + ImmutableList.of( + new CountAggregatorFactory("count") + ) + ); + + Assert.assertEquals( + Optional.of("druid/fastBroker"), + jsStrategy.getBrokerServiceName( + tieredBrokerConfig, + queryBuilder.build() + ) + ); + + + Assert.assertEquals( + Optional.of("druid/broker"), + jsStrategy.getBrokerServiceName( + tieredBrokerConfig, + Druids.newTimeBoundaryQueryBuilder().dataSource("test").bound("maxTime").build() + ) + ); + + Assert.assertEquals( + Optional.of("druid/broker"), + jsStrategy.getBrokerServiceName( + tieredBrokerConfig, + queryBuilder.aggregators( + ImmutableList.of( + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("longSum", "a"), + new DoubleSumAggregatorFactory("doubleSum", "b") + ) + ).build() + ) + ); + + } +} From b32ce31eff5beeca3e890b461ce88dacc1e13440 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 20 Oct 2014 14:11:49 -0700 Subject: [PATCH 49/69] add documentation --- docs/content/Router.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/docs/content/Router.md b/docs/content/Router.md index 73849164216..f046fe4b468 100644 --- a/docs/content/Router.md +++ b/docs/content/Router.md @@ -118,3 +118,16 @@ Including this strategy means all timeBoundary queries are always routed to the ``` Queries with a priority set to less than minPriority are routed to the lowest priority broker. Queries with priority set to greater than maxPriority are routed to the highest priority broker. By default, minPriority is 0 and maxPriority is 1. Using these default values, if a query with priority 0 (the default query priority is 0) is sent, the query skips the priority selection logic. + +### javascript + +Allows defining arbitrary routing rules using a JavaScript function. The function is passed the configuration and the query to be executed, and returns the tier it should be routed to, or null for the default tier. + +*Example*: a function that return the highest priority broker unless the given query has more than two aggregators. + +```json +{ + "type" : "javascript", + "function" : "function (config, query) { if (config.getTierToBrokerMap().values().size() > 0 && query.getAggregatorSpecs && query.getAggregatorSpecs().size() <= 2) { return config.getTierToBrokerMap().values().toArray()[0] } else { return config.getDefaultBrokerServiceName() } }" +} +``` From 23a59db566bc054b3231f7c027239f0239bdaebb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 20 Oct 2014 16:23:45 -0700 Subject: [PATCH 50/69] fix ordering for partitions with same start time, fixes #796 Partitions with same start time may produce results out of order, especially in the presence of data gaps. Results for those segments must be re-ordered prior to being merged --- .../druid/client/CachingClusteredClient.java | 27 ++++++--- .../client/CachingClusteredClientTest.java | 57 +++++++++++++++++++ 2 files changed, 75 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index d0135eb7f4a..f189e060670 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -25,7 +25,6 @@ import com.google.common.base.Function; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -261,14 +260,22 @@ public class CachingClusteredClient implements QueryRunner Ordering.natural().onResultOf(Pair.>lhsFn()) ); - final Sequence> seq = Sequences.simple( - Iterables.transform(listOfSequences, Pair.>rhsFn()) - ); - if (strategy == null) { - return toolChest.mergeSequences(seq); - } else { - return strategy.mergeSequences(seq); + final List> orderedSequences = Lists.newLinkedList(); + DateTime unorderedStart = null; + List> unordered = Lists.newLinkedList(); + for (Pair> sequencePair : listOfSequences) { + if (unorderedStart != null && unorderedStart.getMillis() != sequencePair.lhs.getMillis()) { + orderedSequences.add(toolChest.mergeSequencesUnordered(Sequences.simple(unordered))); + unordered = Lists.newLinkedList(); + } + unorderedStart = sequencePair.lhs; + unordered.add(sequencePair.rhs); } + if(!unordered.isEmpty()) { + orderedSequences.add(toolChest.mergeSequencesUnordered(Sequences.simple(unordered))); + } + + return toolChest.mergeSequences(Sequences.simple(orderedSequences)); } private void addSequencesFromCache(ArrayList>> listOfSequences) @@ -332,7 +339,9 @@ public class CachingClusteredClient implements QueryRunner if (!server.isAssignable() || !populateCache || isBySegment) { resultSeqToAdd = clientQueryable.run(query.withQuerySegmentSpec(segmentSpec)); } else { - resultSeqToAdd = toolChest.mergeSequences( + // this could be more efficient, since we only need to reorder results + // for batches of segments with the same segment start time. + resultSeqToAdd = toolChest.mergeSequencesUnordered( Sequences.map( clientQueryable.run(rewrittenQuery.withQuerySegmentSpec(segmentSpec)), new Function>() diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index f99f3d4346b..c59da0c9906 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -309,6 +309,63 @@ public class CachingClusteredClientTest ); } + @Test + public void testTimeseriesMergingOutOfOrderPartitions() throws Exception + { + final Druids.TimeseriesQueryBuilder builder = Druids.newTimeseriesQueryBuilder() + .dataSource(DATA_SOURCE) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .aggregators(AGGS) + .postAggregators(POST_AGGS) + .context(CONTEXT); + + QueryRunner runner = new FinalizeResultsQueryRunner(client, new TimeseriesQueryQueryToolChest(new QueryConfig())); + + testQueryCaching( + runner, + builder.build(), + new Interval("2011-01-05/2011-01-10"), + makeTimeResults( + new DateTime("2011-01-05T02"), 80, 100, + new DateTime("2011-01-06T02"), 420, 520, + new DateTime("2011-01-07T02"), 12, 2194, + new DateTime("2011-01-08T02"), 59, 201, + new DateTime("2011-01-09T02"), 181, 52 + ), + new Interval("2011-01-05/2011-01-10"), + makeTimeResults( + new DateTime("2011-01-05T00"), 85, 102, + new DateTime("2011-01-06T00"), 412, 521, + new DateTime("2011-01-07T00"), 122, 21894, + new DateTime("2011-01-08T00"), 5, 20, + new DateTime("2011-01-09T00"), 18, 521 + ) + ); + + TestHelper.assertExpectedResults( + makeRenamedTimeResults( + new DateTime("2011-01-05T00"), 85, 102, + new DateTime("2011-01-05T02"), 80, 100, + new DateTime("2011-01-06T00"), 412, 521, + new DateTime("2011-01-06T02"), 420, 520, + new DateTime("2011-01-07T00"), 122, 21894, + new DateTime("2011-01-07T02"), 12, 2194, + new DateTime("2011-01-08T00"), 5, 20, + new DateTime("2011-01-08T02"), 59, 201, + new DateTime("2011-01-09T00"), 18, 521, + new DateTime("2011-01-09T02"), 181, 52 + ), + runner.run( + builder.intervals("2011-01-05/2011-01-10") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS) + .build() + ) + ); + } + @Test @SuppressWarnings("unchecked") public void testTimeseriesCachingTimeZone() throws Exception From 9d5e65913da00f280c38424d70d0e7d9601129fe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 21 Oct 2014 14:09:11 -0700 Subject: [PATCH 51/69] fix serde --- docs/content/Aggregations.md | 2 +- ...avascriptTieredBrokerSelectorStrategy.java | 66 ++++++++++++++++--- .../router/TieredBrokerSelectorStrategy.java | 2 +- ...criptTieredBrokerSelectorStrategyTest.java | 24 +++++-- 4 files changed, 78 insertions(+), 16 deletions(-) diff --git a/docs/content/Aggregations.md b/docs/content/Aggregations.md index 29740a2858c..abd4780b025 100644 --- a/docs/content/Aggregations.md +++ b/docs/content/Aggregations.md @@ -159,4 +159,4 @@ Uses [HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf) to ```json { "type" : "hyperUnique", "name" : , "fieldName" : } -``` \ No newline at end of file +``` diff --git a/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java b/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java index cce7fb25554..629c6a6873e 100644 --- a/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java +++ b/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java @@ -19,9 +19,11 @@ package io.druid.server.router; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.api.client.repackaged.com.google.common.base.Throwables; import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; import io.druid.query.Query; import javax.script.Compilable; @@ -30,19 +32,29 @@ import javax.script.ScriptEngine; import javax.script.ScriptEngineManager; import javax.script.ScriptException; -public class JavascriptTieredBrokerSelectorStrategy implements TieredBrokerSelectorStrategy +public class JavaScriptTieredBrokerSelectorStrategy implements TieredBrokerSelectorStrategy { - private final SelectorFunction function; - - public JavascriptTieredBrokerSelectorStrategy(@JsonProperty("function") String function) + private static interface SelectorFunction { + public String apply(TieredBrokerConfig config, Query query); + } + + private final SelectorFunction fnSelector; + private final String function; + + @JsonCreator + public JavaScriptTieredBrokerSelectorStrategy(@JsonProperty("function") String fn) + { + Preconditions.checkNotNull(fn, "function must not be null"); + final ScriptEngine engine = new ScriptEngineManager().getEngineByName("javascript"); try { - ((Compilable)engine).compile("var apply = " + function).eval(); + ((Compilable)engine).compile("var apply = " + fn).eval(); } catch(ScriptException e) { Throwables.propagate(e); } - this.function = ((Invocable)engine).getInterface(SelectorFunction.class); + this.function = fn; + this.fnSelector = ((Invocable)engine).getInterface(SelectorFunction.class); } @Override @@ -50,11 +62,45 @@ public class JavascriptTieredBrokerSelectorStrategy implements TieredBrokerSelec TieredBrokerConfig config, Query query ) { - return Optional.fromNullable(function.apply(config, query)); + return Optional.fromNullable(fnSelector.apply(config, query)); } - private static interface SelectorFunction + @JsonProperty + public String getFunction() { - public String apply(TieredBrokerConfig config, Query query); + return function; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + JavaScriptTieredBrokerSelectorStrategy that = (JavaScriptTieredBrokerSelectorStrategy) o; + + if (!function.equals(that.function)) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + return function.hashCode(); + } + + @Override + public String toString() + { + return "JavascriptTieredBrokerSelectorStrategy{" + + "function='" + function + '\'' + + '}'; } } diff --git a/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java b/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java index 74c1091397e..10bffe4fb37 100644 --- a/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java +++ b/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java @@ -30,7 +30,7 @@ import io.druid.query.Query; @JsonSubTypes(value = { @JsonSubTypes.Type(name = "timeBoundary", value = TimeBoundaryTieredBrokerSelectorStrategy.class), @JsonSubTypes.Type(name = "priority", value = PriorityTieredBrokerSelectorStrategy.class), - @JsonSubTypes.Type(name = "javascript", value = JavascriptTieredBrokerSelectorStrategy.class) + @JsonSubTypes.Type(name = "javascript", value = JavaScriptTieredBrokerSelectorStrategy.class) }) public interface TieredBrokerSelectorStrategy diff --git a/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java b/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java index 78a6db3c59d..3f7e9f14947 100644 --- a/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java +++ b/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java @@ -19,8 +19,10 @@ package io.druid.server.router; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; +import io.druid.jackson.DefaultObjectMapper; import io.druid.query.Druids; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -31,15 +33,29 @@ import org.junit.Test; import java.util.LinkedHashMap; -public class JavascriptTieredBrokerSelectorStrategyTest +public class JavaScriptTieredBrokerSelectorStrategyTest { + final TieredBrokerSelectorStrategy jsStrategy = new JavaScriptTieredBrokerSelectorStrategy( + "function (config, query) { if (config.getTierToBrokerMap().values().size() > 0 && query.getAggregatorSpecs && query.getAggregatorSpecs().size() <= 2) { return config.getTierToBrokerMap().values().toArray()[0] } else { return config.getDefaultBrokerServiceName() } }" + ); + + @Test + public void testSerde() throws Exception + { + ObjectMapper mapper = new DefaultObjectMapper(); + Assert.assertEquals( + jsStrategy, + mapper.readValue( + mapper.writeValueAsString(jsStrategy), + JavaScriptTieredBrokerSelectorStrategy.class + ) + ); + } @Test public void testGetBrokerServiceName() throws Exception { - TieredBrokerSelectorStrategy jsStrategy = new JavascriptTieredBrokerSelectorStrategy( - "function (config, query) { if (config.getTierToBrokerMap().values().size() > 0 && query.getAggregatorSpecs && query.getAggregatorSpecs().size() <= 2) { return config.getTierToBrokerMap().values().toArray()[0] } else { return config.getDefaultBrokerServiceName() } }" - ); + final LinkedHashMap tierBrokerMap = new LinkedHashMap<>(); tierBrokerMap.put("fast", "druid/fastBroker"); From fd7d3e5aceabb568c37f7eb31db2a4f48b0dffe9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 21 Oct 2014 15:30:29 -0700 Subject: [PATCH 52/69] make it work with Java 8 --- .../server/router/JavascriptTieredBrokerSelectorStrategy.java | 2 +- .../router/JavascriptTieredBrokerSelectorStrategyTest.java | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java b/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java index 629c6a6873e..afc7997de43 100644 --- a/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java +++ b/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java @@ -34,7 +34,7 @@ import javax.script.ScriptException; public class JavaScriptTieredBrokerSelectorStrategy implements TieredBrokerSelectorStrategy { - private static interface SelectorFunction + public static interface SelectorFunction { public String apply(TieredBrokerConfig config, Query query); } diff --git a/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java b/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java index 3f7e9f14947..2b2162148ed 100644 --- a/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java +++ b/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java @@ -55,8 +55,6 @@ public class JavaScriptTieredBrokerSelectorStrategyTest @Test public void testGetBrokerServiceName() throws Exception { - - final LinkedHashMap tierBrokerMap = new LinkedHashMap<>(); tierBrokerMap.put("fast", "druid/fastBroker"); tierBrokerMap.put("slow", "druid/broker"); From 8735272ce39c35cfee3e79e2455b798fa6685855 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 21 Oct 2014 15:36:49 -0700 Subject: [PATCH 53/69] fix file casing --- ...rStrategy.java => JavaScriptTieredBrokerSelectorStrategy.java} | 0 ...yTest.java => JavaScriptTieredBrokerSelectorStrategyTest.java} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename server/src/main/java/io/druid/server/router/{JavascriptTieredBrokerSelectorStrategy.java => JavaScriptTieredBrokerSelectorStrategy.java} (100%) rename server/src/test/java/io/druid/server/router/{JavascriptTieredBrokerSelectorStrategyTest.java => JavaScriptTieredBrokerSelectorStrategyTest.java} (100%) diff --git a/server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java b/server/src/main/java/io/druid/server/router/JavaScriptTieredBrokerSelectorStrategy.java similarity index 100% rename from server/src/main/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategy.java rename to server/src/main/java/io/druid/server/router/JavaScriptTieredBrokerSelectorStrategy.java diff --git a/server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java b/server/src/test/java/io/druid/server/router/JavaScriptTieredBrokerSelectorStrategyTest.java similarity index 100% rename from server/src/test/java/io/druid/server/router/JavascriptTieredBrokerSelectorStrategyTest.java rename to server/src/test/java/io/druid/server/router/JavaScriptTieredBrokerSelectorStrategyTest.java From 2d96bc5f1feef16d682cc34764d59c049ae8b101 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 21 Oct 2014 16:08:56 -0700 Subject: [PATCH 54/69] more FAQ docs --- docs/content/Best-Practices.md | 5 +++ docs/content/Coordinator.md | 4 +-- docs/content/Ingestion-FAQ.md | 9 +++++ docs/content/Rule-Configuration.md | 55 ++++++++++++++++++++++++++---- 4 files changed, 64 insertions(+), 9 deletions(-) diff --git a/docs/content/Best-Practices.md b/docs/content/Best-Practices.md index f4cd8f32595..9688a52f966 100644 --- a/docs/content/Best-Practices.md +++ b/docs/content/Best-Practices.md @@ -21,3 +21,8 @@ SSDs are highly recommended for historical and real-time nodes if you are not ru Although Druid supports schemaless ingestion of dimensions, because of https://github.com/metamx/druid/issues/658, you may sometimes get bigger segments than necessary. To ensure segments are as compact as possible, providing dimension names in lexicographic order is recommended. This may require some ETL processing on your data however. +# Read FAQs + +You should read common problems people have here: +1) [Ingestion-FAQ](Ingestion-FAQ.html) +2) [Performance-FAQ](Performance-FAQ.html) \ No newline at end of file diff --git a/docs/content/Coordinator.md b/docs/content/Coordinator.md index 9021cbe1dff..42d1d17041d 100644 --- a/docs/content/Coordinator.md +++ b/docs/content/Coordinator.md @@ -20,9 +20,7 @@ io.druid.cli.Main server coordinator Rules ----- -Segments are loaded and dropped from the cluster based on a set of rules. Rules indicate how segments should be assigned to different historical node tiers and how many replicants of a segment should exist in each tier. Rules may also indicate when segments should be dropped entirely from the cluster. The coordinator loads a set of rules from the database. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The coordinator will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule. - -For more information on rules, see [Rule Configuration](Rule-Configuration.html). +Segments can be automatically loaded and dropped from the cluster based on a set of rules. For more information on rules, see [Rule Configuration](Rule-Configuration.html). Cleaning Up Segments -------------------- diff --git a/docs/content/Ingestion-FAQ.md b/docs/content/Ingestion-FAQ.md index fc83907c47a..ecf6b2ccdac 100644 --- a/docs/content/Ingestion-FAQ.md +++ b/docs/content/Ingestion-FAQ.md @@ -21,6 +21,12 @@ druid.storage.bucket=druid druid.storage.baseKey=sample ``` +Other common reasons that hand-off fails are as follows: + +1) Historical nodes are out of capacity and cannot download any more segments. You'll see exceptions in the coordinator logs if this occurs. +2) Segments are corrupt and cannot download. You'll see exceptions in your historical nodes if this occurs. +3) Deep storage is improperly configured. Make sure that your segment actually exists in deep storage and that the coordinator logs have no errors. + ## How do I get HDFS to work? Make sure to include the `druid-hdfs-storage` module as one of your extensions and set `druid.storage.type=hdfs`. @@ -50,6 +56,9 @@ To do this use the IngestSegmentFirehose and run an indexer task. The IngestSegm Typically the above will be run as a batch job to say everyday feed in a chunk of data and aggregate it. +## Real-time ingestion seems to be stuck + +There are a few ways this can occur. Druid will throttle ingestion to prevent out of memory problems if the intermediate persists are taking too long or if hand-off is taking too long. If your node logs indicate certain columns are taking a very long time to build (for example, if your segment granularity is hourly, but creating a single column takes 30 minutes), you should re-evaluate your configuration or scale up your real-time ingestion. ## More information diff --git a/docs/content/Rule-Configuration.md b/docs/content/Rule-Configuration.md index bf8b8a9792d..c25c9e62b70 100644 --- a/docs/content/Rule-Configuration.md +++ b/docs/content/Rule-Configuration.md @@ -2,12 +2,34 @@ layout: doc_page --- # Configuring Rules for Coordinator Nodes + +Rules indicate how segments should be assigned to different historical node tiers and how many replicas of a segment should exist in each tier. Rules may also indicate when segments should be dropped entirely from the cluster. The coordinator loads a set of rules from the metadata storage. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The coordinator will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule. + Note: It is recommended that the coordinator console is used to configure rules. However, the coordinator node does have HTTP endpoints to programmatically configure rules. + Load Rules ---------- -Load rules indicate how many replicants of a segment should exist in a server tier. +Load rules indicate how many replicas of a segment should exist in a server tier. + +### Forever Load Rule + +Forever load rules are of the form: + +```json +{ + "type" : "loadForever", + "tieredReplicants": { + "hot": 1, + "_default_tier" : 1 + } +} +``` + +* `type` - this should always be "loadByInterval" +* `tieredReplicants` - A JSON Object where the keys are the tier names and values are the number of replicas for that tier. + ### Interval Load Rule @@ -16,14 +38,17 @@ Interval load rules are of the form: ```json { "type" : "loadByInterval", - "interval" : "2012-01-01/2013-01-01", - "tier" : "hot" + "interval": "2012-01-01/2013-01-01", + "tieredReplicants": { + "hot": 1, + "_default_tier" : 1 + } } ``` * `type` - this should always be "loadByInterval" * `interval` - A JSON Object representing ISO-8601 Intervals -* `tier` - the configured historical node tier +* `tieredReplicants` - A JSON Object where the keys are the tier names and values are the number of replicas for that tier. ### Period Load Rule @@ -33,13 +58,16 @@ Period load rules are of the form: { "type" : "loadByPeriod", "period" : "P1M", - "tier" : "hot" + "tieredReplicants": { + "hot": 1, + "_default_tier" : 1 + } } ``` * `type` - this should always be "loadByPeriod" * `period` - A JSON Object representing ISO-8601 Periods -* `tier` - the configured historical node tier +* `tieredReplicants` - A JSON Object where the keys are the tier names and values are the number of replicas for that tier. The interval of a segment will be compared against the specified period. The rule matches if the period overlaps the interval. @@ -48,6 +76,21 @@ Drop Rules Drop rules indicate when segments should be dropped from the cluster. +### Forever Drop Rule + +Forever drop rules are of the form: + +```json +{ + "type" : "dropForever" +} +``` + +* `type` - this should always be "dropByPeriod" + +All segments that match this rule are dropped from the cluster. + + ### Interval Drop Rule Interval drop rules are of the form: From ee392b6064c7ec04783f348de068cf513249d65b Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 21 Oct 2014 16:26:17 -0700 Subject: [PATCH 55/69] more updates to docs --- docs/content/Ingestion-FAQ.md | 9 ++++++++- .../{Best-Practices.md => Recommendations.md} | 15 +++++++++++---- docs/content/index.md | 13 +------------ docs/content/toc.textile | 2 +- 4 files changed, 21 insertions(+), 18 deletions(-) rename docs/content/{Best-Practices.md => Recommendations.md} (53%) diff --git a/docs/content/Ingestion-FAQ.md b/docs/content/Ingestion-FAQ.md index ecf6b2ccdac..972e62a6a23 100644 --- a/docs/content/Ingestion-FAQ.md +++ b/docs/content/Ingestion-FAQ.md @@ -1,6 +1,11 @@ --- layout: doc_page --- + +## What types of data does Druid support? + +Druid can ingest JSON, CSV, TSV and other delimited data out of the box. Druid supports single dimension values, or multiple dimension values (an array of strings). Druid supports long and float numeric columns. + ## Where do my Druid segments end up after ingestion? Depending on what `druid.storage.type` is set to, Druid will upload segments to some [Deep Storage](Deep-Storage.html). Local disk is used as the default deep storage. @@ -24,7 +29,9 @@ druid.storage.baseKey=sample Other common reasons that hand-off fails are as follows: 1) Historical nodes are out of capacity and cannot download any more segments. You'll see exceptions in the coordinator logs if this occurs. + 2) Segments are corrupt and cannot download. You'll see exceptions in your historical nodes if this occurs. + 3) Deep storage is improperly configured. Make sure that your segment actually exists in deep storage and that the coordinator logs have no errors. ## How do I get HDFS to work? @@ -41,7 +48,7 @@ You can check the coordinator console located at `:/cluste ## My queries are returning empty results -You can check `:/druid/v2/datasources/?interval=0/3000` for the dimensions and metrics that have been created for your datasource. Make sure that the name of the aggregators you use in your query match one of these metrics. Also make sure that the query interval you specify match a valid time range where data exists. Note: the broker endpoint will only return valid results on historical segments. +You can check `:/druid/v2/datasources/?interval=0/3000` for the dimensions and metrics that have been created for your datasource. Make sure that the name of the aggregators you use in your query match one of these metrics. Also make sure that the query interval you specify match a valid time range where data exists. Note: the broker endpoint will only return valid results on historical segments and not segments served by real-time nodes. ## How can I Reindex existing data in Druid with schema changes? diff --git a/docs/content/Best-Practices.md b/docs/content/Recommendations.md similarity index 53% rename from docs/content/Best-Practices.md rename to docs/content/Recommendations.md index 9688a52f966..bf764ffe6c2 100644 --- a/docs/content/Best-Practices.md +++ b/docs/content/Recommendations.md @@ -2,8 +2,8 @@ layout: doc_page --- -Best Practices -============== +Recommendations +=============== # Use UTC Timezone @@ -17,12 +17,19 @@ Druid is not perfect in how it handles mix-cased dimension and metric names. Thi SSDs are highly recommended for historical and real-time nodes if you are not running a cluster that is entirely in memory. SSDs can greatly mitigate the time required to page data in and out of memory. -# Provide Columns Names in Lexicographic Order for Best Results +# Provide Columns Names in Lexicographic Order -Although Druid supports schemaless ingestion of dimensions, because of https://github.com/metamx/druid/issues/658, you may sometimes get bigger segments than necessary. To ensure segments are as compact as possible, providing dimension names in lexicographic order is recommended. This may require some ETL processing on your data however. +Although Druid supports schema-less ingestion of dimensions, because of [https://github.com/metamx/druid/issues/658](https://github.com/metamx/druid/issues/658), you may sometimes get bigger segments than necessary. To ensure segments are as compact as possible, providing dimension names in lexicographic order is recommended. + + +# Use Timeseries and TopN Queries Instead of GroupBy Where Possible + +Timeseries and TopN queries are much more optimized and significantly faster than groupBy queries for their designed use cases. Issuing multiple topN or timeseries queries from your application can potentially be more efficient than a single groupBy query. # Read FAQs You should read common problems people have here: + 1) [Ingestion-FAQ](Ingestion-FAQ.html) + 2) [Performance-FAQ](Performance-FAQ.html) \ No newline at end of file diff --git a/docs/content/index.md b/docs/content/index.md index 529a2325436..3c236cc81be 100644 --- a/docs/content/index.md +++ b/docs/content/index.md @@ -37,17 +37,6 @@ When Druid? * You want to do your analysis on data as it’s happening (in real-time) * You need a data store that is always available, 24x7x365, and years into the future. - -Not Druid? ----------- - -* The amount of data you have can easily be handled by MySQL -* You're querying for individual entries or doing lookups (not analytics) -* Batch ingestion is good enough -* Canned queries are good enough -* Downtime is no big deal - - Druid vs… ---------- @@ -60,7 +49,7 @@ Druid vs… About This Page ---------- -The data store world is vast, confusing and constantly in flux. This page is meant to help potential evaluators decide whether Druid is a good fit for the problem one needs to solve. If anything about it is incorrect please provide that feedback on the mailing list or via some other means so we can fix it. +The data infrastructure world is vast, confusing and constantly in flux. This page is meant to help potential evaluators decide whether Druid is a good fit for the problem one needs to solve. If anything about it is incorrect please provide that feedback on the mailing list or via some other means so we can fix it. diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 21f867bca36..29c08ab4d0e 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -19,7 +19,7 @@ h2. Booting a Druid Cluster * "Production Cluster Configuration":Production-Cluster-Configuration.html * "Production Hadoop Configuration":Hadoop-Configuration.html * "Rolling Cluster Updates":Rolling-Updates.html -* "Best Practices":Best-Practices.html +* "Recommendations":Recommendations.html h2. Configuration * "Common Configuration":Configuration.html From 71831e49d968193892b43a2f096f86bba9326268 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 8 May 2014 19:09:14 -0700 Subject: [PATCH 56/69] rudimentary filtered aggregator --- .../io/druid/jackson/AggregatorsModule.java | 4 +- .../query/aggregation/FilteredAggregator.java | 91 ++++++++++ .../FilteredAggregatorFactory.java | 160 ++++++++++++++++++ .../aggregation/FilteredBufferAggregator.java | 84 +++++++++ 4 files changed, 338 insertions(+), 1 deletion(-) create mode 100644 processing/src/main/java/io/druid/query/aggregation/FilteredAggregator.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java diff --git a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java index 908a17cb058..313ed85fc4c 100644 --- a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java +++ b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java @@ -26,6 +26,7 @@ import com.google.common.hash.Hashing; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.FilteredAggregatorFactory; import io.druid.query.aggregation.HistogramAggregatorFactory; import io.druid.query.aggregation.JavaScriptAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; @@ -68,7 +69,8 @@ public class AggregatorsModule extends SimpleModule @JsonSubTypes.Type(name = "javascript", value = JavaScriptAggregatorFactory.class), @JsonSubTypes.Type(name = "histogram", value = HistogramAggregatorFactory.class), @JsonSubTypes.Type(name = "hyperUnique", value = HyperUniquesAggregatorFactory.class), - @JsonSubTypes.Type(name = "cardinality", value = CardinalityAggregatorFactory.class) + @JsonSubTypes.Type(name = "cardinality", value = CardinalityAggregatorFactory.class), + @JsonSubTypes.Type(name = "filtered", value = FilteredAggregatorFactory.class) }) public static interface AggregatorFactoryMixin { diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregator.java b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregator.java new file mode 100644 index 00000000000..9a124c5e7a1 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregator.java @@ -0,0 +1,91 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.aggregation; + +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; +import io.druid.segment.DimensionSelector; + +import javax.annotation.Nullable; + +public class FilteredAggregator implements Aggregator +{ + private final String name; + private final DimensionSelector dimSelector; + private final Aggregator delegate; + private final Predicate predicate; + + public FilteredAggregator(String name, DimensionSelector dimSelector, Predicate predicate, Aggregator delegate) + { + this.name = name; + this.dimSelector = dimSelector; + this.delegate = delegate; + this.predicate = predicate; + } + + @Override + public void aggregate() + { + if ( + Iterables.any( + dimSelector.getRow(), new Predicate() + { + @Override + public boolean apply(@Nullable Integer input) + { + return predicate.apply(dimSelector.lookupName(input)); + } + } + ) + ) { + delegate.aggregate(); + } + } + + @Override + public void reset() + { + delegate.reset(); + } + + @Override + public Object get() + { + return delegate.get(); + } + + @Override + public float getFloat() + { + return delegate.getFloat(); + } + + @Override + public String getName() + { + return name; + } + + @Override + public void close() + { + delegate.close(); + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java new file mode 100644 index 00000000000..9d3e676ecb7 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java @@ -0,0 +1,160 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.aggregation; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; +import io.druid.query.filter.DimFilter; +import io.druid.query.filter.SelectorDimFilter; +import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.DimensionSelector; + +import java.nio.ByteBuffer; +import java.util.Comparator; +import java.util.List; + +public class FilteredAggregatorFactory implements AggregatorFactory +{ + private static final byte CACHE_TYPE_ID = 0x9; + + private final String name; + private final AggregatorFactory delegate; + private final DimFilter filter; + + public FilteredAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("aggregator") AggregatorFactory delegate, + @JsonProperty("filter") DimFilter filter + ) + { + Preconditions.checkNotNull(delegate); + Preconditions.checkNotNull(filter); + Preconditions.checkArgument(filter instanceof SelectorDimFilter, "Filtered Aggregator only supports "); + + this.name = name; + this.delegate = delegate; + this.filter = filter; + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + final Aggregator aggregator = delegate.factorize(metricFactory); + final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(((SelectorDimFilter)filter).getDimension()); + Predicate predicate = Predicates.equalTo(((SelectorDimFilter)filter).getValue()); + return new FilteredAggregator(name, dimSelector, predicate, aggregator); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + final BufferAggregator aggregator = delegate.factorizeBuffered(metricFactory); + final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(((SelectorDimFilter)filter).getDimension()); + Predicate predicate = Predicates.equalTo(((SelectorDimFilter)filter).getValue()); + return new FilteredBufferAggregator(dimSelector, predicate, aggregator); + } + + @Override + public Comparator getComparator() + { + return delegate.getComparator(); + } + + @Override + public Object combine(Object lhs, Object rhs) + { + return delegate.combine(lhs, rhs); + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return delegate.getCombiningFactory(); + } + + @Override + public Object deserialize(Object object) + { + return delegate.deserialize(object); + } + + @Override + public Object finalizeComputation(Object object) + { + return delegate.finalizeComputation(object); + } + + @JsonProperty + @Override + public String getName() + { + return name; + } + + @Override + public List requiredFields() + { + return delegate.requiredFields(); + } + + @Override + public byte[] getCacheKey() + { + byte[] filterCacheKey = filter.getCacheKey(); + byte[] aggregatorCacheKey = delegate.getCacheKey(); + return ByteBuffer.allocate(1 + filterCacheKey.length + aggregatorCacheKey.length) + .put(CACHE_TYPE_ID) + .put(filterCacheKey) + .put(aggregatorCacheKey) + .array(); + } + + @Override + public String getTypeName() + { + return delegate.getTypeName(); + } + + @Override + public int getMaxIntermediateSize() + { + return delegate.getMaxIntermediateSize(); + } + + @Override + public Object getAggregatorStartValue() + { + return delegate.getAggregatorStartValue(); + } + + @JsonProperty + public AggregatorFactory getAggregator() + { + return delegate; + } + + @JsonProperty + public DimFilter getFilter() + { + return filter; + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java new file mode 100644 index 00000000000..ce8795d49da --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java @@ -0,0 +1,84 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.aggregation; + +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; +import io.druid.segment.DimensionSelector; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; + +public class FilteredBufferAggregator implements BufferAggregator +{ + private final DimensionSelector dimSelector; + private final Predicate predicate; + private final BufferAggregator delegate; + + public FilteredBufferAggregator(DimensionSelector dimSelector, Predicate predicate, BufferAggregator delegate) + { + this.dimSelector = dimSelector; + this.predicate = predicate; + this.delegate = delegate; + } + + @Override + public void init(ByteBuffer buf, int position) + { + delegate.init(buf, position); + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + if ( + Iterables.any( + dimSelector.getRow(), new Predicate() + { + @Override + public boolean apply(@Nullable Integer input) + { + return predicate.apply(dimSelector.lookupName(input)); + } + } + ) + ) { + delegate.aggregate(buf, position); + } + } + + @Override + public Object get(ByteBuffer buf, int position) + { + return delegate.get(buf, position); + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + return delegate.getFloat(buf, position); + } + + @Override + public void close() + { + delegate.close(); + } +} From 10db94101ffabdf0f781064a8d2f60b2fc9e5969 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Tue, 21 Oct 2014 16:31:39 +0530 Subject: [PATCH 57/69] skip value lookup compare ids instead of values --- .../druid/query/aggregation/FilteredAggregator.java | 13 +++---------- .../aggregation/FilteredAggregatorFactory.java | 4 ++-- .../query/aggregation/FilteredBufferAggregator.java | 6 +++--- 3 files changed, 8 insertions(+), 15 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregator.java b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregator.java index 9a124c5e7a1..2134317d74f 100644 --- a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregator.java @@ -30,9 +30,9 @@ public class FilteredAggregator implements Aggregator private final String name; private final DimensionSelector dimSelector; private final Aggregator delegate; - private final Predicate predicate; + private final Predicate predicate; - public FilteredAggregator(String name, DimensionSelector dimSelector, Predicate predicate, Aggregator delegate) + public FilteredAggregator(String name, DimensionSelector dimSelector, Predicate predicate, Aggregator delegate) { this.name = name; this.dimSelector = dimSelector; @@ -45,14 +45,7 @@ public class FilteredAggregator implements Aggregator { if ( Iterables.any( - dimSelector.getRow(), new Predicate() - { - @Override - public boolean apply(@Nullable Integer input) - { - return predicate.apply(dimSelector.lookupName(input)); - } - } + dimSelector.getRow(), predicate ) ) { delegate.aggregate(); diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java index 9d3e676ecb7..817846e48c8 100644 --- a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java @@ -60,7 +60,7 @@ public class FilteredAggregatorFactory implements AggregatorFactory { final Aggregator aggregator = delegate.factorize(metricFactory); final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(((SelectorDimFilter)filter).getDimension()); - Predicate predicate = Predicates.equalTo(((SelectorDimFilter)filter).getValue()); + Predicate predicate = Predicates.equalTo(dimSelector.lookupId(((SelectorDimFilter)filter).getValue())); return new FilteredAggregator(name, dimSelector, predicate, aggregator); } @@ -69,7 +69,7 @@ public class FilteredAggregatorFactory implements AggregatorFactory { final BufferAggregator aggregator = delegate.factorizeBuffered(metricFactory); final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(((SelectorDimFilter)filter).getDimension()); - Predicate predicate = Predicates.equalTo(((SelectorDimFilter)filter).getValue()); + Predicate predicate = Predicates.equalTo(dimSelector.lookupId(((SelectorDimFilter)filter).getValue())); return new FilteredBufferAggregator(dimSelector, predicate, aggregator); } diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java index ce8795d49da..9444bee0bff 100644 --- a/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java @@ -29,10 +29,10 @@ import java.nio.ByteBuffer; public class FilteredBufferAggregator implements BufferAggregator { private final DimensionSelector dimSelector; - private final Predicate predicate; + private final Predicate predicate; private final BufferAggregator delegate; - public FilteredBufferAggregator(DimensionSelector dimSelector, Predicate predicate, BufferAggregator delegate) + public FilteredBufferAggregator(DimensionSelector dimSelector, Predicate predicate, BufferAggregator delegate) { this.dimSelector = dimSelector; this.predicate = predicate; @@ -55,7 +55,7 @@ public class FilteredBufferAggregator implements BufferAggregator @Override public boolean apply(@Nullable Integer input) { - return predicate.apply(dimSelector.lookupName(input)); + return predicate.apply(input); } } ) From 56912adccbe93448c39cb89763d3fb92c76b0714 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Tue, 21 Oct 2014 22:53:48 +0530 Subject: [PATCH 58/69] avoid typecasts --- .../query/aggregation/FilteredAggregator.java | 18 +++++----- .../FilteredAggregatorFactory.java | 34 +++++++++++++------ .../aggregation/FilteredBufferAggregator.java | 25 ++++++-------- .../druid/query/aggregation/IntPredicate.java | 29 ++++++++++++++++ 4 files changed, 73 insertions(+), 33 deletions(-) create mode 100644 processing/src/main/java/io/druid/query/aggregation/IntPredicate.java diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregator.java b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregator.java index 2134317d74f..2b78ed2cd22 100644 --- a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregator.java @@ -22,6 +22,7 @@ package io.druid.query.aggregation; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import io.druid.segment.DimensionSelector; +import io.druid.segment.data.IndexedInts; import javax.annotation.Nullable; @@ -30,9 +31,9 @@ public class FilteredAggregator implements Aggregator private final String name; private final DimensionSelector dimSelector; private final Aggregator delegate; - private final Predicate predicate; + private final IntPredicate predicate; - public FilteredAggregator(String name, DimensionSelector dimSelector, Predicate predicate, Aggregator delegate) + public FilteredAggregator(String name, DimensionSelector dimSelector, IntPredicate predicate, Aggregator delegate) { this.name = name; this.dimSelector = dimSelector; @@ -43,12 +44,13 @@ public class FilteredAggregator implements Aggregator @Override public void aggregate() { - if ( - Iterables.any( - dimSelector.getRow(), predicate - ) - ) { - delegate.aggregate(); + final IndexedInts row = dimSelector.getRow(); + final int size = row.size(); + for (int i = 0; i < size; ++i) { + if (predicate.apply(row.get(i))) { + delegate.aggregate(); + break; + } } } diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java index 817846e48c8..bb55f8257f9 100644 --- a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java @@ -21,8 +21,6 @@ package io.druid.query.aggregation; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.base.Predicates; import io.druid.query.filter.DimFilter; import io.druid.query.filter.SelectorDimFilter; import io.druid.segment.ColumnSelectorFactory; @@ -59,8 +57,16 @@ public class FilteredAggregatorFactory implements AggregatorFactory public Aggregator factorize(ColumnSelectorFactory metricFactory) { final Aggregator aggregator = delegate.factorize(metricFactory); - final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(((SelectorDimFilter)filter).getDimension()); - Predicate predicate = Predicates.equalTo(dimSelector.lookupId(((SelectorDimFilter)filter).getValue())); + final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(((SelectorDimFilter) filter).getDimension()); + final int lookupId = dimSelector.lookupId(((SelectorDimFilter) filter).getValue()); + final IntPredicate predicate = new IntPredicate() + { + @Override + public boolean apply(int value) + { + return lookupId == value; + } + }; return new FilteredAggregator(name, dimSelector, predicate, aggregator); } @@ -68,8 +74,16 @@ public class FilteredAggregatorFactory implements AggregatorFactory public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { final BufferAggregator aggregator = delegate.factorizeBuffered(metricFactory); - final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(((SelectorDimFilter)filter).getDimension()); - Predicate predicate = Predicates.equalTo(dimSelector.lookupId(((SelectorDimFilter)filter).getValue())); + final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(((SelectorDimFilter) filter).getDimension()); + final int lookupId = dimSelector.lookupId(((SelectorDimFilter) filter).getValue()); + final IntPredicate predicate = new IntPredicate() + { + @Override + public boolean apply(int value) + { + return lookupId == value; + } + }; return new FilteredBufferAggregator(dimSelector, predicate, aggregator); } @@ -122,10 +136,10 @@ public class FilteredAggregatorFactory implements AggregatorFactory byte[] filterCacheKey = filter.getCacheKey(); byte[] aggregatorCacheKey = delegate.getCacheKey(); return ByteBuffer.allocate(1 + filterCacheKey.length + aggregatorCacheKey.length) - .put(CACHE_TYPE_ID) - .put(filterCacheKey) - .put(aggregatorCacheKey) - .array(); + .put(CACHE_TYPE_ID) + .put(filterCacheKey) + .put(aggregatorCacheKey) + .array(); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java index 9444bee0bff..4d1eeda99ba 100644 --- a/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java @@ -22,6 +22,7 @@ package io.druid.query.aggregation; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import io.druid.segment.DimensionSelector; +import io.druid.segment.data.IndexedInts; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -29,10 +30,10 @@ import java.nio.ByteBuffer; public class FilteredBufferAggregator implements BufferAggregator { private final DimensionSelector dimSelector; - private final Predicate predicate; + private final IntPredicate predicate; private final BufferAggregator delegate; - public FilteredBufferAggregator(DimensionSelector dimSelector, Predicate predicate, BufferAggregator delegate) + public FilteredBufferAggregator(DimensionSelector dimSelector, IntPredicate predicate, BufferAggregator delegate) { this.dimSelector = dimSelector; this.predicate = predicate; @@ -48,19 +49,13 @@ public class FilteredBufferAggregator implements BufferAggregator @Override public void aggregate(ByteBuffer buf, int position) { - if ( - Iterables.any( - dimSelector.getRow(), new Predicate() - { - @Override - public boolean apply(@Nullable Integer input) - { - return predicate.apply(input); - } - } - ) - ) { - delegate.aggregate(buf, position); + final IndexedInts row = dimSelector.getRow(); + final int size = row.size(); + for (int i = 0; i < size; ++i) { + if (predicate.apply(row.get(i))) { + delegate.aggregate(buf, position); + break; + } } } diff --git a/processing/src/main/java/io/druid/query/aggregation/IntPredicate.java b/processing/src/main/java/io/druid/query/aggregation/IntPredicate.java new file mode 100644 index 00000000000..fd013004e91 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/IntPredicate.java @@ -0,0 +1,29 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.aggregation; + +/** + * can be replaced with http://docs.oracle.com/javase/8/docs/api/java/util/function/IntPredicate.html + * when druid moves to java 8. + */ +public interface IntPredicate +{ + boolean apply(int value); +} From 23a6ab19b0e3fb7b1dfa2221b2281e0986b0a09f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 21 Oct 2014 17:00:39 -0700 Subject: [PATCH 59/69] cleanup and add missing methods --- .../FilteredAggregatorFactory.java | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java index bb55f8257f9..f19c8fd43be 100644 --- a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java @@ -36,7 +36,7 @@ public class FilteredAggregatorFactory implements AggregatorFactory private final String name; private final AggregatorFactory delegate; - private final DimFilter filter; + private final SelectorDimFilter filter; public FilteredAggregatorFactory( @JsonProperty("name") String name, @@ -46,19 +46,19 @@ public class FilteredAggregatorFactory implements AggregatorFactory { Preconditions.checkNotNull(delegate); Preconditions.checkNotNull(filter); - Preconditions.checkArgument(filter instanceof SelectorDimFilter, "Filtered Aggregator only supports "); + Preconditions.checkArgument(filter instanceof SelectorDimFilter, "FilteredAggregator currently only supports filters of type selector"); this.name = name; this.delegate = delegate; - this.filter = filter; + this.filter = (SelectorDimFilter)filter; } @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { final Aggregator aggregator = delegate.factorize(metricFactory); - final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(((SelectorDimFilter) filter).getDimension()); - final int lookupId = dimSelector.lookupId(((SelectorDimFilter) filter).getValue()); + final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(filter.getDimension()); + final int lookupId = dimSelector.lookupId(filter.getValue()); final IntPredicate predicate = new IntPredicate() { @Override @@ -74,8 +74,8 @@ public class FilteredAggregatorFactory implements AggregatorFactory public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { final BufferAggregator aggregator = delegate.factorizeBuffered(metricFactory); - final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(((SelectorDimFilter) filter).getDimension()); - final int lookupId = dimSelector.lookupId(((SelectorDimFilter) filter).getValue()); + final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(filter.getDimension()); + final int lookupId = dimSelector.lookupId(filter.getValue()); final IntPredicate predicate = new IntPredicate() { @Override @@ -171,4 +171,10 @@ public class FilteredAggregatorFactory implements AggregatorFactory { return filter; } + + @Override + public List getRequiredColumns() + { + return delegate.getRequiredColumns(); + } } From 34f2c74331991a1cc1bf3083ced511632fdf141d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 21 Oct 2014 17:24:44 -0700 Subject: [PATCH 60/69] add docs --- docs/content/Aggregations.md | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/docs/content/Aggregations.md b/docs/content/Aggregations.md index abd4780b025..1ff60721c39 100644 --- a/docs/content/Aggregations.md +++ b/docs/content/Aggregations.md @@ -160,3 +160,28 @@ Uses [HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf) to ```json { "type" : "hyperUnique", "name" : , "fieldName" : } ``` + +## Miscellaneous Aggregations + +### Filtered Aggregator + +A filtered aggregator wraps any given aggregator, but only aggregates the values for which the given dimension filter matches. + +This makes it possible to compute the results of a filtered and an unfiltered aggregation simultaneously, without having to issue multiple queries, and use both results as part of post-aggregations. + +*Limitations:* The filtered aggregator currently only supports selector filters, i.e. matching a dimension against a single value. + +*Note:* If only the filtered results are required, consider putting the filter on the query itself, which will be much faster since it does not require scanning all the data. + +```json +{ + "type" : "filtered", + "name" : "aggMatching", + "filter" : { + "type" : "selector", + "dimension" : , + "value" : + } + "aggregator" : +} +``` From e24a6d222f26db8aee64aba6618d36c25ee08ee4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 21 Oct 2014 17:46:24 -0700 Subject: [PATCH 61/69] we even have tests now --- .../aggregation/FilteredAggregatorTest.java | 155 ++++++++++++++++++ .../aggregation/TestFloatColumnSelector.java | 5 + 2 files changed, 160 insertions(+) create mode 100644 processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java diff --git a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java new file mode 100644 index 00000000000..ef0388a8264 --- /dev/null +++ b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java @@ -0,0 +1,155 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.aggregation; + +import io.druid.query.filter.SelectorDimFilter; +import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.DimensionSelector; +import io.druid.segment.FloatColumnSelector; +import io.druid.segment.ObjectColumnSelector; +import io.druid.segment.TimestampColumnSelector; +import io.druid.segment.data.ArrayBasedIndexedInts; +import io.druid.segment.data.IndexedInts; +import org.junit.Assert; +import org.junit.Test; + +public class FilteredAggregatorTest +{ + private void aggregate(TestFloatColumnSelector selector, FilteredAggregator agg) + { + agg.aggregate(); + selector.increment(); + } + + @Test + public void testAggregate() + { + final float[] values = {0.15f, 0.27f}; + final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); + + FilteredAggregatorFactory factory = new FilteredAggregatorFactory( + "test", + new DoubleSumAggregatorFactory("billy", "value"), + new SelectorDimFilter("dim", "a") + ); + + FilteredAggregator agg = (FilteredAggregator) factory.factorize( + new ColumnSelectorFactory() + { + @Override + public TimestampColumnSelector makeTimestampColumnSelector() + { + throw new UnsupportedOperationException(); + } + + @Override + public DimensionSelector makeDimensionSelector(String dimensionName) + { + if (dimensionName.equals("dim")) { + return new DimensionSelector() + { + @Override + public IndexedInts getRow() + { + if (selector.getIndex() % 3 == 2) { + return new ArrayBasedIndexedInts(new int[]{1}); + } else { + return new ArrayBasedIndexedInts(new int[]{0}); + } + } + + @Override + public int getValueCardinality() + { + return 2; + } + + @Override + public String lookupName(int id) + { + switch (id) { + case 0: + return "a"; + case 1: + return "b"; + default: + throw new IllegalArgumentException(); + } + } + + @Override + public int lookupId(String name) + { + switch (name) { + case "a": + return 0; + case "b": + return 1; + default: + throw new IllegalArgumentException(); + } + } + }; + } else { + throw new UnsupportedOperationException(); + } + } + + @Override + public FloatColumnSelector makeFloatColumnSelector(String columnName) + { + if (columnName.equals("value")) { + return selector; + } else { + throw new UnsupportedOperationException(); + } + } + + @Override + public ObjectColumnSelector makeObjectColumnSelector(String columnName) + { + throw new UnsupportedOperationException(); + } + } + ); + + Assert.assertEquals("test", agg.getName()); + + double expectedFirst = new Float(values[0]).doubleValue(); + double expectedSecond = new Float(values[1]).doubleValue() + expectedFirst; + double expectedThird = expectedSecond; + + Assert.assertEquals(0.0d, agg.get()); + Assert.assertEquals(0.0d, agg.get()); + Assert.assertEquals(0.0d, agg.get()); + aggregate(selector, agg); + Assert.assertEquals(expectedFirst, agg.get()); + Assert.assertEquals(expectedFirst, agg.get()); + Assert.assertEquals(expectedFirst, agg.get()); + aggregate(selector, agg); + Assert.assertEquals(expectedSecond, agg.get()); + Assert.assertEquals(expectedSecond, agg.get()); + Assert.assertEquals(expectedSecond, agg.get()); + aggregate(selector, agg); + Assert.assertEquals(expectedThird, agg.get()); + Assert.assertEquals(expectedThird, agg.get()); + Assert.assertEquals(expectedThird, agg.get()); + } +} diff --git a/processing/src/test/java/io/druid/query/aggregation/TestFloatColumnSelector.java b/processing/src/test/java/io/druid/query/aggregation/TestFloatColumnSelector.java index 8aea1f41503..874ffb978a4 100644 --- a/processing/src/test/java/io/druid/query/aggregation/TestFloatColumnSelector.java +++ b/processing/src/test/java/io/druid/query/aggregation/TestFloatColumnSelector.java @@ -44,4 +44,9 @@ public class TestFloatColumnSelector implements FloatColumnSelector { ++index; } + + public int getIndex() + { + return index; + } } From 6cb6ec39cfc22c4c70849954a6d330bcef492a47 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 22 Oct 2014 10:25:24 +0530 Subject: [PATCH 62/69] add support for not filter and add test --- docs/content/Aggregations.md | 2 +- .../FilteredAggregatorFactory.java | 84 +++++-- .../aggregation/FilteredAggregatorTest.java | 213 ++++++++++-------- 3 files changed, 181 insertions(+), 118 deletions(-) diff --git a/docs/content/Aggregations.md b/docs/content/Aggregations.md index 1ff60721c39..a73255c2730 100644 --- a/docs/content/Aggregations.md +++ b/docs/content/Aggregations.md @@ -169,7 +169,7 @@ A filtered aggregator wraps any given aggregator, but only aggregates the values This makes it possible to compute the results of a filtered and an unfiltered aggregation simultaneously, without having to issue multiple queries, and use both results as part of post-aggregations. -*Limitations:* The filtered aggregator currently only supports selector filters, i.e. matching a dimension against a single value. +*Limitations:* The filtered aggregator currently only supports selector and not filters, i.e. matching a dimension against a single value. *Note:* If only the filtered results are required, consider putting the filter on the query itself, which will be much faster since it does not require scanning all the data. diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java index f19c8fd43be..367c49ddbb5 100644 --- a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java @@ -21,7 +21,9 @@ package io.druid.query.aggregation; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import com.metamx.common.Pair; import io.druid.query.filter.DimFilter; +import io.druid.query.filter.NotDimFilter; import io.druid.query.filter.SelectorDimFilter; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionSelector; @@ -36,7 +38,7 @@ public class FilteredAggregatorFactory implements AggregatorFactory private final String name; private final AggregatorFactory delegate; - private final SelectorDimFilter filter; + private final DimFilter filter; public FilteredAggregatorFactory( @JsonProperty("name") String name, @@ -46,45 +48,36 @@ public class FilteredAggregatorFactory implements AggregatorFactory { Preconditions.checkNotNull(delegate); Preconditions.checkNotNull(filter); - Preconditions.checkArgument(filter instanceof SelectorDimFilter, "FilteredAggregator currently only supports filters of type selector"); + Preconditions.checkArgument( + filter instanceof SelectorDimFilter || filter instanceof NotDimFilter, + "FilteredAggregator currently only supports filters of type selector and not" + ); this.name = name; this.delegate = delegate; - this.filter = (SelectorDimFilter)filter; + this.filter = filter; } @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { final Aggregator aggregator = delegate.factorize(metricFactory); - final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(filter.getDimension()); - final int lookupId = dimSelector.lookupId(filter.getValue()); - final IntPredicate predicate = new IntPredicate() - { - @Override - public boolean apply(int value) - { - return lookupId == value; - } - }; - return new FilteredAggregator(name, dimSelector, predicate, aggregator); + final Pair selectorPredicatePair = makeFilterPredicate( + filter, + metricFactory + ); + return new FilteredAggregator(name, selectorPredicatePair.lhs, selectorPredicatePair.rhs, aggregator); } @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { final BufferAggregator aggregator = delegate.factorizeBuffered(metricFactory); - final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(filter.getDimension()); - final int lookupId = dimSelector.lookupId(filter.getValue()); - final IntPredicate predicate = new IntPredicate() - { - @Override - public boolean apply(int value) - { - return lookupId == value; - } - }; - return new FilteredBufferAggregator(dimSelector, predicate, aggregator); + final Pair selectorPredicatePair = makeFilterPredicate( + filter, + metricFactory + ); + return new FilteredBufferAggregator(selectorPredicatePair.lhs, selectorPredicatePair.rhs, aggregator); } @Override @@ -177,4 +170,45 @@ public class FilteredAggregatorFactory implements AggregatorFactory { return delegate.getRequiredColumns(); } + + private static Pair makeFilterPredicate( + DimFilter dimFilter, + ColumnSelectorFactory metricFactory + ) + { + if (dimFilter instanceof SelectorDimFilter) { + final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(((SelectorDimFilter) dimFilter).getDimension()); + final int lookupId = dimSelector.lookupId(((SelectorDimFilter) dimFilter).getValue()); + return Pair.of( + dimSelector, new IntPredicate() + { + @Override + public boolean apply(int value) + { + return lookupId == value; + } + } + ); + } else if (dimFilter instanceof NotDimFilter) { + final Pair selectorPredicatePair = makeFilterPredicate( + ((NotDimFilter) dimFilter).getField(), + metricFactory + ); + return Pair.of( + selectorPredicatePair.lhs, new IntPredicate() + { + @Override + public boolean apply(int value) + { + return !selectorPredicatePair.rhs.apply(value); + } + } + ); + } else { + throw new UnsupportedOperationException( + "FilteredAggregator does not support DimFilter of type " + + dimFilter.getClass() + ); + } + } } diff --git a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java index ef0388a8264..b50ebb810f6 100644 --- a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java @@ -19,6 +19,7 @@ package io.druid.query.aggregation; +import io.druid.query.filter.NotDimFilter; import io.druid.query.filter.SelectorDimFilter; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionSelector; @@ -51,83 +52,7 @@ public class FilteredAggregatorTest ); FilteredAggregator agg = (FilteredAggregator) factory.factorize( - new ColumnSelectorFactory() - { - @Override - public TimestampColumnSelector makeTimestampColumnSelector() - { - throw new UnsupportedOperationException(); - } - - @Override - public DimensionSelector makeDimensionSelector(String dimensionName) - { - if (dimensionName.equals("dim")) { - return new DimensionSelector() - { - @Override - public IndexedInts getRow() - { - if (selector.getIndex() % 3 == 2) { - return new ArrayBasedIndexedInts(new int[]{1}); - } else { - return new ArrayBasedIndexedInts(new int[]{0}); - } - } - - @Override - public int getValueCardinality() - { - return 2; - } - - @Override - public String lookupName(int id) - { - switch (id) { - case 0: - return "a"; - case 1: - return "b"; - default: - throw new IllegalArgumentException(); - } - } - - @Override - public int lookupId(String name) - { - switch (name) { - case "a": - return 0; - case "b": - return 1; - default: - throw new IllegalArgumentException(); - } - } - }; - } else { - throw new UnsupportedOperationException(); - } - } - - @Override - public FloatColumnSelector makeFloatColumnSelector(String columnName) - { - if (columnName.equals("value")) { - return selector; - } else { - throw new UnsupportedOperationException(); - } - } - - @Override - public ObjectColumnSelector makeObjectColumnSelector(String columnName) - { - throw new UnsupportedOperationException(); - } - } + makeColumnSelector(selector) ); Assert.assertEquals("test", agg.getName()); @@ -136,20 +61,124 @@ public class FilteredAggregatorTest double expectedSecond = new Float(values[1]).doubleValue() + expectedFirst; double expectedThird = expectedSecond; - Assert.assertEquals(0.0d, agg.get()); - Assert.assertEquals(0.0d, agg.get()); - Assert.assertEquals(0.0d, agg.get()); - aggregate(selector, agg); - Assert.assertEquals(expectedFirst, agg.get()); - Assert.assertEquals(expectedFirst, agg.get()); - Assert.assertEquals(expectedFirst, agg.get()); - aggregate(selector, agg); - Assert.assertEquals(expectedSecond, agg.get()); - Assert.assertEquals(expectedSecond, agg.get()); - Assert.assertEquals(expectedSecond, agg.get()); - aggregate(selector, agg); - Assert.assertEquals(expectedThird, agg.get()); - Assert.assertEquals(expectedThird, agg.get()); - Assert.assertEquals(expectedThird, agg.get()); + assertValues(agg, selector, expectedFirst, expectedSecond, expectedThird); } + + private ColumnSelectorFactory makeColumnSelector(final TestFloatColumnSelector selector){ + + return new ColumnSelectorFactory() + { + @Override + public TimestampColumnSelector makeTimestampColumnSelector() + { + throw new UnsupportedOperationException(); + } + + @Override + public DimensionSelector makeDimensionSelector(String dimensionName) + { + if (dimensionName.equals("dim")) { + return new DimensionSelector() + { + @Override + public IndexedInts getRow() + { + if (selector.getIndex() % 3 == 2) { + return new ArrayBasedIndexedInts(new int[]{1}); + } else { + return new ArrayBasedIndexedInts(new int[]{0}); + } + } + + @Override + public int getValueCardinality() + { + return 2; + } + + @Override + public String lookupName(int id) + { + switch (id) { + case 0: + return "a"; + case 1: + return "b"; + default: + throw new IllegalArgumentException(); + } + } + + @Override + public int lookupId(String name) + { + switch (name) { + case "a": + return 0; + case "b": + return 1; + default: + throw new IllegalArgumentException(); + } + } + }; + } else { + throw new UnsupportedOperationException(); + } + } + + @Override + public FloatColumnSelector makeFloatColumnSelector(String columnName) + { + if (columnName.equals("value")) { + return selector; + } else { + throw new UnsupportedOperationException(); + } + } + + @Override + public ObjectColumnSelector makeObjectColumnSelector(String columnName) + { + throw new UnsupportedOperationException(); + } + }; + } + + private void assertValues(FilteredAggregator agg,TestFloatColumnSelector selector, double... expectedVals){ + Assert.assertEquals(0.0d, agg.get()); + Assert.assertEquals(0.0d, agg.get()); + Assert.assertEquals(0.0d, agg.get()); + for(double expectedVal : expectedVals){ + aggregate(selector, agg); + Assert.assertEquals(expectedVal, agg.get()); + Assert.assertEquals(expectedVal, agg.get()); + Assert.assertEquals(expectedVal, agg.get()); + } + } + + @Test + public void testAggregateWithNotFilter() + { + final float[] values = {0.15f, 0.27f}; + final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); + + FilteredAggregatorFactory factory = new FilteredAggregatorFactory( + "test", + new DoubleSumAggregatorFactory("billy", "value"), + new NotDimFilter(new SelectorDimFilter("dim", "b")) + ); + + FilteredAggregator agg = (FilteredAggregator) factory.factorize( + makeColumnSelector(selector) + ); + + Assert.assertEquals("test", agg.getName()); + + double expectedFirst = new Float(values[0]).doubleValue(); + double expectedSecond = new Float(values[1]).doubleValue() + expectedFirst; + double expectedThird = expectedSecond; + assertValues(agg, selector, expectedFirst, expectedSecond, expectedThird); + } + } From 0785baf54ce913b0f711affb318508fdafdc7303 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 22 Oct 2014 21:28:31 +0530 Subject: [PATCH 63/69] review comments --- docs/content/Aggregations.md | 2 +- .../FilteredAggregatorFactory.java | 37 ++++++++++++++----- 2 files changed, 28 insertions(+), 11 deletions(-) diff --git a/docs/content/Aggregations.md b/docs/content/Aggregations.md index a73255c2730..c8c79c6acf0 100644 --- a/docs/content/Aggregations.md +++ b/docs/content/Aggregations.md @@ -169,7 +169,7 @@ A filtered aggregator wraps any given aggregator, but only aggregates the values This makes it possible to compute the results of a filtered and an unfiltered aggregation simultaneously, without having to issue multiple queries, and use both results as part of post-aggregations. -*Limitations:* The filtered aggregator currently only supports selector and not filters, i.e. matching a dimension against a single value. +*Limitations:* The filtered aggregator currently only supports selector and not filter with a single selector, i.e. matching a dimension against a single value. *Note:* If only the filtered results are required, consider putting the filter on the query itself, which will be much faster since it does not require scanning all the data. diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java index 367c49ddbb5..45171a9aa60 100644 --- a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java @@ -52,6 +52,12 @@ public class FilteredAggregatorFactory implements AggregatorFactory filter instanceof SelectorDimFilter || filter instanceof NotDimFilter, "FilteredAggregator currently only supports filters of type selector and not" ); + if (filter instanceof NotDimFilter) { + Preconditions.checkArgument( + ((NotDimFilter) filter).getField() instanceof SelectorDimFilter, + "FilteredAggregator currently only support not filter with selector filter" + ); + } this.name = name; this.delegate = delegate; @@ -172,13 +178,27 @@ public class FilteredAggregatorFactory implements AggregatorFactory } private static Pair makeFilterPredicate( - DimFilter dimFilter, - ColumnSelectorFactory metricFactory + final DimFilter dimFilter, + final ColumnSelectorFactory metricFactory ) { + final SelectorDimFilter selector; + if (dimFilter instanceof NotDimFilter) { + // we only support NotDimFilter with Selector filter + selector = (SelectorDimFilter) ((NotDimFilter) dimFilter).getField(); + } else if (dimFilter instanceof SelectorDimFilter) { + selector = (SelectorDimFilter) dimFilter; + } else { + throw new UnsupportedOperationException( + "FilteredAggregator does not support DimFilter of type " + + dimFilter.getClass() + ); + } + + final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(selector.getDimension()); + final int lookupId = dimSelector.lookupId(selector.getValue()); if (dimFilter instanceof SelectorDimFilter) { - final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(((SelectorDimFilter) dimFilter).getDimension()); - final int lookupId = dimSelector.lookupId(((SelectorDimFilter) dimFilter).getValue()); + return Pair.of( dimSelector, new IntPredicate() { @@ -190,17 +210,13 @@ public class FilteredAggregatorFactory implements AggregatorFactory } ); } else if (dimFilter instanceof NotDimFilter) { - final Pair selectorPredicatePair = makeFilterPredicate( - ((NotDimFilter) dimFilter).getField(), - metricFactory - ); return Pair.of( - selectorPredicatePair.lhs, new IntPredicate() + dimSelector, new IntPredicate() { @Override public boolean apply(int value) { - return !selectorPredicatePair.rhs.apply(value); + return lookupId != value; } } ); @@ -211,4 +227,5 @@ public class FilteredAggregatorFactory implements AggregatorFactory ); } } + } From 4ec1e6ef13060e9e520502f88026d4d0cad17842 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 22 Oct 2014 14:25:59 -0700 Subject: [PATCH 64/69] simplify code a bit --- .../FilteredAggregatorFactory.java | 63 +++++++------------ 1 file changed, 24 insertions(+), 39 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java index 45171a9aa60..cd5685a83f7 100644 --- a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java @@ -21,6 +21,7 @@ package io.druid.query.aggregation; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import com.metamx.common.ISE; import com.metamx.common.Pair; import io.druid.query.filter.DimFilter; import io.druid.query.filter.NotDimFilter; @@ -49,15 +50,10 @@ public class FilteredAggregatorFactory implements AggregatorFactory Preconditions.checkNotNull(delegate); Preconditions.checkNotNull(filter); Preconditions.checkArgument( - filter instanceof SelectorDimFilter || filter instanceof NotDimFilter, - "FilteredAggregator currently only supports filters of type selector and not" + filter instanceof SelectorDimFilter || + (filter instanceof NotDimFilter && ((NotDimFilter) filter).getField() instanceof SelectorDimFilter), + "FilteredAggregator currently only supports filters of type 'selector' and their negation" ); - if (filter instanceof NotDimFilter) { - Preconditions.checkArgument( - ((NotDimFilter) filter).getField() instanceof SelectorDimFilter, - "FilteredAggregator currently only support not filter with selector filter" - ); - } this.name = name; this.delegate = delegate; @@ -189,43 +185,32 @@ public class FilteredAggregatorFactory implements AggregatorFactory } else if (dimFilter instanceof SelectorDimFilter) { selector = (SelectorDimFilter) dimFilter; } else { - throw new UnsupportedOperationException( - "FilteredAggregator does not support DimFilter of type " - + dimFilter.getClass() - ); + throw new ISE("Unsupported DimFilter type [%d]", dimFilter.getClass()); } final DimensionSelector dimSelector = metricFactory.makeDimensionSelector(selector.getDimension()); final int lookupId = dimSelector.lookupId(selector.getValue()); - if (dimFilter instanceof SelectorDimFilter) { - - return Pair.of( - dimSelector, new IntPredicate() - { - @Override - public boolean apply(int value) - { - return lookupId == value; - } - } - ); - } else if (dimFilter instanceof NotDimFilter) { - return Pair.of( - dimSelector, new IntPredicate() - { - @Override - public boolean apply(int value) - { - return lookupId != value; - } - } - ); + final IntPredicate predicate; + if (dimFilter instanceof NotDimFilter) { + predicate = new IntPredicate() + { + @Override + public boolean apply(int value) + { + return lookupId != value; + } + }; } else { - throw new UnsupportedOperationException( - "FilteredAggregator does not support DimFilter of type " - + dimFilter.getClass() - ); + predicate = new IntPredicate() + { + @Override + public boolean apply(int value) + { + return lookupId == value; + } + }; } + return Pair.of(dimSelector, predicate); } } From 94da2b6ad3e794e573afebf46abc99441e2425e4 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 22 Oct 2014 16:16:10 -0700 Subject: [PATCH 65/69] more logging for potential NPE because of segment unmapping --- .../query/ChainedExecutionQueryRunner.java | 20 +++++++++---------- .../query/GroupByParallelQueryRunner.java | 8 +++++--- 2 files changed, 15 insertions(+), 13 deletions(-) diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index 77ca1b32fe1..6b474799de0 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -47,14 +47,14 @@ import java.util.concurrent.TimeoutException; /** * A QueryRunner that combines a list of other QueryRunners and executes them in parallel on an executor. - * + *

* When using this, it is important to make sure that the list of QueryRunners provided is fully flattened. * If, for example, you were to pass a list of a Chained QueryRunner (A) and a non-chained QueryRunner (B). Imagine * A has 2 QueryRunner chained together (Aa and Ab), the fact that the Queryables are run in parallel on an * executor would mean that the Queryables are actually processed in the order - * + *

*

A -> B -> Aa -> Ab
- * + *

* That is, the two sub queryables for A would run *after* B is run, effectively meaning that the results for B * must be fully cached in memory before the results for Aa and Ab are computed. */ @@ -113,6 +113,10 @@ public class ChainedExecutionQueryRunner implements QueryRunner @Override public ListenableFuture> apply(final QueryRunner input) { + if (input == null) { + throw new ISE("Null queryRunner! Looks to be some segment unmapping action happening"); + } + return exec.submit( new AbstractPrioritizedCallable>(priority) { @@ -120,10 +124,6 @@ public class ChainedExecutionQueryRunner implements QueryRunner public Iterable call() throws Exception { try { - if (input == null) { - throw new ISE("Input is null?! How is this possible?!"); - } - Sequence result = input.run(query); if (result == null) { throw new ISE("Got a null result! Segments are missing!"); @@ -155,7 +155,7 @@ public class ChainedExecutionQueryRunner implements QueryRunner queryWatcher.registerQuery(query, futures); try { - final Number timeout = query.getContextValue("timeout", (Number)null); + final Number timeout = query.getContextValue("timeout", (Number) null); return new MergeIterable<>( ordering.nullsFirst(), timeout == null ? @@ -168,10 +168,10 @@ public class ChainedExecutionQueryRunner implements QueryRunner futures.cancel(true); throw new QueryInterruptedException("Query interrupted"); } - catch(CancellationException e) { + catch (CancellationException e) { throw new QueryInterruptedException("Query cancelled"); } - catch(TimeoutException e) { + catch (TimeoutException e) { log.info("Query timeout, cancelling pending results for query id [%s]", query.getId()); futures.cancel(true); throw new QueryInterruptedException("Query timeout"); diff --git a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java index 1edf07b94d4..7e97ce03099 100644 --- a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java @@ -29,6 +29,7 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; +import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.Sequence; @@ -81,9 +82,6 @@ public class GroupByParallelQueryRunner implements QueryRunner final boolean bySegment = query.getContextBySegment(false); final int priority = query.getContextPriority(0); - if (Iterables.isEmpty(queryables)) { - log.warn("No queryables found."); - } ListenableFuture> futures = Futures.allAsList( Lists.newArrayList( Iterables.transform( @@ -93,6 +91,10 @@ public class GroupByParallelQueryRunner implements QueryRunner @Override public ListenableFuture apply(final QueryRunner input) { + if (input == null) { + throw new ISE("Null queryRunner! Looks to be some segment unmapping action happening"); + } + return exec.submit( new AbstractPrioritizedCallable(priority) { From d76d57d95db35c15eed7415b4234d02979711afd Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 22 Oct 2014 16:16:28 -0700 Subject: [PATCH 66/69] update docs --- docs/content/Examples.md | 4 ++-- docs/content/Kafka-Eight.md | 4 ++-- docs/content/Production-Cluster-Configuration.md | 6 +++--- docs/content/Realtime-Config.md | 4 ++-- docs/content/Router.md | 2 +- docs/content/Simple-Cluster-Configuration.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-1.md | 2 +- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.md | 2 +- .../src/main/java/io/druid/indexer/HadoopIngestionSpec.java | 2 +- 12 files changed, 21 insertions(+), 21 deletions(-) diff --git a/docs/content/Examples.md b/docs/content/Examples.md index dfd44ffe927..45c5d573892 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.159 +git checkout druid-0.6.160 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.159-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.160-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Kafka-Eight.md b/docs/content/Kafka-Eight.md index 5819a931b2a..175c71a65c6 100644 --- a/docs/content/Kafka-Eight.md +++ b/docs/content/Kafka-Eight.md @@ -8,9 +8,9 @@ The previous examples are for Kafka 7. To support Kafka 8, a couple changes need - Update realtime node's configs for Kafka 8 extensions - e.g. - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.159",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.160",...]` - becomes - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.159",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.160",...]` - Update realtime task config for changed keys - `firehose.type`, `plumber.rejectionPolicyFactory`, and all of `firehose.consumerProps` changes. diff --git a/docs/content/Production-Cluster-Configuration.md b/docs/content/Production-Cluster-Configuration.md index 6fc319712de..21190792663 100644 --- a/docs/content/Production-Cluster-Configuration.md +++ b/docs/content/Production-Cluster-Configuration.md @@ -57,7 +57,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/overlord -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.159"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.160"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -139,7 +139,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/middlemanager -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.159","io.druid.extensions:druid-kafka-seven:0.6.159"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.160","io.druid.extensions:druid-kafka-seven:0.6.160"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -286,7 +286,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/historical -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.159"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.160"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Realtime-Config.md b/docs/content/Realtime-Config.md index 12ab55ff8e3..92ab9382f81 100644 --- a/docs/content/Realtime-Config.md +++ b/docs/content/Realtime-Config.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.159"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.160"] druid.zk.service.host=localhost @@ -76,7 +76,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/realtime -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.159","io.druid.extensions:druid-kafka-seven:0.6.159"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.160","io.druid.extensions:druid-kafka-seven:0.6.160"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Router.md b/docs/content/Router.md index f046fe4b468..064bf1d646d 100644 --- a/docs/content/Router.md +++ b/docs/content/Router.md @@ -51,7 +51,7 @@ druid.service=druid/prod/router druid.extensions.remoteRepositories=[] druid.extensions.localRepository=lib -druid.extensions.coordinates=["io.druid.extensions:druid-histogram:0.6.159"] +druid.extensions.coordinates=["io.druid.extensions:druid-histogram:0.6.160"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Simple-Cluster-Configuration.md b/docs/content/Simple-Cluster-Configuration.md index 8bfade9ab9a..c25f349f0b0 100644 --- a/docs/content/Simple-Cluster-Configuration.md +++ b/docs/content/Simple-Cluster-Configuration.md @@ -28,7 +28,7 @@ Configuration: -Ddruid.zk.service.host=localhost --Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.159"] +-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.160"] -Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid -Ddruid.db.connector.user=druid diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 4a74b4f5b20..fadc7fbc963 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.159-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.160-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.159 +cd druid-services-0.6.160 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md index 7ef8648cc0f..eed3030c71e 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md @@ -91,7 +91,7 @@ druid.service=overlord druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.159"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.160"] druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid druid.db.connector.user=druid diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 3797746ca20..7b21b72aa69 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes and external depende If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.159-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.160-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.159"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.160"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -240,7 +240,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.159","io.druid.extensions:druid-kafka-seven:0.6.159"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.160","io.druid.extensions:druid-kafka-seven:0.6.160"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index 47ebec7e049..5739f625a5e 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.159-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.160-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.159 +cd druid-services-0.6.160 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.md b/docs/content/Twitter-Tutorial.md index 95993732449..844788a33b2 100644 --- a/docs/content/Twitter-Tutorial.md +++ b/docs/content/Twitter-Tutorial.md @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. # Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.159-bin.tar.gz). +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.160-bin.tar.gz). Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopIngestionSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopIngestionSpec.java index b9947e81fe9..bb560e27656 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopIngestionSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopIngestionSpec.java @@ -289,4 +289,4 @@ public class HadoopIngestionSpec extends IngestionSpec Date: Wed, 22 Oct 2014 16:25:27 -0700 Subject: [PATCH 67/69] [maven-release-plugin] prepare release druid-0.6.160 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 5 ++--- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 17 insertions(+), 18 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 5682138b49d..1bc9bd5c220 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160-SNAPSHOT + 0.6.160 diff --git a/common/pom.xml b/common/pom.xml index da437e99e1f..c9bf7c6a1e3 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160-SNAPSHOT + 0.6.160 diff --git a/examples/pom.xml b/examples/pom.xml index fb2597326e7..a9ee4c7a43c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160-SNAPSHOT + 0.6.160 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 6cb8f84483e..57cbf1de582 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160-SNAPSHOT + 0.6.160 diff --git a/histogram/pom.xml b/histogram/pom.xml index a26ce5f5601..99b2e687d5e 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.160-SNAPSHOT + 0.6.160 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index cfd2e8da529..45458aee97b 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160-SNAPSHOT + 0.6.160 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 5fe0ebae569..705e5ff1c5d 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160-SNAPSHOT + 0.6.160 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index be87d5b4d79..8999a6fb41a 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160-SNAPSHOT + 0.6.160 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index a26e940b725..46ef0deded2 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160-SNAPSHOT + 0.6.160 diff --git a/pom.xml b/pom.xml index c8eca1e8738..caaca66834f 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.160-SNAPSHOT + 0.6.160 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.131-SNAPSHOT + druid-0.6.160 diff --git a/processing/pom.xml b/processing/pom.xml index 867b720ca57..9deee1b42da 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160-SNAPSHOT + 0.6.160 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index a35f3b16c15..077f3385f33 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -1,6 +1,5 @@ - + 4.0.0 io.druid.extensions druid-rabbitmq @@ -10,7 +9,7 @@ io.druid druid - 0.6.160-SNAPSHOT + 0.6.160 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index e4914713493..33d346d2f79 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160-SNAPSHOT + 0.6.160 diff --git a/server/pom.xml b/server/pom.xml index 740e06d56ad..7e9461d2f00 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160-SNAPSHOT + 0.6.160 diff --git a/services/pom.xml b/services/pom.xml index 9968d0b6146..7f9aa5b3325 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.160-SNAPSHOT + 0.6.160 From 3b29e778668dbacd322fa05c7c8e0407c0c1524a Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 22 Oct 2014 16:25:32 -0700 Subject: [PATCH 68/69] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 1bc9bd5c220..57150629f14 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160 + 0.6.161-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index c9bf7c6a1e3..b4bbaf0f4f6 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160 + 0.6.161-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index a9ee4c7a43c..33515fffe61 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160 + 0.6.161-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 57cbf1de582..7cdaea48728 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160 + 0.6.161-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index 99b2e687d5e..f0e11ea375e 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.160 + 0.6.161-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 45458aee97b..24373fc733f 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160 + 0.6.161-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 705e5ff1c5d..653a2f6c033 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160 + 0.6.161-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 8999a6fb41a..79e28c016ec 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160 + 0.6.161-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 46ef0deded2..21acf89e6a4 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160 + 0.6.161-SNAPSHOT diff --git a/pom.xml b/pom.xml index caaca66834f..40f16c5b693 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.160 + 0.6.161-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.160 + druid-0.6.131-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index 9deee1b42da..0b562f2a2fa 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160 + 0.6.161-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 077f3385f33..210771cd139 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.160 + 0.6.161-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 33d346d2f79..1b34ec00764 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160 + 0.6.161-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 7e9461d2f00..e264ce024c7 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.160 + 0.6.161-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 7f9aa5b3325..65215245df7 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.160 + 0.6.161-SNAPSHOT From bef74104d994dc66d2163cb8b3feb963d426892c Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 23 Oct 2014 17:24:06 -0700 Subject: [PATCH 69/69] merge with 0.7.x and resolve any conflicts --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- .../io/druid/guice/DruidSecondaryModule.java | 3 +- examples/bin/run_example_server.sh | 1 + .../config/_global/global.runtime.properties | 23 + examples/config/broker/runtime.properties | 4 +- .../config/coordinator/runtime.properties | 6 - examples/config/historical/runtime.properties | 6 +- examples/config/overlord/runtime.properties | 14 +- examples/config/realtime/runtime.properties | 12 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- .../ApproximateHistogramBufferAggregator.java | 7 + ...imateHistogramFoldingBufferAggregator.java | 6 + .../ApproximateHistogramGroupByQueryTest.java | 6 +- .../ApproximateHistogramTopNQueryTest.java | 4 +- indexing-hadoop/pom.xml | 2 +- .../indexer/HadoopDruidIndexerConfig.java | 6 +- .../druid/indexer/HadoopDruidIndexerJob.java | 26 +- .../io/druid/indexer/HadoopIngestionSpec.java | 8 +- .../io/druid/indexer/HadoopTuningConfig.java | 33 +- .../io/druid/indexer/IndexGeneratorJob.java | 258 +- .../indexer/LegacyIndexGeneratorJob.java | 86 + .../druid/indexer/rollup/DataRollupSpec.java | 20 +- .../indexer/HadoopDruidIndexerConfigTest.java | 6 +- indexing-service/pom.xml | 2 +- .../common/index/YeOldePlumberSchool.java | 7 +- .../indexing/common/task/AppendTask.java | 4 +- .../indexing/common/task/DeleteTask.java | 109 - .../druid/indexing/common/task/IndexTask.java | 10 +- .../druid/indexing/common/task/MergeTask.java | 4 +- .../common/task/RealtimeIndexTask.java | 4 +- .../io/druid/indexing/common/task/Task.java | 1 - .../IngestSegmentFirehoseFactory.java | 7 +- .../config/ForkingTaskRunnerConfig.java | 2 +- .../indexing/common/task/TaskSerdeTest.java | 47 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 18 +- processing/pom.xml | 21 +- .../java/io/druid/guice/GuiceInjectors.java | 5 +- .../java/io/druid/guice/PropertiesModule.java | 53 +- .../main/java/io/druid/query/BaseQuery.java | 8 +- .../io/druid/query/BySegmentQueryRunner.java | 8 +- .../query/BySegmentSkippingQueryRunner.java | 10 +- .../query/ChainedExecutionQueryRunner.java | 9 +- .../io/druid/query/ConcatQueryRunner.java | 7 +- .../query/FinalizeResultsQueryRunner.java | 8 +- .../query/GroupByParallelQueryRunner.java | 47 +- .../query/IntervalChunkingQueryRunner.java | 8 +- .../query/MetricsEmittingQueryRunner.java | 8 +- .../java/io/druid/query/NoopQueryRunner.java | 5 +- .../src/main/java/io/druid/query/Query.java | 4 +- .../main/java/io/druid/query/QueryRunner.java | 6 +- .../ReferenceCountingSegmentQueryRunner.java | 6 +- .../druid/query/ResultMergeQueryRunner.java | 7 +- .../java/io/druid/query/RetryQueryRunner.java | 113 + .../druid/query/RetryQueryRunnerConfig.java | 33 + .../io/druid/query/SubqueryQueryRunner.java | 9 +- .../java/io/druid/query/TimewarpOperator.java | 7 +- .../java/io/druid/query/UnionQueryRunner.java | 10 +- .../druid/query/aggregation/Aggregators.java | 7 + .../query/aggregation/BufferAggregator.java | 17 + .../aggregation/CountBufferAggregator.java | 7 + .../DoubleSumBufferAggregator.java | 7 + .../aggregation/FilteredBufferAggregator.java | 9 +- .../HistogramBufferAggregator.java | 6 + .../JavaScriptBufferAggregator.java | 7 + .../query/aggregation/LongSumAggregator.java | 7 +- .../aggregation/LongSumAggregatorFactory.java | 6 +- .../aggregation/LongSumBufferAggregator.java | 14 +- .../aggregation/MaxBufferAggregator.java | 6 + .../aggregation/MinBufferAggregator.java | 7 + .../CardinalityBufferAggregator.java | 9 +- .../HyperUniquesBufferAggregator.java | 9 +- .../query/groupby/GroupByQueryHelper.java | 29 +- .../groupby/GroupByQueryQueryToolChest.java | 46 +- .../groupby/GroupByQueryRunnerFactory.java | 27 +- .../druid/query/metadata/SegmentAnalyzer.java | 2 +- .../SegmentMetadataQueryRunnerFactory.java | 9 +- .../search/SearchQueryQueryToolChest.java | 9 +- .../druid/query/search/SearchQueryRunner.java | 5 +- .../druid/query/select/SelectQueryEngine.java | 7 +- .../select/SelectQueryRunnerFactory.java | 7 +- .../spec/SpecificSegmentQueryRunner.java | 49 +- .../druid/query/spec/SpecificSegmentSpec.java | 2 + .../TimeBoundaryQueryQueryToolChest.java | 5 +- .../TimeBoundaryQueryRunnerFactory.java | 7 +- .../timeseries/TimeseriesQueryEngine.java | 4 +- .../TimeseriesQueryRunnerFactory.java | 7 +- .../io/druid/query/topn/TopNQueryEngine.java | 4 +- .../query/topn/TopNQueryQueryToolChest.java | 9 +- .../query/topn/TopNQueryRunnerFactory.java | 7 +- .../druid/segment/BaseProgressIndicator.java | 61 + .../java/io/druid/segment/ColumnSelector.java | 1 - .../ColumnSelectorBitmapIndexSelector.java | 2 +- .../druid/segment/ColumnSelectorFactory.java | 2 +- .../segment/FloatMetricColumnSerializer.java | 4 +- .../segment/IncrementalIndexSegment.java | 2 +- .../main/java/io/druid/segment/IndexIO.java | 23 +- .../java/io/druid/segment/IndexMaker.java | 1739 +++ .../java/io/druid/segment/IndexMerger.java | 111 +- .../io/druid/segment/IndexableAdapter.java | 9 +- .../segment/LoggingProgressIndicator.java | 99 + ...nSelector.java => LongColumnSelector.java} | 4 +- .../segment/LongMetricColumnSerializer.java | 81 + .../io/druid/segment/MMappedIndexAdapter.java | 197 - .../java/io/druid/segment/MetricHolder.java | 35 +- .../io/druid/segment/ProgressIndicator.java | 37 + .../QueryableIndexIndexableAdapter.java | 45 +- .../segment/QueryableIndexStorageAdapter.java | 114 +- .../main/java/io/druid/segment/Rowboat.java | 11 +- .../segment/RowboatFilteringIndexAdapter.java | 15 +- .../segment/SegmentMissingException.java | 29 + .../druid/segment/SimpleQueryableIndex.java | 21 +- .../java/io/druid/segment/column/Column.java | 1 + .../segment/column/ColumnCapabilities.java | 2 + .../column/ColumnCapabilitiesImpl.java | 25 + .../column/IndexedFloatsGenericColumn.java | 2 +- .../column/IndexedLongsGenericColumn.java | 2 +- .../CompressedFloatBufferObjectStrategy.java | 12 +- .../data/CompressedFloatsIndexedSupplier.java | 61 +- .../CompressedFloatsSupplierSerializer.java | 17 +- .../CompressedLongBufferObjectStrategy.java | 13 +- .../data/CompressedLongsIndexedSupplier.java | 45 +- .../CompressedLongsSupplierSerializer.java | 16 +- .../data/CompressedObjectStrategy.java | 255 +- .../FixedSizeCompressedObjectStrategy.java | 56 + .../segment/data/GenericIndexedWriter.java | 2 - .../data/InMemoryCompressedFloats.java | 10 +- .../segment/data/InMemoryCompressedLongs.java | 10 +- .../druid/segment/data/VSizeIndexedInts.java | 13 +- .../segment/data/VSizeIndexedWriter.java | 2 +- .../segment/incremental/IncrementalIndex.java | 655 +- .../incremental/IncrementalIndexAdapter.java | 35 +- .../incremental/IncrementalIndexSchema.java | 65 +- .../IncrementalIndexStorageAdapter.java | 103 +- .../incremental/OffheapIncrementalIndex.java | 274 + ...va => SpatialDimensionRowTransformer.java} | 15 +- .../DictionaryEncodedColumnPartSerde.java | 56 +- .../ChainedExecutionQueryRunnerTest.java | 14 +- .../io/druid/query/QueryRunnerTestHelper.java | 13 +- .../io/druid/query/RetryQueryRunnerTest.java | 266 + .../java/io/druid/query/TestQueryRunners.java | 3 +- .../io/druid/query/TimewarpOperatorTest.java | 36 +- .../aggregation/FilteredAggregatorTest.java | 14 +- .../aggregation/LongSumAggregatorTest.java | 6 +- .../aggregation/TestLongColumnSelector.java | 48 + .../query/groupby/GroupByQueryRunnerTest.java | 524 +- .../groupby/GroupByQueryRunnerTestHelper.java | 2 +- .../GroupByTimeseriesQueryRunnerTest.java | 22 +- .../query/metadata/SegmentAnalyzerTest.java | 6 +- .../metadata/SegmentMetadataQueryTest.java | 6 +- .../query/search/SearchQueryRunnerTest.java | 4 +- .../query/select/SelectQueryRunnerTest.java | 21 +- .../TimeBoundaryQueryRunnerTest.java | 18 +- .../TimeSeriesUnionQueryRunnerTest.java | 12 +- .../TimeseriesQueryRunnerBonusTest.java | 9 +- .../timeseries/TimeseriesQueryRunnerTest.java | 83 +- .../druid/query/topn/TopNQueryRunnerTest.java | 134 +- .../druid/query/topn/TopNUnionQueryTest.java | 5 +- .../java/io/druid/segment/AppendTest.java | 47 +- .../java/io/druid/segment/EmptyIndexTest.java | 16 +- ...dexMergerTest.java => IndexMakerTest.java} | 45 +- .../io/druid/segment/SchemalessIndex.java | 25 +- .../io/druid/segment/SchemalessTestFull.java | 39 +- .../druid/segment/SchemalessTestSimple.java | 13 +- .../test/java/io/druid/segment/TestIndex.java | 55 +- .../CompressedFloatsIndexedSupplierTest.java | 17 +- ...ompressedFloatsSupplierSerializerTest.java | 24 +- .../CompressedLongsIndexedSupplierTest.java | 17 +- ...CompressedLongsSupplierSerializerTest.java | 20 +- .../segment/data/CompressionStrategyTest.java | 52 + .../segment/data/IncrementalIndexTest.java | 9 +- .../filter/SpatialFilterBonusTest.java | 112 +- .../segment/filter/SpatialFilterTest.java | 117 +- .../IncrementalIndexStorageAdapterTest.java | 15 +- rabbitmq/pom.xml | 4 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 5 +- .../druid/client/CachingClusteredClient.java | 8 +- .../io/druid/client/CachingQueryRunner.java | 7 +- .../io/druid/client/DirectDruidClient.java | 81 +- .../java/io/druid/curator/CuratorConfig.java | 2 +- .../inventory/CuratorInventoryManager.java | 3 +- .../io/druid/guice/DruidProcessingModule.java | 28 +- .../io/druid/offheap/OffheapBufferPool.java | 53 + .../indexing/RealtimeTuningConfig.java | 34 +- .../segment/realtime/FireDepartment.java | 4 +- .../druid/segment/realtime/FireHydrant.java | 26 +- .../firehose/WikipediaIrcDecoder.java | 6 + .../realtime/plumber/RealtimePlumber.java | 53 +- .../druid/segment/realtime/plumber/Sink.java | 38 +- .../server/ClientQuerySegmentWalker.java | 17 +- .../main/java/io/druid/server/DruidNode.java | 21 +- .../java/io/druid/server/QueryResource.java | 9 +- .../bridge/BridgeQuerySegmentWalker.java | 3 +- .../DataSegmentAnnouncerProvider.java | 2 +- .../server/coordinator/DruidCoordinator.java | 2 +- .../coordinator/DruidCoordinatorConfig.java | 3 - .../server/initialization/EmitterModule.java | 5 +- .../initialization/NoopEmitterModule.java | 48 + .../resources/static/css/bootstrap-theme.css | 347 + .../main/resources/static/css/bootstrap.css | 5785 +++++++ .../resources/static/css/font-awesome.css | 1338 ++ server/src/main/resources/static/druid.css | 985 ++ server/src/main/resources/static/druid.js | 12761 ++++++++++++++++ server/src/main/resources/static/favicon.ico | Bin 0 -> 318 bytes .../resources/static/fonts/FontAwesome.otf | Bin 0 -> 62856 bytes .../static/fonts/fontawesome-webfont.eot | Bin 0 -> 38205 bytes .../static/fonts/fontawesome-webfont.svg | 414 + .../static/fonts/fontawesome-webfont.ttf | Bin 0 -> 80652 bytes .../static/fonts/fontawesome-webfont.woff | Bin 0 -> 44432 bytes server/src/main/resources/static/index.html | 60 +- .../static/{ => old-console}/cluster.html | 0 .../static/{ => old-console}/config.html | 0 .../static/{ => old-console}/css/config.css | 0 .../{ => old-console}/css/demo_table.css | 0 .../static/{ => old-console}/css/enable.css | 0 .../css/images/ui-bg_flat_0_aaaaaa_40x100.png | Bin .../images/ui-bg_flat_75_ffffff_40x100.png | Bin .../images/ui-bg_glass_55_fbf9ee_1x400.png | Bin .../images/ui-bg_glass_65_ffffff_1x400.png | Bin .../images/ui-bg_glass_75_dadada_1x400.png | Bin .../images/ui-bg_glass_75_e6e6e6_1x400.png | Bin .../images/ui-bg_glass_95_fef1ec_1x400.png | Bin .../ui-bg_highlight-soft_75_cccccc_1x100.png | Bin .../css/images/ui-icons_222222_256x240.png | Bin .../css/images/ui-icons_2e83ff_256x240.png | Bin .../css/images/ui-icons_454545_256x240.png | Bin .../css/images/ui-icons_888888_256x240.png | Bin .../css/images/ui-icons_cd0a0a_256x240.png | Bin .../static/{ => old-console}/css/index.css | 0 .../{ => old-console}/css/jquery-ui-1.9.2.css | 0 .../static/{ => old-console}/css/rules.css | 0 .../static/{ => old-console}/css/style.css | 0 .../static/{ => old-console}/enable.html | 0 .../images/back_disabled.jpg | Bin .../{ => old-console}/images/back_enabled.jpg | Bin .../{ => old-console}/images/favicon.ico | Bin .../images/forward_disabled.jpg | Bin .../images/forward_enabled.jpg | Bin .../{ => old-console}/images/sort_asc.png | Bin .../images/sort_asc_disabled.png | Bin .../{ => old-console}/images/sort_both.png | Bin .../{ => old-console}/images/sort_desc.png | Bin .../images/sort_desc_disabled.png | Bin .../resources/static/old-console/index.html | 52 + .../{ => old-console}/js/config-0.0.2.js | 0 .../{ => old-console}/js/druidTable-0.0.1.js | 0 .../{ => old-console}/js/enable-0.0.1.js | 0 .../{ => old-console}/js/handlers-0.0.2.js | 0 .../static/{ => old-console}/js/init-0.0.2.js | 0 .../{ => old-console}/js/jquery-1.11.0.min.js | 0 .../{ => old-console}/js/jquery-ui-1.9.2.js | 0 .../js/jquery.dataTables-1.8.2.js | 0 .../static/{ => old-console}/js/kill-0.0.1.js | 0 .../{ => old-console}/js/rules-0.0.2.js | 0 .../{ => old-console}/js/tablehelper-0.0.2.js | 0 .../{ => old-console}/js/underscore-1.2.2.js | 0 .../static/{ => old-console}/kill.html | 0 .../static/{ => old-console}/rules.html | 0 .../static/{ => old-console}/view.html | 0 .../main/resources/static/pages/cluster.html | 99 + .../resources/static/pages/data-source.html | 271 + .../resources/static/pages/data-sources.html | 175 + .../main/resources/static/pages/disable.html | 13 + .../main/resources/static/pages/rules.html | 124 + .../main/resources/static/pages/site-nav.html | 18 + .../client/CachingClusteredClientTest.java | 76 +- .../druid/client/CachingQueryRunnerTest.java | 13 +- .../druid/client/DirectDruidClientTest.java | 21 +- .../initialization/InitializationTest.java | 2 +- .../CombiningFirehoseFactoryTest.java | 6 + .../segment/realtime/FireDepartmentTest.java | 2 +- .../segment/realtime/RealtimeManagerTest.java | 8 + .../plumber/RealtimePlumberSchoolTest.java | 2 + .../segment/realtime/plumber/SinkTest.java | 18 +- .../coordination/ServerManagerTest.java | 10 +- .../coordinator/DruidCoordinatorTest.java | 6 - .../server/initialization/JettyTest.java | 75 +- .../shard/HashBasedNumberedShardSpecTest.java | 6 + services/pom.xml | 2 +- services/src/assembly/assembly.xml | 7 + .../src/main/java/io/druid/cli/CliBridge.java | 4 + .../src/main/java/io/druid/cli/CliBroker.java | 6 + .../java/io/druid/cli/CliCoordinator.java | 4 + .../main/java/io/druid/cli/CliHistorical.java | 4 + .../java/io/druid/cli/CliMiddleManager.java | 4 + .../main/java/io/druid/cli/CliOverlord.java | 4 + .../main/java/io/druid/cli/CliRealtime.java | 13 +- .../src/main/java/io/druid/cli/CliRouter.java | 4 + .../main/java/io/druid/cli/CreateTables.java | 116 + services/src/main/java/io/druid/cli/Main.java | 2 +- 295 files changed, 29010 insertions(+), 2080 deletions(-) create mode 100644 examples/config/_global/global.runtime.properties create mode 100644 indexing-hadoop/src/main/java/io/druid/indexer/LegacyIndexGeneratorJob.java delete mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/DeleteTask.java create mode 100644 processing/src/main/java/io/druid/query/RetryQueryRunner.java create mode 100644 processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java create mode 100644 processing/src/main/java/io/druid/segment/BaseProgressIndicator.java create mode 100644 processing/src/main/java/io/druid/segment/IndexMaker.java create mode 100644 processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java rename processing/src/main/java/io/druid/segment/{TimestampColumnSelector.java => LongColumnSelector.java} (92%) create mode 100644 processing/src/main/java/io/druid/segment/LongMetricColumnSerializer.java delete mode 100644 processing/src/main/java/io/druid/segment/MMappedIndexAdapter.java create mode 100644 processing/src/main/java/io/druid/segment/ProgressIndicator.java create mode 100644 processing/src/main/java/io/druid/segment/SegmentMissingException.java create mode 100644 processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java create mode 100644 processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java rename processing/src/main/java/io/druid/segment/incremental/{SpatialDimensionRowFormatter.java => SpatialDimensionRowTransformer.java} (94%) create mode 100644 processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java create mode 100644 processing/src/test/java/io/druid/query/aggregation/TestLongColumnSelector.java rename processing/src/test/java/io/druid/segment/{IndexMergerTest.java => IndexMakerTest.java} (74%) create mode 100644 processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java create mode 100644 server/src/main/java/io/druid/offheap/OffheapBufferPool.java create mode 100644 server/src/main/java/io/druid/server/initialization/NoopEmitterModule.java create mode 100644 server/src/main/resources/static/css/bootstrap-theme.css create mode 100644 server/src/main/resources/static/css/bootstrap.css create mode 100644 server/src/main/resources/static/css/font-awesome.css create mode 100644 server/src/main/resources/static/druid.css create mode 100644 server/src/main/resources/static/druid.js create mode 100644 server/src/main/resources/static/favicon.ico create mode 100644 server/src/main/resources/static/fonts/FontAwesome.otf create mode 100755 server/src/main/resources/static/fonts/fontawesome-webfont.eot create mode 100755 server/src/main/resources/static/fonts/fontawesome-webfont.svg create mode 100755 server/src/main/resources/static/fonts/fontawesome-webfont.ttf create mode 100755 server/src/main/resources/static/fonts/fontawesome-webfont.woff rename server/src/main/resources/static/{ => old-console}/cluster.html (100%) rename server/src/main/resources/static/{ => old-console}/config.html (100%) rename server/src/main/resources/static/{ => old-console}/css/config.css (100%) rename server/src/main/resources/static/{ => old-console}/css/demo_table.css (100%) rename server/src/main/resources/static/{ => old-console}/css/enable.css (100%) rename server/src/main/resources/static/{ => old-console}/css/images/ui-bg_flat_0_aaaaaa_40x100.png (100%) rename server/src/main/resources/static/{ => old-console}/css/images/ui-bg_flat_75_ffffff_40x100.png (100%) rename server/src/main/resources/static/{ => old-console}/css/images/ui-bg_glass_55_fbf9ee_1x400.png (100%) rename server/src/main/resources/static/{ => old-console}/css/images/ui-bg_glass_65_ffffff_1x400.png (100%) rename server/src/main/resources/static/{ => old-console}/css/images/ui-bg_glass_75_dadada_1x400.png (100%) rename server/src/main/resources/static/{ => old-console}/css/images/ui-bg_glass_75_e6e6e6_1x400.png (100%) rename server/src/main/resources/static/{ => old-console}/css/images/ui-bg_glass_95_fef1ec_1x400.png (100%) rename server/src/main/resources/static/{ => old-console}/css/images/ui-bg_highlight-soft_75_cccccc_1x100.png (100%) rename server/src/main/resources/static/{ => old-console}/css/images/ui-icons_222222_256x240.png (100%) rename server/src/main/resources/static/{ => old-console}/css/images/ui-icons_2e83ff_256x240.png (100%) rename server/src/main/resources/static/{ => old-console}/css/images/ui-icons_454545_256x240.png (100%) rename server/src/main/resources/static/{ => old-console}/css/images/ui-icons_888888_256x240.png (100%) rename server/src/main/resources/static/{ => old-console}/css/images/ui-icons_cd0a0a_256x240.png (100%) rename server/src/main/resources/static/{ => old-console}/css/index.css (100%) rename server/src/main/resources/static/{ => old-console}/css/jquery-ui-1.9.2.css (100%) rename server/src/main/resources/static/{ => old-console}/css/rules.css (100%) rename server/src/main/resources/static/{ => old-console}/css/style.css (100%) rename server/src/main/resources/static/{ => old-console}/enable.html (100%) rename server/src/main/resources/static/{ => old-console}/images/back_disabled.jpg (100%) rename server/src/main/resources/static/{ => old-console}/images/back_enabled.jpg (100%) rename server/src/main/resources/static/{ => old-console}/images/favicon.ico (100%) rename server/src/main/resources/static/{ => old-console}/images/forward_disabled.jpg (100%) rename server/src/main/resources/static/{ => old-console}/images/forward_enabled.jpg (100%) rename server/src/main/resources/static/{ => old-console}/images/sort_asc.png (100%) rename server/src/main/resources/static/{ => old-console}/images/sort_asc_disabled.png (100%) rename server/src/main/resources/static/{ => old-console}/images/sort_both.png (100%) rename server/src/main/resources/static/{ => old-console}/images/sort_desc.png (100%) rename server/src/main/resources/static/{ => old-console}/images/sort_desc_disabled.png (100%) create mode 100644 server/src/main/resources/static/old-console/index.html rename server/src/main/resources/static/{ => old-console}/js/config-0.0.2.js (100%) rename server/src/main/resources/static/{ => old-console}/js/druidTable-0.0.1.js (100%) rename server/src/main/resources/static/{ => old-console}/js/enable-0.0.1.js (100%) rename server/src/main/resources/static/{ => old-console}/js/handlers-0.0.2.js (100%) rename server/src/main/resources/static/{ => old-console}/js/init-0.0.2.js (100%) rename server/src/main/resources/static/{ => old-console}/js/jquery-1.11.0.min.js (100%) rename server/src/main/resources/static/{ => old-console}/js/jquery-ui-1.9.2.js (100%) rename server/src/main/resources/static/{ => old-console}/js/jquery.dataTables-1.8.2.js (100%) rename server/src/main/resources/static/{ => old-console}/js/kill-0.0.1.js (100%) rename server/src/main/resources/static/{ => old-console}/js/rules-0.0.2.js (100%) rename server/src/main/resources/static/{ => old-console}/js/tablehelper-0.0.2.js (100%) rename server/src/main/resources/static/{ => old-console}/js/underscore-1.2.2.js (100%) rename server/src/main/resources/static/{ => old-console}/kill.html (100%) rename server/src/main/resources/static/{ => old-console}/rules.html (100%) rename server/src/main/resources/static/{ => old-console}/view.html (100%) create mode 100644 server/src/main/resources/static/pages/cluster.html create mode 100644 server/src/main/resources/static/pages/data-source.html create mode 100644 server/src/main/resources/static/pages/data-sources.html create mode 100644 server/src/main/resources/static/pages/disable.html create mode 100644 server/src/main/resources/static/pages/rules.html create mode 100644 server/src/main/resources/static/pages/site-nav.html create mode 100644 services/src/main/java/io/druid/cli/CreateTables.java diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 57150629f14..c8fec800e3e 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.161-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index b4bbaf0f4f6..03e902df7c5 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.161-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/common/src/main/java/io/druid/guice/DruidSecondaryModule.java b/common/src/main/java/io/druid/guice/DruidSecondaryModule.java index fbafb29d42b..d768a60a7c1 100644 --- a/common/src/main/java/io/druid/guice/DruidSecondaryModule.java +++ b/common/src/main/java/io/druid/guice/DruidSecondaryModule.java @@ -71,7 +71,8 @@ public class DruidSecondaryModule implements Module binder.install(new DruidGuiceExtensions()); binder.bind(Properties.class).toInstance(properties); binder.bind(ConfigurationObjectFactory.class).toInstance(factory); - binder.bind(ObjectMapper.class).to(Key.get(ObjectMapper.class, Json.class)); + // make objectMapper eager to ensure jackson gets setup with guice injection for JsonConfigurator + binder.bind(ObjectMapper.class).to(Key.get(ObjectMapper.class, Json.class)).asEagerSingleton(); binder.bind(Validator.class).toInstance(validator); binder.bind(JsonConfigurator.class).toInstance(jsonConfigurator); } diff --git a/examples/bin/run_example_server.sh b/examples/bin/run_example_server.sh index 3d63e7cb0c3..461f937beb0 100755 --- a/examples/bin/run_example_server.sh +++ b/examples/bin/run_example_server.sh @@ -58,6 +58,7 @@ DRUID_CP=${EXAMPLE_LOC} DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/../config/realtime #For the kit DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/lib/* +DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/config/_global DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/config/realtime echo "Running command:" diff --git a/examples/config/_global/global.runtime.properties b/examples/config/_global/global.runtime.properties new file mode 100644 index 00000000000..beb3dba77c2 --- /dev/null +++ b/examples/config/_global/global.runtime.properties @@ -0,0 +1,23 @@ +# Extensions +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.7.0","io.druid.extensions:druid-kafka-seven:0.7.0","io.druid.extensions:druid-rabbitmq:0.7.0", "io.druid.extensions:druid-s3-extensions:0.7.0"] + +# Zookeeper +druid.zk.service.host=localhost + +# Metadata Storage +druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid +druid.db.connector.user=druid +druid.db.connector.password=diurd + +# Deep storage +druid.storage.type=local +druid.storage.storage.storageDirectory=/tmp/druid/localStorage + +# Indexing service discovery +druid.selectors.indexing.serviceName=overlord + +# Monitoring (disabled for examples) +# druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor"] + +# Metrics logging (disabled for examples) +druid.emitter=noop diff --git a/examples/config/broker/runtime.properties b/examples/config/broker/runtime.properties index 8afae982654..23d1170343b 100644 --- a/examples/config/broker/runtime.properties +++ b/examples/config/broker/runtime.properties @@ -2,8 +2,6 @@ druid.host=localhost druid.service=broker druid.port=8080 -druid.zk.service.host=localhost - -# Change these to make Druid faster +# Bump these up only for faster nested groupBy druid.processing.buffer.sizeBytes=100000000 druid.processing.numThreads=1 diff --git a/examples/config/coordinator/runtime.properties b/examples/config/coordinator/runtime.properties index 3d68fec772e..c9f16857af4 100644 --- a/examples/config/coordinator/runtime.properties +++ b/examples/config/coordinator/runtime.properties @@ -2,10 +2,4 @@ druid.host=localhost druid.service=coordinator druid.port=8082 -druid.zk.service.host=localhost - -druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid -druid.db.connector.user=druid -druid.db.connector.password=diurd - druid.coordinator.startDelay=PT70s \ No newline at end of file diff --git a/examples/config/historical/runtime.properties b/examples/config/historical/runtime.properties index e21651760b0..544e5c860e1 100644 --- a/examples/config/historical/runtime.properties +++ b/examples/config/historical/runtime.properties @@ -2,9 +2,7 @@ druid.host=localhost druid.service=historical druid.port=8081 -druid.zk.service.host=localhost - -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.147"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.7.0"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -16,4 +14,4 @@ druid.server.maxSize=10000000000 druid.processing.buffer.sizeBytes=100000000 druid.processing.numThreads=1 -druid.segmentCache.locations=[{"path": "/tmp/druid/indexCache", "maxSize"\: 10000000000}] \ No newline at end of file +druid.segmentCache.locations=[{"path": "/tmp/druid/indexCache", "maxSize"\: 10000000000}] diff --git a/examples/config/overlord/runtime.properties b/examples/config/overlord/runtime.properties index f727c0e012b..3eebd15a9a9 100644 --- a/examples/config/overlord/runtime.properties +++ b/examples/config/overlord/runtime.properties @@ -1,18 +1,8 @@ druid.host=localhost -druid.port=8087 +druid.port=8080 druid.service=overlord -druid.zk.service.host=localhost - -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.147"] - -druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid -druid.db.connector.user=druid -druid.db.connector.password=diurd - -druid.selectors.indexing.serviceName=overlord druid.indexer.queue.startDelay=PT0M druid.indexer.runner.javaOpts="-server -Xmx256m" -druid.indexer.runner.startPort=8088 druid.indexer.fork.property.druid.processing.numThreads=1 -druid.indexer.fork.property.druid.computation.buffer.size=100000000 \ No newline at end of file +druid.indexer.fork.property.druid.computation.buffer.size=100000000 diff --git a/examples/config/realtime/runtime.properties b/examples/config/realtime/runtime.properties index e20d1b56b8f..ccde5b5bb3b 100644 --- a/examples/config/realtime/runtime.properties +++ b/examples/config/realtime/runtime.properties @@ -2,19 +2,11 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.zk.service.host=localhost - -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.147","io.druid.extensions:druid-kafka-seven:0.6.147","io.druid.extensions:druid-rabbitmq:0.6.147"] - # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop -# These configs are only required for real hand off -# druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid -# druid.db.connector.user=druid -# druid.db.connector.password=diurd - druid.processing.buffer.sizeBytes=100000000 druid.processing.numThreads=1 -druid.monitoring.monitors=["io.druid.segment.realtime.RealtimeMetricsMonitor"] +# Enable Real monitoring +# druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor","io.druid.segment.realtime.RealtimeMetricsMonitor"] diff --git a/examples/pom.xml b/examples/pom.xml index 33515fffe61..896b772738e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.161-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 7cdaea48728..ac2c29b7405 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.161-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index f0e11ea375e..a520de6725b 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.161-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java index dbd566f2693..c150d3810ce 100644 --- a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java @@ -87,6 +87,13 @@ public class ApproximateHistogramBufferAggregator implements BufferAggregator throw new UnsupportedOperationException("ApproximateHistogramBufferAggregator does not support getFloat()"); } + + @Override + public long getLong(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("ApproximateHistogramBufferAggregator does not support getLong()"); + } + @Override public void close() { diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java index 4190ae50a40..cbf3e115500 100644 --- a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java @@ -91,6 +91,12 @@ public class ApproximateHistogramFoldingBufferAggregator implements BufferAggreg throw new UnsupportedOperationException("ApproximateHistogramFoldingBufferAggregator does not support getFloat()"); } + @Override + public long getLong(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("ApproximateHistogramFoldingBufferAggregator does not support getLong()"); + } + @Override public void close() { diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index 1c7b23b3a3f..48814642f5b 100644 --- a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -86,7 +86,8 @@ public class ApproximateHistogramGroupByQueryTest engine, QueryRunnerTestHelper.NOOP_QUERYWATCHER, configSupplier, - new GroupByQueryQueryToolChest(configSupplier, mapper, engine) + new GroupByQueryQueryToolChest(configSupplier, mapper, engine, pool), + pool ); GroupByQueryConfig singleThreadedConfig = new GroupByQueryConfig() @@ -106,7 +107,8 @@ public class ApproximateHistogramGroupByQueryTest singleThreadEngine, QueryRunnerTestHelper.NOOP_QUERYWATCHER, singleThreadedConfigSupplier, - new GroupByQueryQueryToolChest(singleThreadedConfigSupplier, mapper, singleThreadEngine) + new GroupByQueryQueryToolChest(singleThreadedConfigSupplier, mapper, singleThreadEngine, pool), + pool ); diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java index 25f58c064bf..bba68f5339e 100644 --- a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java @@ -48,6 +48,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -240,7 +241,8 @@ public class ApproximateHistogramTopNQueryTest ) ) ); + HashMap context = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } } diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 24373fc733f..f23382b6de9 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.161-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index 335c8c1858b..8c6eaf8d528 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -48,7 +48,6 @@ import io.druid.guice.annotations.Self; import io.druid.indexer.partitions.PartitionsSpec; import io.druid.indexer.path.PathSpec; import io.druid.initialization.Initialization; -import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.server.DruidNode; import io.druid.timeline.DataSegment; @@ -394,6 +393,11 @@ public class HadoopDruidIndexerConfig } } + public boolean isPersistInHeap() + { + return schema.getTuningConfig().isPersistInHeap(); + } + /****************************************** Path helper logic ******************************************/ diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java index 64bc1267146..4f41cc52df8 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java @@ -57,7 +57,11 @@ public class HadoopDruidIndexerJob implements Jobby List jobs = Lists.newArrayList(); JobHelper.ensurePaths(config); - indexJob = new IndexGeneratorJob(config); + if (config.isPersistInHeap()) { + indexJob = new IndexGeneratorJob(config); + } else { + indexJob = new LegacyIndexGeneratorJob(config); + } jobs.add(indexJob); if (dbUpdaterJob != null) { @@ -66,15 +70,17 @@ public class HadoopDruidIndexerJob implements Jobby log.info("No updaterJobSpec set, not uploading to database"); } - jobs.add(new Jobby() - { - @Override - public boolean run() - { - publishedSegments = IndexGeneratorJob.getPublishedSegments(config); - return true; - } - }); + jobs.add( + new Jobby() + { + @Override + public boolean run() + { + publishedSegments = IndexGeneratorJob.getPublishedSegments(config); + return true; + } + } + ); JobHelper.runJobs(jobs, config); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopIngestionSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopIngestionSpec.java index bb560e27656..fec163e4164 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopIngestionSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopIngestionSpec.java @@ -159,13 +159,15 @@ public class HadoopIngestionSpec extends IngestionSpec jobProperties; private final boolean combineText; + private final boolean persistInHeap; + private final boolean ingestOffheap; @JsonCreator public HadoopTuningConfig( @@ -81,7 +85,9 @@ public class HadoopTuningConfig implements TuningConfig final @JsonProperty("overwriteFiles") boolean overwriteFiles, final @JsonProperty("ignoreInvalidRows") boolean ignoreInvalidRows, final @JsonProperty("jobProperties") Map jobProperties, - final @JsonProperty("combineText") boolean combineText + final @JsonProperty("combineText") boolean combineText, + final @JsonProperty("persistInHeap") boolean persistInHeap, + final @JsonProperty("ingestOffheap") boolean ingestOffheap ) { this.workingPath = workingPath == null ? null : workingPath; @@ -97,6 +103,8 @@ public class HadoopTuningConfig implements TuningConfig ? ImmutableMap.of() : ImmutableMap.copyOf(jobProperties)); this.combineText = combineText; + this.persistInHeap = persistInHeap; + this.ingestOffheap = ingestOffheap; } @JsonProperty @@ -165,6 +173,17 @@ public class HadoopTuningConfig implements TuningConfig return combineText; } + @JsonProperty + public boolean isPersistInHeap() + { + return persistInHeap; + } + + @JsonProperty + public boolean isIngestOffheap(){ + return ingestOffheap; + } + public HadoopTuningConfig withWorkingPath(String path) { return new HadoopTuningConfig( @@ -178,7 +197,9 @@ public class HadoopTuningConfig implements TuningConfig overwriteFiles, ignoreInvalidRows, jobProperties, - combineText + combineText, + persistInHeap, + ingestOffheap ); } @@ -195,7 +216,9 @@ public class HadoopTuningConfig implements TuningConfig overwriteFiles, ignoreInvalidRows, jobProperties, - combineText + combineText, + persistInHeap, + ingestOffheap ); } @@ -212,7 +235,9 @@ public class HadoopTuningConfig implements TuningConfig overwriteFiles, ignoreInvalidRows, jobProperties, - combineText + combineText, + persistInHeap, + ingestOffheap ); } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index e562f7f45d6..df3bdcaecce 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -35,13 +35,17 @@ import com.metamx.common.guava.CloseQuietly; import com.metamx.common.logger.Logger; import io.druid.data.input.InputRow; import io.druid.data.input.impl.StringInputRowParser; +import io.druid.offheap.OffheapBufferPool; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMaker; +import io.druid.segment.LoggingProgressIndicator; +import io.druid.segment.ProgressIndicator; import io.druid.segment.QueryableIndex; import io.druid.segment.SegmentUtils; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.segment.incremental.OffheapIncrementalIndex; import io.druid.timeline.DataSegment; import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configurable; @@ -86,20 +90,9 @@ import java.util.zip.ZipOutputStream; public class IndexGeneratorJob implements Jobby { private static final Logger log = new Logger(IndexGeneratorJob.class); - private final HadoopDruidIndexerConfig config; - private IndexGeneratorStats jobStats; - - public IndexGeneratorJob( - HadoopDruidIndexerConfig config - ) - { - this.config = config; - this.jobStats = new IndexGeneratorStats(); - } public static List getPublishedSegments(HadoopDruidIndexerConfig config) { - final Configuration conf = new Configuration(); final ObjectMapper jsonMapper = HadoopDruidIndexerConfig.jsonMapper; @@ -130,6 +123,22 @@ public class IndexGeneratorJob implements Jobby return publishedSegments; } + private final HadoopDruidIndexerConfig config; + private IndexGeneratorStats jobStats; + + public IndexGeneratorJob( + HadoopDruidIndexerConfig config + ) + { + this.config = config; + this.jobStats = new IndexGeneratorStats(); + } + + protected void setReducerClass(final Job job) + { + job.setReducerClass(IndexGeneratorReducer.class); + } + public IndexGeneratorStats getJobStats() { return jobStats; @@ -161,7 +170,7 @@ public class IndexGeneratorJob implements Jobby job.setNumReduceTasks(Iterables.size(config.getAllBuckets().get())); job.setPartitionerClass(IndexGeneratorPartitioner.class); - job.setReducerClass(IndexGeneratorReducer.class); + setReducerClass(job); job.setOutputKeyClass(BytesWritable.class); job.setOutputValueClass(Text.class); job.setOutputFormatClass(IndexGeneratorOutputFormat.class); @@ -190,7 +199,6 @@ public class IndexGeneratorJob implements Jobby } public static class IndexGeneratorMapper extends HadoopDruidIndexerMapper - { @Override protected void innerMap( @@ -256,6 +264,42 @@ public class IndexGeneratorJob implements Jobby private List metricNames = Lists.newArrayList(); private StringInputRowParser parser; + protected ProgressIndicator makeProgressIndicator(final Context context) + { + return new LoggingProgressIndicator("IndexGeneratorJob") + { + @Override + public void progress() + { + context.progress(); + } + }; + } + + protected File persist( + final IncrementalIndex index, + final Interval interval, + final File file, + final ProgressIndicator progressIndicator + ) throws IOException + { + return IndexMaker.persist( + index, interval, file, progressIndicator + ); + } + + protected File mergeQueryableIndex( + final List indexes, + final AggregatorFactory[] aggs, + final File file, + ProgressIndicator progressIndicator + ) throws IOException + { + return IndexMaker.mergeQueryableIndex( + indexes, aggs, file, progressIndicator + ); + } + @Override protected void setup(Context context) throws IOException, InterruptedException @@ -282,113 +326,84 @@ public class IndexGeneratorJob implements Jobby final AggregatorFactory[] aggs = config.getSchema().getDataSchema().getAggregators(); IncrementalIndex index = makeIncrementalIndex(bucket, aggs); + try { + File baseFlushFile = File.createTempFile("base", "flush"); + baseFlushFile.delete(); + baseFlushFile.mkdirs(); - File baseFlushFile = File.createTempFile("base", "flush"); - baseFlushFile.delete(); - baseFlushFile.mkdirs(); + Set toMerge = Sets.newTreeSet(); + int indexCount = 0; + int lineCount = 0; + int runningTotalLineCount = 0; + long startTime = System.currentTimeMillis(); - Set toMerge = Sets.newTreeSet(); - int indexCount = 0; - int lineCount = 0; - int runningTotalLineCount = 0; - long startTime = System.currentTimeMillis(); - - Set allDimensionNames = Sets.newHashSet(); - - for (final Text value : values) { - context.progress(); - final InputRow inputRow = index.getSpatialDimensionRowFormatter().formatRow(parser.parse(value.toString())); - allDimensionNames.addAll(inputRow.getDimensions()); - - int numRows = index.add(inputRow); - ++lineCount; - - if (numRows >= config.getSchema().getTuningConfig().getRowFlushBoundary()) { - log.info( - "%,d lines to %,d rows in %,d millis", - lineCount - runningTotalLineCount, - numRows, - System.currentTimeMillis() - startTime - ); - runningTotalLineCount = lineCount; - - final File file = new File(baseFlushFile, String.format("index%,05d", indexCount)); - toMerge.add(file); + Set allDimensionNames = Sets.newHashSet(); + final ProgressIndicator progressIndicator = makeProgressIndicator(context); + for (final Text value : values) { context.progress(); - IndexMerger.persist( - index, interval, file, new IndexMerger.ProgressIndicator() - { - @Override - public void progress() - { - context.progress(); - } - } - ); - index = makeIncrementalIndex(bucket, aggs); + final InputRow inputRow = index.formatRow(parser.parse(value.toString())); + allDimensionNames.addAll(inputRow.getDimensions()); - startTime = System.currentTimeMillis(); - ++indexCount; - } - } + int numRows = index.add(inputRow); + ++lineCount; - log.info("%,d lines completed.", lineCount); + if (numRows >= config.getSchema().getTuningConfig().getRowFlushBoundary()) { + log.info( + "%,d lines to %,d rows in %,d millis", + lineCount - runningTotalLineCount, + numRows, + System.currentTimeMillis() - startTime + ); + runningTotalLineCount = lineCount; - List indexes = Lists.newArrayListWithCapacity(indexCount); - final File mergedBase; + final File file = new File(baseFlushFile, String.format("index%,05d", indexCount)); + toMerge.add(file); - if (toMerge.size() == 0) { - if (index.isEmpty()) { - throw new IAE("If you try to persist empty indexes you are going to have a bad time"); - } - - mergedBase = new File(baseFlushFile, "merged"); - IndexMerger.persist( - index, interval, mergedBase, new IndexMerger.ProgressIndicator() - { - @Override - public void progress() - { context.progress(); + persist(index, interval, file, progressIndicator); + // close this index and make a new one + index.close(); + index = makeIncrementalIndex(bucket, aggs); + + startTime = System.currentTimeMillis(); + ++indexCount; } } - ); - } else { - if (!index.isEmpty()) { - final File finalFile = new File(baseFlushFile, "final"); - IndexMerger.persist( - index, interval, finalFile, new IndexMerger.ProgressIndicator() - { - @Override - public void progress() - { - context.progress(); - } - } - ); - toMerge.add(finalFile); - } + log.info("%,d lines completed.", lineCount); + + List indexes = Lists.newArrayListWithCapacity(indexCount); + final File mergedBase; + + if (toMerge.size() == 0) { + if (index.isEmpty()) { + throw new IAE("If you try to persist empty indexes you are going to have a bad time"); + } + + mergedBase = new File(baseFlushFile, "merged"); + persist(index, interval, mergedBase, progressIndicator); + } else { + if (!index.isEmpty()) { + final File finalFile = new File(baseFlushFile, "final"); + persist(index, interval, finalFile, progressIndicator); + toMerge.add(finalFile); + } + + for (File file : toMerge) { + indexes.add(IndexIO.loadIndex(file)); + } + mergedBase = mergeQueryableIndex( + indexes, aggs, new File(baseFlushFile, "merged"), progressIndicator + ); + } + serializeOutIndex(context, bucket, mergedBase, Lists.newArrayList(allDimensionNames)); for (File file : toMerge) { - indexes.add(IndexIO.loadIndex(file)); + FileUtils.deleteDirectory(file); } - mergedBase = IndexMerger.mergeQueryableIndex( - indexes, aggs, new File(baseFlushFile, "merged"), new IndexMerger.ProgressIndicator() - { - @Override - public void progress() - { - context.progress(); - } - } - ); } - - serializeOutIndex(context, bucket, mergedBase, Lists.newArrayList(allDimensionNames)); - - for (File file : toMerge) { - FileUtils.deleteDirectory(file); + finally { + index.close(); } } @@ -616,14 +631,29 @@ public class IndexGeneratorJob implements Jobby private IncrementalIndex makeIncrementalIndex(Bucket theBucket, AggregatorFactory[] aggs) { - return new IncrementalIndex( - new IncrementalIndexSchema.Builder() - .withMinTimestamp(theBucket.time.getMillis()) - .withSpatialDimensions(config.getSchema().getDataSchema().getParser()) - .withQueryGranularity(config.getSchema().getDataSchema().getGranularitySpec().getQueryGranularity()) - .withMetrics(aggs) - .build() - ); + int aggsSize = 0; + for (AggregatorFactory agg : aggs) { + aggsSize += agg.getMaxIntermediateSize(); + } + final HadoopTuningConfig tuningConfig = config.getSchema().getTuningConfig(); + int bufferSize = aggsSize * tuningConfig.getRowFlushBoundary(); + final IncrementalIndexSchema indexSchema = new IncrementalIndexSchema.Builder() + .withMinTimestamp(theBucket.time.getMillis()) + .withDimensionsSpec(config.getSchema().getDataSchema().getParser()) + .withQueryGranularity(config.getSchema().getDataSchema().getGranularitySpec().getQueryGranularity()) + .withMetrics(aggs) + .build(); + if (tuningConfig.isIngestOffheap()) { + return new OffheapIncrementalIndex( + indexSchema, + new OffheapBufferPool(bufferSize) + ); + } else { + return new IncrementalIndex( + indexSchema, + new OffheapBufferPool(bufferSize) + ); + } } private void createNewZipEntry(ZipOutputStream out, String name) throws IOException diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/LegacyIndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/LegacyIndexGeneratorJob.java new file mode 100644 index 00000000000..876492ba8f2 --- /dev/null +++ b/indexing-hadoop/src/main/java/io/druid/indexer/LegacyIndexGeneratorJob.java @@ -0,0 +1,86 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.indexer; + +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.BaseProgressIndicator; +import io.druid.segment.IndexMerger; +import io.druid.segment.ProgressIndicator; +import io.druid.segment.QueryableIndex; +import io.druid.segment.incremental.IncrementalIndex; +import org.apache.hadoop.mapreduce.Job; +import org.joda.time.Interval; + +import java.io.File; +import java.io.IOException; +import java.util.List; + +/** + */ +public class LegacyIndexGeneratorJob extends IndexGeneratorJob +{ + public LegacyIndexGeneratorJob( + HadoopDruidIndexerConfig config + ) + { + super(config); + } + + @Override + protected void setReducerClass(Job job) + { + job.setReducerClass(LegacyIndexGeneratorReducer.class); + } + + public static class LegacyIndexGeneratorReducer extends IndexGeneratorJob.IndexGeneratorReducer + { + @Override + protected ProgressIndicator makeProgressIndicator(final Context context) + { + return new BaseProgressIndicator() + { + @Override + public void progress() + { + context.progress(); + } + }; + } + + @Override + protected File persist( + IncrementalIndex index, Interval interval, File file, ProgressIndicator progressIndicator + ) throws IOException + { + return IndexMerger.persist(index, interval, file, progressIndicator); + } + + @Override + protected File mergeQueryableIndex( + List indexes, + AggregatorFactory[] aggs, + File file, + ProgressIndicator progressIndicator + ) throws IOException + { + return IndexMerger.mergeQueryableIndex(indexes, aggs, file, progressIndicator); + } + } +} diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/rollup/DataRollupSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/rollup/DataRollupSpec.java index 86d04f9276f..3f0fd29736e 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/rollup/DataRollupSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/rollup/DataRollupSpec.java @@ -20,6 +20,7 @@ package io.druid.indexer.rollup; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.api.client.util.Lists; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; @@ -30,32 +31,17 @@ import java.util.List; * * Adjust to JsonCreator and final fields when resolved. */ +@Deprecated public class DataRollupSpec { @JsonProperty - public List aggs; + public List aggs = Lists.newArrayList(); @JsonProperty public QueryGranularity rollupGranularity = QueryGranularity.NONE; - @JsonProperty - public int rowFlushBoundary = 500000; - - public DataRollupSpec() {} - - public DataRollupSpec(List aggs, QueryGranularity rollupGranularity) - { - this.aggs = aggs; - this.rollupGranularity = rollupGranularity; - } - public List getAggs() { return aggs; } - - public QueryGranularity getRollupGranularity() - { - return rollupGranularity; - } } diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java index 1fe6068f2ad..9149a127fda 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -152,6 +152,10 @@ public class HadoopDruidIndexerConfigTest for (int i = 0; i < partitionCount; i++) { specs.add(new HadoopyShardSpec(new HashBasedNumberedShardSpec(i, partitionCount, new DefaultObjectMapper()), i)); } + // Backwards compatibility + DataRollupSpec rollupSpec = new DataRollupSpec(); + rollupSpec.rollupGranularity = QueryGranularity.MINUTE; + HadoopIngestionSpec spec = new HadoopIngestionSpec( null, null, null, "foo", @@ -172,7 +176,7 @@ public class HadoopDruidIndexerConfigTest true, ImmutableMap.of(new DateTime("2010-01-01T01:00:00"), specs), false, - new DataRollupSpec(ImmutableList.of(), QueryGranularity.MINUTE), + rollupSpec, null, false, ImmutableMap.of("foo", "bar"), diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 653a2f6c033..81260ba3295 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.161-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java index 11f6bb2264d..0f7859f72a6 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java @@ -35,7 +35,7 @@ import io.druid.data.input.InputRow; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMaker; import io.druid.segment.QueryableIndex; import io.druid.segment.SegmentUtils; import io.druid.segment.indexing.DataSchema; @@ -166,7 +166,7 @@ public class YeOldePlumberSchool implements PlumberSchool } fileToUpload = new File(tmpSegmentDir, "merged"); - IndexMerger.mergeQueryableIndex(indexes, schema.getAggregators(), fileToUpload); + IndexMaker.mergeQueryableIndex(indexes, schema.getAggregators(), fileToUpload); } // Map merged segment so we can extract dimensions @@ -211,8 +211,7 @@ public class YeOldePlumberSchool implements PlumberSchool log.info("Spilling index[%d] with rows[%d] to: %s", indexToPersist.getCount(), rowsToPersist, dirToPersist); try { - - IndexMerger.persist( + IndexMaker.persist( indexToPersist.getIndex(), dirToPersist ); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java index 51fb358ac8c..5b6bd37db16 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java @@ -27,7 +27,7 @@ import com.google.common.base.Predicate; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMaker; import io.druid.segment.IndexableAdapter; import io.druid.segment.QueryableIndexIndexableAdapter; import io.druid.segment.Rowboat; @@ -106,7 +106,7 @@ public class AppendTask extends MergeTaskBase ); } - return IndexMerger.append(adapters, outDir); + return IndexMaker.append(adapters, outDir); } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/DeleteTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/DeleteTask.java deleted file mode 100644 index 39d443d1330..00000000000 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/DeleteTask.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012, 2013 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package io.druid.indexing.common.task; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.metamx.common.logger.Logger; -import io.druid.granularity.QueryGranularity; -import io.druid.indexing.common.TaskLock; -import io.druid.indexing.common.TaskStatus; -import io.druid.indexing.common.TaskToolbox; -import io.druid.query.aggregation.AggregatorFactory; -import io.druid.segment.IndexMerger; -import io.druid.segment.IndexableAdapter; -import io.druid.segment.incremental.IncrementalIndex; -import io.druid.segment.incremental.IncrementalIndexAdapter; -import io.druid.timeline.DataSegment; -import io.druid.timeline.partition.NoneShardSpec; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -import java.io.File; - -public class DeleteTask extends AbstractFixedIntervalTask -{ - private static final Logger log = new Logger(DeleteTask.class); - - @JsonCreator - public DeleteTask( - @JsonProperty("id") String id, - @JsonProperty("dataSource") String dataSource, - @JsonProperty("interval") Interval interval - ) - { - super( - id != null ? id : String.format( - "delete_%s_%s_%s_%s", - dataSource, - interval.getStart(), - interval.getEnd(), - new DateTime().toString() - ), - dataSource, - Preconditions.checkNotNull(interval, "interval") - ); - } - - @Override - public String getType() - { - return "delete"; - } - - @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception - { - // Strategy: Create an empty segment covering the interval to be deleted - final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox)); - final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]); - final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(getInterval(), empty); - - // Create DataSegment - final DataSegment segment = - DataSegment.builder() - .dataSource(this.getDataSource()) - .interval(getInterval()) - .version(myLock.getVersion()) - .shardSpec(new NoneShardSpec()) - .build(); - - final File outDir = new File(toolbox.getTaskWorkDir(), segment.getIdentifier()); - final File fileToUpload = IndexMerger.merge(Lists.newArrayList(emptyAdapter), new AggregatorFactory[0], outDir); - - // Upload the segment - final DataSegment uploadedSegment = toolbox.getSegmentPusher().push(fileToUpload, segment); - - log.info( - "Uploaded tombstone segment for[%s] interval[%s] with version[%s]", - segment.getDataSource(), - segment.getInterval(), - segment.getVersion() - ); - - toolbox.pushSegments(ImmutableList.of(uploadedSegment)); - - return TaskStatus.success(getId()); - } -} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index f04736a66e3..d1495e6b6d2 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -147,7 +147,7 @@ public class IndexTask extends AbstractFixedIntervalTask granularitySpec.withQueryGranularity(indexGranularity == null ? QueryGranularity.NONE : indexGranularity) ), new IndexIOConfig(firehoseFactory), - new IndexTuningConfig(targetPartitionSize, rowFlushBoundary, null) + new IndexTuningConfig(targetPartitionSize, 0, null) ); } this.jsonMapper = jsonMapper; @@ -401,7 +401,11 @@ public class IndexTask extends AbstractFixedIntervalTask version, wrappedDataSegmentPusher, tmpDir - ).findPlumber(schema, new RealtimeTuningConfig(null, null, null, null, null, null, null, shardSpec), metrics); + ).findPlumber( + schema, + new RealtimeTuningConfig(null, null, null, null, null, null, null, shardSpec, null, null), + metrics + ); // rowFlushBoundary for this job final int myRowFlushBoundary = rowFlushBoundary > 0 @@ -557,7 +561,7 @@ public class IndexTask extends AbstractFixedIntervalTask @JsonProperty("targetPartitionSize") int targetPartitionSize, @JsonProperty("rowFlushBoundary") int rowFlushBoundary, @JsonProperty("numShards") @Nullable Integer numShards - ) + ) { this.targetPartitionSize = targetPartitionSize == 0 ? DEFAULT_TARGET_PARTITION_SIZE : targetPartitionSize; this.rowFlushBoundary = rowFlushBoundary == 0 ? DEFAULT_ROW_FLUSH_BOUNDARY : rowFlushBoundary; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java index 9bca4a3eee5..d856d1505bc 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java @@ -28,7 +28,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMaker; import io.druid.segment.QueryableIndex; import io.druid.timeline.DataSegment; @@ -60,7 +60,7 @@ public class MergeTask extends MergeTaskBase public File merge(final Map segments, final File outDir) throws Exception { - return IndexMerger.mergeQueryableIndex( + return IndexMaker.mergeQueryableIndex( Lists.transform( ImmutableList.copyOf(segments.values()), new Function() diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index 231febf66df..906e8e7a901 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -143,7 +143,9 @@ public class RealtimeIndexTask extends AbstractTask null, rejectionPolicy == null ? rejectionPolicyFactory : rejectionPolicy, maxPendingPersists, - spec.getShardSpec() + spec.getShardSpec(), + false, + false ), null, null, null, null ); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java index e5db129128c..a4c093179b3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java @@ -43,7 +43,6 @@ import io.druid.query.QueryRunner; @JsonSubTypes(value = { @JsonSubTypes.Type(name = "append", value = AppendTask.class), @JsonSubTypes.Type(name = "merge", value = MergeTask.class), - @JsonSubTypes.Type(name = "delete", value = DeleteTask.class), @JsonSubTypes.Type(name = "kill", value = KillTask.class), @JsonSubTypes.Type(name = "move", value = MoveTask.class), @JsonSubTypes.Type(name = "archive", value = ArchiveTask.class), diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index 5744fa7f006..f0dacac6fa8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -51,10 +51,11 @@ import io.druid.query.select.EventHolder; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; import io.druid.segment.IndexIO; +import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.QueryableIndexStorageAdapter; import io.druid.segment.StorageAdapter; -import io.druid.segment.TimestampColumnSelector; +import io.druid.segment.column.Column; import io.druid.segment.data.IndexedInts; import io.druid.segment.filter.Filters; import io.druid.segment.loading.SegmentLoadingException; @@ -250,7 +251,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory apply(final Cursor cursor) { - final TimestampColumnSelector timestampColumnSelector = cursor.makeTimestampColumnSelector(); + final LongColumnSelector timestampColumnSelector = cursor.makeLongColumnSelector(Column.TIME_COLUMN_NAME); final Map dimSelectors = Maps.newHashMap(); for (String dim : dims) { @@ -287,7 +288,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory theEvent = Maps.newLinkedHashMap(); - final long timestamp = timestampColumnSelector.getTimestamp(); + final long timestamp = timestampColumnSelector.get(); theEvent.put(EventHolder.timestampKey, new DateTime(timestamp)); for (Map.Entry dimSelector : dimSelectors.entrySet()) { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/config/ForkingTaskRunnerConfig.java b/indexing-service/src/main/java/io/druid/indexing/overlord/config/ForkingTaskRunnerConfig.java index 8d89d834785..8807debcb3d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/config/ForkingTaskRunnerConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/config/ForkingTaskRunnerConfig.java @@ -49,7 +49,7 @@ public class ForkingTaskRunnerConfig @JsonProperty @Min(1024) @Max(65535) - private int startPort = 8081; + private int startPort = 8100; @JsonProperty @NotNull diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index f3a6e9e1d1f..ed4769644d1 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -243,51 +243,6 @@ public class TaskSerdeTest ); } - @Test - public void testDeleteTaskSerde() throws Exception - { - final DeleteTask task = new DeleteTask( - null, - "foo", - new Interval("2010-01-01/P1D") - ); - - final String json = jsonMapper.writeValueAsString(task); - - Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change - final DeleteTask task2 = (DeleteTask) jsonMapper.readValue(json, Task.class); - - Assert.assertEquals("foo", task.getDataSource()); - Assert.assertEquals(new Interval("2010-01-01/P1D"), task.getInterval()); - - Assert.assertEquals(task.getId(), task2.getId()); - Assert.assertEquals(task.getGroupId(), task2.getGroupId()); - Assert.assertEquals(task.getDataSource(), task2.getDataSource()); - Assert.assertEquals(task.getInterval(), task2.getInterval()); - } - - @Test - public void testDeleteTaskFromJson() throws Exception - { - final DeleteTask task = (DeleteTask) jsonMapper.readValue( - "{\"type\":\"delete\",\"dataSource\":\"foo\",\"interval\":\"2010-01-01/P1D\"}", - Task.class - ); - final String json = jsonMapper.writeValueAsString(task); - - Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change - final DeleteTask task2 = (DeleteTask) jsonMapper.readValue(json, Task.class); - - Assert.assertNotNull(task.getId()); - Assert.assertEquals("foo", task.getDataSource()); - Assert.assertEquals(new Interval("2010-01-01/P1D"), task.getInterval()); - - Assert.assertEquals(task.getId(), task2.getId()); - Assert.assertEquals(task.getGroupId(), task2.getGroupId()); - Assert.assertEquals(task.getDataSource(), task2.getDataSource()); - Assert.assertEquals(task.getInterval(), task2.getInterval()); - } - @Test public void testAppendTaskSerde() throws Exception { @@ -413,7 +368,7 @@ public class TaskSerdeTest true, null, false, - new DataRollupSpec(ImmutableList.of(), QueryGranularity.NONE), + null, null, false, ImmutableMap.of("foo", "bar"), diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 79e28c016ec..e0c5fb43ebe 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.161-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 21acf89e6a4..a5e6ef4b806 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.161-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/pom.xml b/pom.xml index e5baf549052..1ca0ace022d 100644 --- a/pom.xml +++ b/pom.xml @@ -18,19 +18,20 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid pom - 0.6.161-SNAPSHOT + 0.7.0-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.159-SNAPSHOT + druid-0.7.0-SNAPSHOT @@ -41,7 +42,7 @@ UTF-8 0.26.9 2.6.0 - 0.2.15 + 0.2.16 @@ -74,7 +75,7 @@ com.metamx emitter - 0.2.11 + 0.2.12 com.metamx @@ -389,7 +390,7 @@ net.jpountz.lz4 lz4 - 1.1.2 + 1.2.0 com.google.protobuf @@ -434,6 +435,11 @@ 2.3.0 provided + + org.mapdb + mapdb + 1.0.6 + diff --git a/processing/pom.xml b/processing/pom.xml index 0b562f2a2fa..2f60f53d2bc 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-processing @@ -28,7 +29,7 @@ io.druid druid - 0.6.161-SNAPSHOT + 0.7.0-SNAPSHOT @@ -82,6 +83,14 @@ com.davekoelle alphanum + + net.jpountz.lz4 + lz4 + + + org.mapdb + mapdb + @@ -95,10 +104,10 @@ easymock test - - com.google.caliper - caliper - + + com.google.caliper + caliper + diff --git a/processing/src/main/java/io/druid/guice/GuiceInjectors.java b/processing/src/main/java/io/druid/guice/GuiceInjectors.java index d979938d5d7..2628e2feee4 100644 --- a/processing/src/main/java/io/druid/guice/GuiceInjectors.java +++ b/processing/src/main/java/io/druid/guice/GuiceInjectors.java @@ -26,6 +26,7 @@ import com.google.inject.Injector; import com.google.inject.Module; import io.druid.jackson.JacksonModule; +import java.util.Arrays; import java.util.List; /** @@ -37,7 +38,7 @@ public class GuiceInjectors return Guice.createInjector( new DruidGuiceExtensions(), new JacksonModule(), - new PropertiesModule("runtime.properties"), + new PropertiesModule(Arrays.asList("global.runtime.properties", "runtime.properties")), new ConfigModule(), new Module() { @@ -56,7 +57,7 @@ public class GuiceInjectors List theModules = Lists.newArrayList(); theModules.add(new DruidGuiceExtensions()); theModules.add(new JacksonModule()); - theModules.add(new PropertiesModule("runtime.properties")); + theModules.add(new PropertiesModule(Arrays.asList("global.runtime.properties", "runtime.properties"))); theModules.add(new ConfigModule()); theModules.add( new Module() diff --git a/processing/src/main/java/io/druid/guice/PropertiesModule.java b/processing/src/main/java/io/druid/guice/PropertiesModule.java index 2314134894b..cc766576450 100644 --- a/processing/src/main/java/io/druid/guice/PropertiesModule.java +++ b/processing/src/main/java/io/druid/guice/PropertiesModule.java @@ -17,10 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.guice; +package io.druid.guice;; import com.google.common.base.Charsets; import com.google.common.base.Throwables; +import com.google.common.io.Closeables; import com.google.inject.Binder; import com.google.inject.Module; import com.metamx.common.guava.CloseQuietly; @@ -33,7 +34,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; -import java.io.Reader; +import java.util.List; import java.util.Properties; /** @@ -42,11 +43,11 @@ public class PropertiesModule implements Module { private static final Logger log = new Logger(PropertiesModule.class); - private final String propertiesFile; + private final List propertiesFiles; - public PropertiesModule(String propertiesFile) + public PropertiesModule(List propertiesFiles) { - this.propertiesFile = propertiesFile; + this.propertiesFiles = propertiesFiles; } @Override @@ -58,30 +59,32 @@ public class PropertiesModule implements Module Properties props = new Properties(fileProps); props.putAll(systemProps); - InputStream stream = ClassLoader.getSystemResourceAsStream(propertiesFile); - try { - if (stream == null) { - File workingDirectoryFile = new File(systemProps.getProperty("druid.properties.file", propertiesFile)); - if (workingDirectoryFile.exists()) { - stream = new BufferedInputStream(new FileInputStream(workingDirectoryFile)); + for (String propertiesFile : propertiesFiles) { + InputStream stream = ClassLoader.getSystemResourceAsStream(propertiesFile); + try { + if (stream == null) { + File workingDirectoryFile = new File(systemProps.getProperty("druid.properties.file", propertiesFile)); + if (workingDirectoryFile.exists()) { + stream = new BufferedInputStream(new FileInputStream(workingDirectoryFile)); + } } - } - if (stream != null) { - log.info("Loading properties from %s", propertiesFile); - try(Reader reader = new InputStreamReader(stream, Charsets.UTF_8)) { - fileProps.load(reader); - } - catch (IOException e) { - throw Throwables.propagate(e); + if (stream != null) { + log.info("Loading properties from %s", propertiesFile); + try { + fileProps.load(new InputStreamReader(stream, Charsets.UTF_8)); + } + catch (IOException e) { + throw Throwables.propagate(e); + } } } - } - catch (FileNotFoundException e) { - log.wtf(e, "This can only happen if the .exists() call lied. That's f'd up."); - } - finally { - CloseQuietly.close(stream); + catch (FileNotFoundException e) { + log.wtf(e, "This can only happen if the .exists() call lied. That's f'd up."); + } + finally { + CloseQuietly.close(stream); + } } binder.bind(Properties.class).toInstance(props); diff --git a/processing/src/main/java/io/druid/query/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java index a648f484c78..e6191fee6b0 100644 --- a/processing/src/main/java/io/druid/query/BaseQuery.java +++ b/processing/src/main/java/io/druid/query/BaseQuery.java @@ -70,14 +70,14 @@ public abstract class BaseQuery implements Query } @Override - public Sequence run(QuerySegmentWalker walker) + public Sequence run(QuerySegmentWalker walker, Map context) { - return run(querySegmentSpec.lookup(this, walker)); + return run(querySegmentSpec.lookup(this, walker), context); } - public Sequence run(QueryRunner runner) + public Sequence run(QueryRunner runner, Map context) { - return runner.run(this); + return runner.run(this, context); } @Override diff --git a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java index 02bb5232e18..c1cf7886bfc 100644 --- a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java @@ -26,6 +26,7 @@ import org.joda.time.DateTime; import java.util.Arrays; import java.util.List; +import java.util.Map; /** */ @@ -48,11 +49,11 @@ public class BySegmentQueryRunner implements QueryRunner @Override @SuppressWarnings("unchecked") - public Sequence run(final Query query) + public Sequence run(final Query query, Map context) { if (query.getContextBySegment(false)) { - final Sequence baseSequence = base.run(query); + final Sequence baseSequence = base.run(query, context); final List results = Sequences.toList(baseSequence, Lists.newArrayList()); return Sequences.simple( Arrays.asList( @@ -67,7 +68,6 @@ public class BySegmentQueryRunner implements QueryRunner ) ); } - - return base.run(query); + return base.run(query, context); } } diff --git a/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java b/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java index 13ca4dd75df..5f9651f5222 100644 --- a/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java @@ -21,6 +21,8 @@ package io.druid.query; import com.metamx.common.guava.Sequence; +import java.util.Map; + /** */ public abstract class BySegmentSkippingQueryRunner implements QueryRunner @@ -35,14 +37,14 @@ public abstract class BySegmentSkippingQueryRunner implements QueryRunner } @Override - public Sequence run(Query query) + public Sequence run(Query query, Map context) { if (query.getContextBySegment(false)) { - return baseRunner.run(query); + return baseRunner.run(query, context); } - return doRun(baseRunner, query); + return doRun(baseRunner, query, context); } - protected abstract Sequence doRun(QueryRunner baseRunner, Query query); + protected abstract Sequence doRun(QueryRunner baseRunner, Query query, Map context); } diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index 6b474799de0..e5edfd3d4cf 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -39,6 +39,7 @@ import com.metamx.common.logger.Logger; import java.util.Arrays; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -93,7 +94,7 @@ public class ChainedExecutionQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query) + public Sequence run(final Query query, final Map context) { final int priority = query.getContextPriority(0); @@ -124,7 +125,11 @@ public class ChainedExecutionQueryRunner implements QueryRunner public Iterable call() throws Exception { try { - Sequence result = input.run(query); + if (input == null) { + throw new ISE("Input is null?! How is this possible?!"); + } + + Sequence result = input.run(query, context); if (result == null) { throw new ISE("Got a null result! Segments are missing!"); } diff --git a/processing/src/main/java/io/druid/query/ConcatQueryRunner.java b/processing/src/main/java/io/druid/query/ConcatQueryRunner.java index 9735d0f5a94..74c4a6481f5 100644 --- a/processing/src/main/java/io/druid/query/ConcatQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ConcatQueryRunner.java @@ -23,6 +23,9 @@ import com.google.common.base.Function; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import java.util.List; +import java.util.Map; + /** */ public class ConcatQueryRunner implements QueryRunner @@ -36,7 +39,7 @@ public class ConcatQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query) + public Sequence run(final Query query, final Map context) { return Sequences.concat( Sequences.map( @@ -46,7 +49,7 @@ public class ConcatQueryRunner implements QueryRunner @Override public Sequence apply(final QueryRunner input) { - return input.run(query); + return input.run(query, context); } } ) diff --git a/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java index 80c0f2d351b..10bff0a65e2 100644 --- a/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java +++ b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java @@ -28,6 +28,10 @@ import com.metamx.common.guava.Sequences; import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.aggregation.MetricManipulatorFns; +import javax.annotation.Nullable; +import java.util.List; +import java.util.Map; + /** */ public class FinalizeResultsQueryRunner implements QueryRunner @@ -45,7 +49,7 @@ public class FinalizeResultsQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query) + public Sequence run(final Query query, Map context) { final boolean isBySegment = query.getContextBySegment(false); final boolean shouldFinalize = query.getContextFinalize(true); @@ -98,7 +102,7 @@ public class FinalizeResultsQueryRunner implements QueryRunner return Sequences.map( - baseRunner.run(queryToRun), + baseRunner.run(queryToRun, context), finalizerFn ); diff --git a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java index 7e97ce03099..5997518dddd 100644 --- a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java @@ -32,16 +32,20 @@ import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.common.guava.Accumulator; +import com.metamx.common.guava.ResourceClosingSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.logger.Logger; +import io.druid.collections.StupidPool; import io.druid.data.input.Row; import io.druid.query.groupby.GroupByQuery; import io.druid.query.groupby.GroupByQueryConfig; import io.druid.query.groupby.GroupByQueryHelper; import io.druid.segment.incremental.IncrementalIndex; +import java.nio.ByteBuffer; import java.util.List; +import java.util.Map; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -56,11 +60,13 @@ public class GroupByParallelQueryRunner implements QueryRunner private final ListeningExecutorService exec; private final Supplier configSupplier; private final QueryWatcher queryWatcher; + private final StupidPool bufferPool; public GroupByParallelQueryRunner( ExecutorService exec, Supplier configSupplier, QueryWatcher queryWatcher, + StupidPool bufferPool, Iterable> queryables ) { @@ -68,15 +74,17 @@ public class GroupByParallelQueryRunner implements QueryRunner this.queryWatcher = queryWatcher; this.queryables = Iterables.unmodifiableIterable(Iterables.filter(queryables, Predicates.notNull())); this.configSupplier = configSupplier; + this.bufferPool = bufferPool; } @Override - public Sequence run(final Query queryParam) + public Sequence run(final Query queryParam, final Map context) { final GroupByQuery query = (GroupByQuery) queryParam; final Pair> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair( query, - configSupplier.get() + configSupplier.get(), + bufferPool ); final Pair> bySegmentAccumulatorPair = GroupByQueryHelper.createBySegmentAccumulatorPair(); final boolean bySegment = query.getContextBySegment(false); @@ -103,10 +111,11 @@ public class GroupByParallelQueryRunner implements QueryRunner { try { if (bySegment) { - input.run(queryParam) + input.run(queryParam, context) .accumulate(bySegmentAccumulatorPair.lhs, bySegmentAccumulatorPair.rhs); } else { - input.run(queryParam).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); + input.run(queryParam, context) + .accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); } return null; @@ -140,17 +149,21 @@ public class GroupByParallelQueryRunner implements QueryRunner catch (InterruptedException e) { log.warn(e, "Query interrupted, cancelling pending results, query id [%s]", query.getId()); futures.cancel(true); + indexAccumulatorPair.lhs.close(); throw new QueryInterruptedException("Query interrupted"); } catch (CancellationException e) { + indexAccumulatorPair.lhs.close(); throw new QueryInterruptedException("Query cancelled"); } catch (TimeoutException e) { + indexAccumulatorPair.lhs.close(); log.info("Query timeout, cancelling pending results for query id [%s]", query.getId()); futures.cancel(true); throw new QueryInterruptedException("Query timeout"); } catch (ExecutionException e) { + indexAccumulatorPair.lhs.close(); throw Throwables.propagate(e.getCause()); } @@ -158,18 +171,20 @@ public class GroupByParallelQueryRunner implements QueryRunner return Sequences.simple(bySegmentAccumulatorPair.lhs); } - return Sequences.simple( - Iterables.transform( - indexAccumulatorPair.lhs.iterableWithPostAggregations(null), - new Function() - { - @Override - public T apply(Row input) - { - return (T) input; - } - } - ) + return new ResourceClosingSequence( + Sequences.simple( + Iterables.transform( + indexAccumulatorPair.lhs.iterableWithPostAggregations(null), + new Function() + { + @Override + public T apply(Row input) + { + return (T) input; + } + } + ) + ), indexAccumulatorPair.lhs ); } } diff --git a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java index df4c8596217..557420aa377 100644 --- a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java @@ -33,6 +33,7 @@ import javax.annotation.Nullable; import java.util.Arrays; import java.util.Iterator; import java.util.List; +import java.util.Map; /** */ @@ -48,10 +49,10 @@ public class IntervalChunkingQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query) + public Sequence run(final Query query, final Map context) { if (period.getMillis() == 0) { - return baseRunner.run(query); + return baseRunner.run(query, context); } return Sequences.concat( @@ -74,7 +75,8 @@ public class IntervalChunkingQueryRunner implements QueryRunner public Sequence apply(Interval singleInterval) { return baseRunner.run( - query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(singleInterval))) + query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(singleInterval))), + context ); } } diff --git a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java index 0d1d39910fb..5a8005185a7 100644 --- a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java @@ -28,6 +28,8 @@ import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import java.io.IOException; +import java.util.List; +import java.util.Map; /** */ @@ -82,7 +84,7 @@ public class MetricsEmittingQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query) + public Sequence run(final Query query, final Map context) { final ServiceMetricEvent.Builder builder = builderFn.apply(query); String queryId = query.getId(); @@ -100,7 +102,7 @@ public class MetricsEmittingQueryRunner implements QueryRunner long startTime = System.currentTimeMillis(); try { - retVal = queryRunner.run(query).accumulate(outType, accumulator); + retVal = queryRunner.run(query, context).accumulate(outType, accumulator); } catch (RuntimeException e) { builder.setUser10("failed"); @@ -130,7 +132,7 @@ public class MetricsEmittingQueryRunner implements QueryRunner long startTime = System.currentTimeMillis(); try { - retVal = queryRunner.run(query).toYielder(initValue, accumulator); + retVal = queryRunner.run(query, context).toYielder(initValue, accumulator); } catch (RuntimeException e) { builder.setUser10("failed"); diff --git a/processing/src/main/java/io/druid/query/NoopQueryRunner.java b/processing/src/main/java/io/druid/query/NoopQueryRunner.java index 355ee4f7a20..d2f3863ab62 100644 --- a/processing/src/main/java/io/druid/query/NoopQueryRunner.java +++ b/processing/src/main/java/io/druid/query/NoopQueryRunner.java @@ -22,12 +22,15 @@ package io.druid.query; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import java.util.List; +import java.util.Map; + /** */ public class NoopQueryRunner implements QueryRunner { @Override - public Sequence run(Query query) + public Sequence run(Query query, Map context) { return Sequences.empty(); } diff --git a/processing/src/main/java/io/druid/query/Query.java b/processing/src/main/java/io/druid/query/Query.java index 8c34be7ec28..c255fc46d60 100644 --- a/processing/src/main/java/io/druid/query/Query.java +++ b/processing/src/main/java/io/druid/query/Query.java @@ -62,9 +62,9 @@ public interface Query public String getType(); - public Sequence run(QuerySegmentWalker walker); + public Sequence run(QuerySegmentWalker walker, Map context); - public Sequence run(QueryRunner runner); + public Sequence run(QueryRunner runner, Map context); public List getIntervals(); diff --git a/processing/src/main/java/io/druid/query/QueryRunner.java b/processing/src/main/java/io/druid/query/QueryRunner.java index 62c44ad5163..d7a3f8af36f 100644 --- a/processing/src/main/java/io/druid/query/QueryRunner.java +++ b/processing/src/main/java/io/druid/query/QueryRunner.java @@ -21,9 +21,11 @@ package io.druid.query; import com.metamx.common.guava.Sequence; +import java.util.Map; + /** */ public interface QueryRunner { - public Sequence run(Query query); -} + public Sequence run(Query query, Map context); +} \ No newline at end of file diff --git a/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java index e801e5516ac..736c60f76ab 100644 --- a/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java @@ -25,6 +25,8 @@ import com.metamx.common.guava.Sequence; import io.druid.segment.ReferenceCountingSegment; import java.io.Closeable; +import java.util.List; +import java.util.Map; /** */ @@ -43,11 +45,11 @@ public class ReferenceCountingSegmentQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query) + public Sequence run(final Query query, Map context) { final Closeable closeable = adapter.increment(); try { - final Sequence baseSequence = factory.createRunner(adapter).run(query); + final Sequence baseSequence = factory.createRunner(adapter).run(query, context); return new ResourceClosingSequence(baseSequence, closeable); } diff --git a/processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java b/processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java index 94b670444e8..f5378cfc4ed 100644 --- a/processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java @@ -24,6 +24,9 @@ import com.metamx.common.guava.Sequence; import com.metamx.common.guava.nary.BinaryFn; import io.druid.common.guava.CombiningSequence; +import java.util.List; +import java.util.Map; + /** */ public abstract class ResultMergeQueryRunner extends BySegmentSkippingQueryRunner @@ -36,9 +39,9 @@ public abstract class ResultMergeQueryRunner extends BySegmentSkippingQueryRu } @Override - public Sequence doRun(QueryRunner baseRunner, Query query) + public Sequence doRun(QueryRunner baseRunner, Query query, Map context) { - return CombiningSequence.create(baseRunner.run(query), makeOrdering(query), createMergeFn(query)); + return CombiningSequence.create(baseRunner.run(query, context), makeOrdering(query), createMergeFn(query)); } protected abstract Ordering makeOrdering(Query query); diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunner.java b/processing/src/main/java/io/druid/query/RetryQueryRunner.java new file mode 100644 index 00000000000..9f3921bd87d --- /dev/null +++ b/processing/src/main/java/io/druid/query/RetryQueryRunner.java @@ -0,0 +1,113 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; +import com.google.common.collect.MapMaker; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Yielder; +import com.metamx.common.guava.YieldingAccumulator; +import com.metamx.common.guava.YieldingSequenceBase; +import io.druid.query.spec.MultipleSpecificSegmentSpec; +import io.druid.segment.SegmentMissingException; + +import java.util.List; +import java.util.Map; + +public class RetryQueryRunner implements QueryRunner +{ + public static String MISSING_SEGMENTS_KEY = "missingSegments"; + + private final QueryRunner baseRunner; + private final RetryQueryRunnerConfig config; + private final ObjectMapper jsonMapper; + + public RetryQueryRunner( + QueryRunner baseRunner, + RetryQueryRunnerConfig config, + ObjectMapper jsonMapper + ) + { + this.baseRunner = baseRunner; + this.config = config; + this.jsonMapper = jsonMapper; + } + + @Override + public Sequence run(final Query query, final Map context) + { + final Sequence returningSeq = baseRunner.run(query, context); + + return new YieldingSequenceBase() + { + @Override + public Yielder toYielder( + OutType initValue, YieldingAccumulator accumulator + ) + { + Yielder yielder = returningSeq.toYielder(initValue, accumulator); + + final List missingSegments = getMissingSegments(context); + + if (missingSegments.isEmpty()) { + return yielder; + } + + for (int i = 0; i < config.numTries(); i++) { + context.put(MISSING_SEGMENTS_KEY, Lists.newArrayList()); + final Query retryQuery = query.withQuerySegmentSpec( + new MultipleSpecificSegmentSpec( + missingSegments + ) + ); + yielder = baseRunner.run(retryQuery, context).toYielder(initValue, accumulator); + if (getMissingSegments(context).isEmpty()) { + break; + } + } + + final List finalMissingSegs = getMissingSegments(context); + if (!config.returnPartialResults() && !finalMissingSegs.isEmpty()) { + throw new SegmentMissingException("No results found for segments[%s]", finalMissingSegs); + } + + return yielder; + } + }; + } + + private List getMissingSegments(final Map context) + { + final Object maybeMissingSegments = context.get(MISSING_SEGMENTS_KEY); + if (maybeMissingSegments == null) { + return Lists.newArrayList(); + } + + return jsonMapper.convertValue( + maybeMissingSegments, + new TypeReference>() + { + } + ); + } +} + diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java new file mode 100644 index 00000000000..2b8bb730b68 --- /dev/null +++ b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java @@ -0,0 +1,33 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class RetryQueryRunnerConfig +{ + @JsonProperty + private int numTries = 0; + @JsonProperty + private boolean returnPartialResults = false; + + public int numTries() { return numTries; } + public boolean returnPartialResults() { return returnPartialResults; } +} diff --git a/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java b/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java index 8e13d9219e9..d16a660e25a 100644 --- a/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java @@ -23,6 +23,9 @@ package io.druid.query; import com.metamx.common.guava.Sequence; +import java.util.List; +import java.util.Map; + /** * If there's a subquery, run it instead of the outer query */ @@ -36,13 +39,13 @@ public class SubqueryQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query) + public Sequence run(final Query query, Map context) { DataSource dataSource = query.getDataSource(); if (dataSource instanceof QueryDataSource) { - return run((Query) ((QueryDataSource) dataSource).getQuery()); + return run((Query) ((QueryDataSource) dataSource).getQuery(), context); } else { - return baseRunner.run(query); + return baseRunner.run(query, context); } } } diff --git a/processing/src/main/java/io/druid/query/TimewarpOperator.java b/processing/src/main/java/io/druid/query/TimewarpOperator.java index c59908172d3..2f26f6890f1 100644 --- a/processing/src/main/java/io/druid/query/TimewarpOperator.java +++ b/processing/src/main/java/io/druid/query/TimewarpOperator.java @@ -33,6 +33,8 @@ import org.joda.time.Interval; import org.joda.time.Period; import java.util.Arrays; +import java.util.Map; + /** * TimewarpOperator is an example post-processing operator that maps current time @@ -79,7 +81,7 @@ public class TimewarpOperator implements PostProcessingOperator return new QueryRunner() { @Override - public Sequence run(final Query query) + public Sequence run(final Query query, final Map context) { final long offset = computeOffset(now); @@ -90,7 +92,8 @@ public class TimewarpOperator implements PostProcessingOperator ); return Sequences.map( baseRunner.run( - query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(modifiedInterval))) + query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(modifiedInterval))), + context ), new Function() { diff --git a/processing/src/main/java/io/druid/query/UnionQueryRunner.java b/processing/src/main/java/io/druid/query/UnionQueryRunner.java index f14bb180f62..2426bda9310 100644 --- a/processing/src/main/java/io/druid/query/UnionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/UnionQueryRunner.java @@ -26,6 +26,9 @@ import com.google.common.collect.Lists; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import java.util.List; +import java.util.Map; + public class UnionQueryRunner implements QueryRunner { private final QueryRunner baseRunner; @@ -41,7 +44,7 @@ public class UnionQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query) + public Sequence run(final Query query, final Map context) { DataSource dataSource = query.getDataSource(); if (dataSource instanceof UnionDataSource) { @@ -55,7 +58,8 @@ public class UnionQueryRunner implements QueryRunner public Sequence apply(DataSource singleSource) { return baseRunner.run( - query.withDataSource(singleSource) + query.withDataSource(singleSource), + context ); } } @@ -63,7 +67,7 @@ public class UnionQueryRunner implements QueryRunner ) ); } else { - return baseRunner.run(query); + return baseRunner.run(query, context); } } diff --git a/processing/src/main/java/io/druid/query/aggregation/Aggregators.java b/processing/src/main/java/io/druid/query/aggregation/Aggregators.java index bbfc1bc0dec..fb421e5d8d4 100644 --- a/processing/src/main/java/io/druid/query/aggregation/Aggregators.java +++ b/processing/src/main/java/io/druid/query/aggregation/Aggregators.java @@ -95,6 +95,13 @@ public class Aggregators return 0; } + + @Override + public long getLong(ByteBuffer buf, int position) + { + return 0L; + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java index da581ae3064..1078179eb0e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java @@ -89,6 +89,23 @@ public interface BufferAggregator */ float getFloat(ByteBuffer buf, int position); + /** + * Returns the long representation of the given aggregate byte array + * + * Converts the given byte buffer representation into the intermediate aggregate value. + * + * Implementations must not change the position, limit or mark of the given buffer + * + * Implementations are only required to support this method if they are aggregations which + * have an {@link AggregatorFactory#getTypeName()} of "long". + * If unimplemented, throwing an {@link UnsupportedOperationException} is common and recommended. + * + * @param buf byte buffer storing the byte array representation of the aggregate + * @param position offset within the byte buffer at which the aggregate value is stored + * @return the long representation of the aggregate + */ + long getLong(ByteBuffer buf, int position); + /** * Release any resources used by the aggregator */ diff --git a/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java index b4179f475f0..c18781980ad 100644 --- a/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java @@ -50,6 +50,13 @@ public class CountBufferAggregator implements BufferAggregator return buf.getLong(position); } + + @Override + public long getLong(ByteBuffer buf, int position) + { + return buf.getLong(position); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java index 7f6e449396d..3f12e11bb7f 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java @@ -60,6 +60,13 @@ public class DoubleSumBufferAggregator implements BufferAggregator return (float) buf.getDouble(position); } + + @Override + public long getLong(ByteBuffer buf, int position) + { + return (long) buf.getDouble(position); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java index 4d1eeda99ba..42f88138cf5 100644 --- a/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java @@ -19,12 +19,9 @@ package io.druid.query.aggregation; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterables; import io.druid.segment.DimensionSelector; import io.druid.segment.data.IndexedInts; -import javax.annotation.Nullable; import java.nio.ByteBuffer; public class FilteredBufferAggregator implements BufferAggregator @@ -65,6 +62,12 @@ public class FilteredBufferAggregator implements BufferAggregator return delegate.get(buf, position); } + @Override + public long getLong(ByteBuffer buf, int position) + { + return delegate.getLong(buf, position); + } + @Override public float getFloat(ByteBuffer buf, int position) { diff --git a/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java index 5913c32b711..3e8937d49e9 100644 --- a/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java @@ -90,6 +90,12 @@ public class HistogramBufferAggregator implements BufferAggregator throw new UnsupportedOperationException("HistogramBufferAggregator does not support getFloat()"); } + @Override + public long getLong(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("HistogramBufferAggregator does not support getLong()"); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java index ad723c5caa1..f6b8cc71fc7 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java @@ -63,6 +63,13 @@ public class JavaScriptBufferAggregator implements BufferAggregator return (float)buf.getDouble(position); } + + @Override + public long getLong(ByteBuffer buf, int position) + { + return (long) buf.getDouble(position); + } + @Override public void close() { script.close(); diff --git a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregator.java b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregator.java index 203b22f7860..d268afe185f 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregator.java @@ -21,6 +21,7 @@ package io.druid.query.aggregation; import com.google.common.primitives.Longs; import io.druid.segment.FloatColumnSelector; +import io.druid.segment.LongColumnSelector; import java.util.Comparator; @@ -41,12 +42,12 @@ public class LongSumAggregator implements Aggregator return ((Number) lhs).longValue() + ((Number) rhs).longValue(); } - private final FloatColumnSelector selector; + private final LongColumnSelector selector; private final String name; private long sum; - public LongSumAggregator(String name, FloatColumnSelector selector) + public LongSumAggregator(String name, LongColumnSelector selector) { this.name = name; this.selector = selector; @@ -57,7 +58,7 @@ public class LongSumAggregator implements Aggregator @Override public void aggregate() { - sum += (long) selector.get(); + sum += selector.get(); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java index 249d37045a0..cc7d218fc30 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java @@ -58,14 +58,14 @@ public class LongSumAggregatorFactory implements AggregatorFactory { return new LongSumAggregator( name, - metricFactory.makeFloatColumnSelector(fieldName) + metricFactory.makeLongColumnSelector(fieldName) ); } @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { - return new LongSumBufferAggregator(metricFactory.makeFloatColumnSelector(fieldName)); + return new LongSumBufferAggregator(metricFactory.makeLongColumnSelector(fieldName)); } @Override @@ -134,7 +134,7 @@ public class LongSumAggregatorFactory implements AggregatorFactory @Override public String getTypeName() { - return "float"; + return "long"; } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/LongSumBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/LongSumBufferAggregator.java index 72f0fbdd62d..465e21dd9ad 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumBufferAggregator.java @@ -20,6 +20,7 @@ package io.druid.query.aggregation; import io.druid.segment.FloatColumnSelector; +import io.druid.segment.LongColumnSelector; import java.nio.ByteBuffer; @@ -27,10 +28,10 @@ import java.nio.ByteBuffer; */ public class LongSumBufferAggregator implements BufferAggregator { - private final FloatColumnSelector selector; + private final LongColumnSelector selector; public LongSumBufferAggregator( - FloatColumnSelector selector + LongColumnSelector selector ) { this.selector = selector; @@ -45,7 +46,7 @@ public class LongSumBufferAggregator implements BufferAggregator @Override public void aggregate(ByteBuffer buf, int position) { - buf.putLong(position, buf.getLong(position) + (long) selector.get()); + buf.putLong(position, buf.getLong(position) + selector.get()); } @Override @@ -60,6 +61,13 @@ public class LongSumBufferAggregator implements BufferAggregator return (float) buf.getLong(position); } + + @Override + public long getLong(ByteBuffer buf, int position) + { + return buf.getLong(position); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/MaxBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/MaxBufferAggregator.java index 2a793a34d45..74416b7106e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MaxBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/MaxBufferAggregator.java @@ -58,6 +58,12 @@ public class MaxBufferAggregator implements BufferAggregator return (float) buf.getDouble(position); } + @Override + public long getLong(ByteBuffer buf, int position) + { + return (long) buf.getDouble(position); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/MinBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/MinBufferAggregator.java index f3c5964e6c1..a84b7082def 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MinBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/MinBufferAggregator.java @@ -58,6 +58,13 @@ public class MinBufferAggregator implements BufferAggregator return (float) buf.getDouble(position); } + + @Override + public long getLong(ByteBuffer buf, int position) + { + return (long) buf.getDouble(position); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java index 8c084f46a83..97c501874dc 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java @@ -79,7 +79,14 @@ public class CardinalityBufferAggregator implements BufferAggregator @Override public float getFloat(ByteBuffer buf, int position) { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("CardinalityBufferAggregator does not support getFloat()"); + } + + + @Override + public long getLong(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("CardinalityBufferAggregator does not support getLong()"); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java index c088b37beca..a99a52c4f0f 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java @@ -79,7 +79,14 @@ public class HyperUniquesBufferAggregator implements BufferAggregator @Override public float getFloat(ByteBuffer buf, int position) { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("HyperUniquesBufferAggregator does not support getFloat()"); + } + + + @Override + public long getLong(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("HyperUniquesBufferAggregator does not support getLong()"); } @Override diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java index 093b2ee3c7a..a3ca6dbee72 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java @@ -24,20 +24,25 @@ import com.google.common.collect.Lists; import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.common.guava.Accumulator; +import io.druid.collections.StupidPool; import io.druid.data.input.Row; import io.druid.data.input.Rows; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.dimension.DimensionSpec; import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.OffheapIncrementalIndex; +import java.nio.ByteBuffer; import java.util.List; public class GroupByQueryHelper { public static Pair> createIndexAccumulatorPair( final GroupByQuery query, - final GroupByQueryConfig config + final GroupByQueryConfig config, + StupidPool bufferPool + ) { final QueryGranularity gran = query.getGranularity(); @@ -69,21 +74,37 @@ public class GroupByQueryHelper } } ); - IncrementalIndex index = new IncrementalIndex( + final IncrementalIndex index; + if(query.getContextValue("useOffheap", false)){ + index = new OffheapIncrementalIndex( + // use granularity truncated min timestamp + // since incoming truncated timestamps may precede timeStart + granTimeStart, + gran, + aggs.toArray(new AggregatorFactory[aggs.size()]), + bufferPool, + false + ); + } else { + index = new IncrementalIndex( // use granularity truncated min timestamp // since incoming truncated timestamps may precede timeStart granTimeStart, gran, - aggs.toArray(new AggregatorFactory[aggs.size()]) + aggs.toArray(new AggregatorFactory[aggs.size()]), + bufferPool, + false ); + } Accumulator accumulator = new Accumulator() { @Override public IncrementalIndex accumulate(IncrementalIndex accumulated, T in) { + if (in instanceof Row) { - if (accumulated.add(Rows.toCaseInsensitiveInputRow((Row) in, dimensions), false) + if (accumulated.add(Rows.toCaseInsensitiveInputRow((Row) in, dimensions)) > config.getMaxResults()) { throw new ISE("Computation exceeds maxRows limit[%s]", config.getMaxResults()); } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index e4931eaeb47..790d83482c3 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -32,13 +32,16 @@ import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.MergeSequence; +import com.metamx.common.guava.ResourceClosingSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.collections.OrderedMergeSequence; +import io.druid.collections.StupidPool; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.granularity.QueryGranularity; +import io.druid.guice.annotations.Global; import io.druid.query.CacheStrategy; import io.druid.query.DataSource; import io.druid.query.IntervalChunkingQueryRunner; @@ -82,19 +85,24 @@ public class GroupByQueryQueryToolChest extends QueryToolChest configSupplier; + + private final StupidPool bufferPool; private final ObjectMapper jsonMapper; private GroupByQueryEngine engine; // For running the outer query around a subquery + @Inject public GroupByQueryQueryToolChest( Supplier configSupplier, ObjectMapper jsonMapper, - GroupByQueryEngine engine + GroupByQueryEngine engine, + @Global StupidPool bufferPool ) { this.configSupplier = configSupplier; this.jsonMapper = jsonMapper; this.engine = engine; + this.bufferPool = bufferPool; } @Override @@ -103,24 +111,26 @@ public class GroupByQueryQueryToolChest extends QueryToolChest() { @Override - public Sequence run(Query input) + public Sequence run(Query input, Map context) { if (input.getContextBySegment(false)) { - return runner.run(input); + return runner.run(input, context); } if (Boolean.valueOf(input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) { - return mergeGroupByResults(((GroupByQuery) input).withOverriddenContext(NO_MERGE_CONTEXT), runner); + return mergeGroupByResults(((GroupByQuery) input).withOverriddenContext(NO_MERGE_CONTEXT), runner, context); } - return runner.run(input); + return runner.run(input, context); } }; } - private Sequence mergeGroupByResults(final GroupByQuery query, QueryRunner runner) + private Sequence mergeGroupByResults(final GroupByQuery query, QueryRunner runner, Map context) { // If there's a subquery, merge subquery results and then apply the aggregator + final DataSource dataSource = query.getDataSource(); + if (dataSource instanceof QueryDataSource) { GroupByQuery subquery; try { @@ -129,7 +139,8 @@ public class GroupByQueryQueryToolChest extends QueryToolChest subqueryResult = mergeGroupByResults(subquery, runner); + + final Sequence subqueryResult = mergeGroupByResults(subquery, runner, context); final List aggs = Lists.newArrayList(); for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) { aggs.addAll(aggregatorFactory.getRequiredColumns()); @@ -145,13 +156,22 @@ public class GroupByQueryQueryToolChest extends QueryToolChest( + outerQuery.applyLimit( + engine.process( + outerQuery, + new IncrementalIndexStorageAdapter( + index + ) + ) + ), + index ); - return outerQuery.applyLimit(engine.process(outerQuery, adapter)); } else { - return query.applyLimit(postAggregate(query, makeIncrementalIndex(query, runner.run(query)))); + final IncrementalIndex index = makeIncrementalIndex(query, runner.run(query, context)); + return new ResourceClosingSequence<>(query.applyLimit(postAggregate(query, index)), index); } } @@ -180,13 +200,13 @@ public class GroupByQueryQueryToolChest extends QueryToolChest> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair( query, - config + config, + bufferPool ); return rows.accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); } - @Override public Sequence mergeSequences(Sequence> seqOfSequences) { diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java index ceda3065f58..cd86d16a92b 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java @@ -32,7 +32,9 @@ import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.logger.Logger; +import io.druid.collections.StupidPool; import io.druid.data.input.Row; +import io.druid.guice.annotations.Global; import io.druid.query.AbstractPrioritizedCallable; import io.druid.query.ConcatQueryRunner; import io.druid.query.GroupByParallelQueryRunner; @@ -46,7 +48,9 @@ import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; import io.druid.segment.incremental.IncrementalIndex; +import java.nio.ByteBuffer; import java.util.List; +import java.util.Map; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -57,25 +61,27 @@ import java.util.concurrent.TimeoutException; */ public class GroupByQueryRunnerFactory implements QueryRunnerFactory { + private static final Logger log = new Logger(GroupByQueryRunnerFactory.class); private final GroupByQueryEngine engine; private final QueryWatcher queryWatcher; private final Supplier config; private final GroupByQueryQueryToolChest toolChest; - - private static final Logger log = new Logger(GroupByQueryRunnerFactory.class); + private final StupidPool computationBufferPool; @Inject public GroupByQueryRunnerFactory( GroupByQueryEngine engine, QueryWatcher queryWatcher, Supplier config, - GroupByQueryQueryToolChest toolChest + GroupByQueryQueryToolChest toolChest, + @Global StupidPool computationBufferPool ) { this.engine = engine; this.queryWatcher = queryWatcher; this.config = config; this.toolChest = toolChest; + this.computationBufferPool = computationBufferPool; } @Override @@ -102,13 +108,14 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory() { @Override - public Sequence run(final Query query) + public Sequence run(final Query query, final Map context) { final GroupByQuery queryParam = (GroupByQuery) query; final Pair> indexAccumulatorPair = GroupByQueryHelper .createIndexAccumulatorPair( queryParam, - config.get() + config.get(), + computationBufferPool ); final Pair> bySegmentAccumulatorPair = GroupByQueryHelper.createBySegmentAccumulatorPair(); final int priority = query.getContextPriority(0); @@ -121,13 +128,14 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory run(Query input) + public Sequence run(Query input, Map context) { if (!(input instanceof GroupByQuery)) { throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class); diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java b/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java index 9f5c8d19527..02e3a0c16d2 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java @@ -84,7 +84,7 @@ public class SegmentAnalyzer columns.put(columnName, analysis); } - columns.put("__time", lengthBasedAnalysis(index.getTimeColumn(), NUM_BYTES_IN_TIMESTAMP)); + columns.put(Column.TIME_COLUMN_NAME, lengthBasedAnalysis(index.getColumn(Column.TIME_COLUMN_NAME), NUM_BYTES_IN_TIMESTAMP)); return columns; } diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java index 00909ea6cbd..ac46c49d2a2 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java @@ -77,7 +77,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory() { @Override - public Sequence run(Query inQ) + public Sequence run(Query inQ, Map context) { SegmentMetadataQuery query = (SegmentMetadataQuery) inQ; @@ -136,7 +136,10 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory() { @Override - public Sequence run(final Query query) + public Sequence run( + final Query query, + final Map context + ) { final int priority = query.getContextPriority(0); ListenableFuture> future = queryExecutor.submit( @@ -145,7 +148,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory call() throws Exception { - return input.run(query); + return input.run(query, context); } } ); diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index 163e244223e..d6bcd61cfe1 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -278,7 +278,10 @@ public class SearchQueryQueryToolChest extends QueryToolChest> run(Query> input) + public Sequence> run( + Query> input, + Map context + ) { if (!(input instanceof SearchQuery)) { throw new ISE("Can only handle [%s], got [%s]", SearchQuery.class, input.getClass()); @@ -286,13 +289,13 @@ public class SearchQueryQueryToolChest extends QueryToolChest, Result>() { @Override diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java b/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java index 878151bfb64..65f4314057e 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java @@ -68,7 +68,10 @@ public class SearchQueryRunner implements QueryRunner> } @Override - public Sequence> run(final Query> input) + public Sequence> run( + final Query> input, + Map context + ) { if (!(input instanceof SearchQuery)) { throw new ISE("Got a [%s] which isn't a %s", input.getClass(), SearchQuery.class); diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java b/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java index 9ef4f3c1258..746cab65dde 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java @@ -28,10 +28,11 @@ import io.druid.query.QueryRunnerHelper; import io.druid.query.Result; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; +import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; -import io.druid.segment.TimestampColumnSelector; +import io.druid.segment.column.Column; import io.druid.segment.data.IndexedInts; import io.druid.segment.filter.Filters; import org.joda.time.DateTime; @@ -83,7 +84,7 @@ public class SelectQueryEngine .getThreshold() ); - final TimestampColumnSelector timestampColumnSelector = cursor.makeTimestampColumnSelector(); + final LongColumnSelector timestampColumnSelector = cursor.makeLongColumnSelector(Column.TIME_COLUMN_NAME); final Map dimSelectors = Maps.newHashMap(); for (String dim : dims) { @@ -110,7 +111,7 @@ public class SelectQueryEngine int offset = 0; while (!cursor.isDone() && offset < query.getPagingSpec().getThreshold()) { final Map theEvent = Maps.newLinkedHashMap(); - theEvent.put(EventHolder.timestampKey, new DateTime(timestampColumnSelector.getTimestamp())); + theEvent.put(EventHolder.timestampKey, new DateTime(timestampColumnSelector.get())); for (Map.Entry dimSelector : dimSelectors.entrySet()) { final String dim = dimSelector.getKey(); diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java index 1fa3327800d..5210a56ae6a 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java @@ -31,6 +31,8 @@ import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.segment.Segment; +import java.util.List; +import java.util.Map; import java.util.concurrent.ExecutorService; /** @@ -88,7 +90,10 @@ public class SelectQueryRunnerFactory } @Override - public Sequence> run(Query> input) + public Sequence> run( + Query> input, + Map context + ) { if (!(input instanceof SelectQuery)) { throw new ISE("Got a [%s] which isn't a %s", input.getClass(), SelectQuery.class); diff --git a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java index e9bcd73139d..b8b25922e26 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java @@ -20,14 +20,21 @@ package io.druid.query.spec; import com.google.common.base.Throwables; +import com.google.common.collect.Lists; import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Yielder; import com.metamx.common.guava.YieldingAccumulator; import io.druid.query.Query; import io.druid.query.QueryRunner; +import io.druid.query.RetryQueryRunner; +import io.druid.query.SegmentDescriptor; +import io.druid.segment.SegmentMissingException; import java.io.IOException; +import java.util.List; +import java.util.Map; import java.util.concurrent.Callable; /** @@ -35,11 +42,11 @@ import java.util.concurrent.Callable; public class SpecificSegmentQueryRunner implements QueryRunner { private final QueryRunner base; - private final QuerySegmentSpec specificSpec; + private final SpecificSegmentSpec specificSpec; public SpecificSegmentQueryRunner( QueryRunner base, - QuerySegmentSpec specificSpec + SpecificSegmentSpec specificSpec ) { this.base = base; @@ -47,7 +54,7 @@ public class SpecificSegmentQueryRunner implements QueryRunner } @Override - public Sequence run(final Query input) + public Sequence run(final Query input, final Map context) { final Query query = input.withQuerySegmentSpec(specificSpec); @@ -55,14 +62,29 @@ public class SpecificSegmentQueryRunner implements QueryRunner final String currThreadName = currThread.getName(); final String newName = String.format("%s_%s_%s", query.getType(), query.getDataSource(), query.getIntervals()); - final Sequence baseSequence = doNamed(currThread, currThreadName, newName, new Callable>() - { - @Override - public Sequence call() throws Exception - { - return base.run(query); - } - }); + final Sequence baseSequence = doNamed( + currThread, currThreadName, newName, new Callable>() + { + @Override + public Sequence call() throws Exception + { + Sequence returningSeq; + try { + returningSeq = base.run(query, context); + } + catch (SegmentMissingException e) { + List missingSegments = (List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY); + if (missingSegments == null) { + missingSegments = Lists.newArrayList(); + context.put(RetryQueryRunner.MISSING_SEGMENTS_KEY, missingSegments); + } + missingSegments.add(specificSpec.getDescriptor()); + returningSeq = Sequences.empty(); + } + return returningSeq; + } + } + ); return new Sequence() { @@ -82,7 +104,10 @@ public class SpecificSegmentQueryRunner implements QueryRunner } @Override - public Yielder toYielder(final OutType initValue, final YieldingAccumulator accumulator) + public Yielder toYielder( + final OutType initValue, + final YieldingAccumulator accumulator + ) { return doItNamed( new Callable>() diff --git a/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java b/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java index d83ef60b894..eba3886aa9e 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java @@ -52,6 +52,8 @@ public class SpecificSegmentSpec implements QuerySegmentSpec return walker.getQueryRunnerForSegments(query, Arrays.asList(descriptor)); } + public SegmentDescriptor getDescriptor() { return descriptor; } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index a45a63e031d..27ae907f887 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -44,6 +44,7 @@ import org.joda.time.DateTime; import java.nio.ByteBuffer; import java.util.List; +import java.util.Map; /** */ @@ -94,13 +95,13 @@ public class TimeBoundaryQueryQueryToolChest { @Override protected Sequence> doRun( - QueryRunner> baseRunner, Query> input + QueryRunner> baseRunner, Query> input, Map context ) { TimeBoundaryQuery query = (TimeBoundaryQuery) input; return Sequences.simple( query.mergeResults( - Sequences.toList(baseRunner.run(query), Lists.>newArrayList()) + Sequences.toList(baseRunner.run(query, context), Lists.>newArrayList()) ) ); } diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index bdb17694346..7c302706b34 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -35,6 +35,8 @@ import io.druid.segment.StorageAdapter; import org.joda.time.DateTime; import java.util.Iterator; +import java.util.List; +import java.util.Map; import java.util.concurrent.ExecutorService; /** @@ -83,7 +85,10 @@ public class TimeBoundaryQueryRunnerFactory } @Override - public Sequence> run(Query> input) + public Sequence> run( + Query> input, + Map context + ) { if (!(input instanceof TimeBoundaryQuery)) { throw new ISE("Got a [%s] which isn't a %s", input.getClass(), TimeBoundaryQuery.class); diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java index acb3cfbc172..98882a75a9d 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java @@ -20,13 +20,13 @@ package io.druid.query.timeseries; import com.google.common.base.Function; -import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; import io.druid.query.QueryRunnerHelper; import io.druid.query.Result; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.Cursor; +import io.druid.segment.SegmentMissingException; import io.druid.segment.StorageAdapter; import io.druid.segment.filter.Filters; @@ -39,7 +39,7 @@ public class TimeseriesQueryEngine public Sequence> process(final TimeseriesQuery query, final StorageAdapter adapter) { if (adapter == null) { - throw new ISE( + throw new SegmentMissingException( "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." ); } diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java index c46363ac389..3da84f7ff24 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java @@ -32,6 +32,8 @@ import io.druid.query.Result; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; +import java.util.List; +import java.util.Map; import java.util.concurrent.ExecutorService; /** @@ -89,7 +91,10 @@ public class TimeseriesQueryRunnerFactory } @Override - public Sequence> run(Query> input) + public Sequence> run( + Query> input, + Map context + ) { if (!(input instanceof TimeseriesQuery)) { throw new ISE("Got a [%s] which isn't a %s", input.getClass(), TimeseriesQuery.class); diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java index d0d6f6deab1..65f2be86580 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java @@ -22,7 +22,6 @@ package io.druid.query.topn; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Predicates; -import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.logger.Logger; @@ -33,6 +32,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.filter.Filter; import io.druid.segment.Capabilities; import io.druid.segment.Cursor; +import io.druid.segment.SegmentMissingException; import io.druid.segment.StorageAdapter; import io.druid.segment.filter.Filters; import org.joda.time.Interval; @@ -56,7 +56,7 @@ public class TopNQueryEngine public Sequence> query(final TopNQuery query, final StorageAdapter adapter) { if (adapter == null) { - throw new ISE( + throw new SegmentMissingException( "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." ); } diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java index fe55cd7d888..165e3149456 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -409,7 +409,10 @@ public class TopNQueryQueryToolChest extends QueryToolChest> run(Query> input) + public Sequence> run( + Query> input, + Map context + ) { if (!(input instanceof TopNQuery)) { throw new ISE("Can only handle [%s], got [%s]", TopNQuery.class, input.getClass()); @@ -417,13 +420,13 @@ public class TopNQueryQueryToolChest extends QueryToolChest minTopNThreshold) { - return runner.run(query); + return runner.run(query, context); } final boolean isBySegment = query.getContextBySegment(false); return Sequences.map( - runner.run(query.withThreshold(minTopNThreshold)), + runner.run(query.withThreshold(minTopNThreshold), context), new Function, Result>() { @Override diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java index 3fcf074326c..54d5286254b 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java @@ -34,6 +34,8 @@ import io.druid.query.Result; import io.druid.segment.Segment; import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; import java.util.concurrent.ExecutorService; /** @@ -63,7 +65,10 @@ public class TopNQueryRunnerFactory implements QueryRunnerFactory>() { @Override - public Sequence> run(Query> input) + public Sequence> run( + Query> input, + Map context + ) { if (!(input instanceof TopNQuery)) { throw new ISE("Got a [%s] which isn't a %s", input.getClass(), TopNQuery.class); diff --git a/processing/src/main/java/io/druid/segment/BaseProgressIndicator.java b/processing/src/main/java/io/druid/segment/BaseProgressIndicator.java new file mode 100644 index 00000000000..7992f3a220c --- /dev/null +++ b/processing/src/main/java/io/druid/segment/BaseProgressIndicator.java @@ -0,0 +1,61 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment; + +/** + */ +public class BaseProgressIndicator implements ProgressIndicator +{ + @Override + public void progress() + { + // do nothing + } + + @Override + public void start() + { + // do nothing + } + + @Override + public void stop() + { + // do nothing + } + + @Override + public void startSection(String section) + { + // do nothing + } + + @Override + public void progressSection(String section, String message) + { + // do nothing + } + + @Override + public void stopSection(String section) + { + // do nothing + } +} diff --git a/processing/src/main/java/io/druid/segment/ColumnSelector.java b/processing/src/main/java/io/druid/segment/ColumnSelector.java index 0710effb9d2..8831b566edf 100644 --- a/processing/src/main/java/io/druid/segment/ColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/ColumnSelector.java @@ -23,6 +23,5 @@ package io.druid.segment;import io.druid.segment.column.Column; */ public interface ColumnSelector { - public Column getTimeColumn(); public Column getColumn(String columnName); } diff --git a/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java b/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java index 6aea75e2532..3c0d1352fff 100644 --- a/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java +++ b/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java @@ -91,7 +91,7 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector { GenericColumn column = null; try { - column = index.getTimeColumn().getGenericColumn(); + column = index.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn(); return column.length(); } finally { diff --git a/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java b/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java index 318f93af9a3..48613d16903 100644 --- a/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java +++ b/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java @@ -24,8 +24,8 @@ package io.druid.segment; */ public interface ColumnSelectorFactory { - public TimestampColumnSelector makeTimestampColumnSelector(); public DimensionSelector makeDimensionSelector(String dimensionName); public FloatColumnSelector makeFloatColumnSelector(String columnName); + public LongColumnSelector makeLongColumnSelector(String columnName); public ObjectColumnSelector makeObjectColumnSelector(String columnName); } diff --git a/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java index 4caa6b95fe2..2bc50c19d30 100644 --- a/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java @@ -21,6 +21,7 @@ package io.druid.segment; import com.google.common.io.Files; import io.druid.segment.data.CompressedFloatsSupplierSerializer; +import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.IOPeon; import java.io.File; @@ -51,7 +52,8 @@ public class FloatMetricColumnSerializer implements MetricColumnSerializer public void open() throws IOException { writer = CompressedFloatsSupplierSerializer.create( - ioPeon, String.format("%s_little", metricName), IndexIO.BYTE_ORDER + ioPeon, String.format("%s_little", metricName), IndexIO.BYTE_ORDER, + CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY ); writer.open(); diff --git a/processing/src/main/java/io/druid/segment/IncrementalIndexSegment.java b/processing/src/main/java/io/druid/segment/IncrementalIndexSegment.java index f21f7f1fa09..641ad41ba61 100644 --- a/processing/src/main/java/io/druid/segment/IncrementalIndexSegment.java +++ b/processing/src/main/java/io/druid/segment/IncrementalIndexSegment.java @@ -68,6 +68,6 @@ public class IncrementalIndexSegment implements Segment @Override public void close() throws IOException { - // do nothing + index.close(); } } diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index b532c3c5729..7e1001afb11 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -228,7 +228,7 @@ public class IndexIO case 6: case 7: log.info("Old version, re-persisting."); - IndexMerger.append( + IndexMaker.append( Arrays.asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert))), converted ); @@ -537,6 +537,10 @@ public class IndexIO final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); switch (holder.getType()) { + case LONG: + builder.setValueType(ValueType.LONG); + builder.addSerde(new LongGenericColumnPartSerde(holder.longType, BYTE_ORDER)); + break; case FLOAT: builder.setValueType(ValueType.FLOAT); builder.addSerde(new FloatGenericColumnPartSerde(holder.floatType, BYTE_ORDER)); @@ -715,15 +719,14 @@ public class IndexIO } String[] cols = colSet.toArray(new String[colSet.size()]); - + columns.put(Column.TIME_COLUMN_NAME, new ColumnBuilder() + .setType(ValueType.LONG) + .setGenericColumn(new LongGenericColumnSupplier(index.timestamps)) + .build()); return new SimpleQueryableIndex( index.getDataInterval(), - new ArrayIndexed(cols, String.class), + new ArrayIndexed<>(cols, String.class), index.getAvailableDimensions(), - new ColumnBuilder() - .setType(ValueType.LONG) - .setGenericColumn(new LongGenericColumnSupplier(index.timestamps)) - .build(), columns, index.getFileMapper() ); @@ -752,14 +755,14 @@ public class IndexIO Map columns = Maps.newHashMap(); - ObjectMapper mapper = new DefaultObjectMapper(); - for (String columnName : cols) { columns.put(columnName, deserializeColumn(mapper, smooshedFiles.mapFile(columnName))); } + columns.put(Column.TIME_COLUMN_NAME, deserializeColumn(mapper, smooshedFiles.mapFile("__time"))); + final QueryableIndex index = new SimpleQueryableIndex( - dataInterval, cols, dims, deserializeColumn(mapper, smooshedFiles.mapFile("__time")), columns, smooshedFiles + dataInterval, cols, dims, columns, smooshedFiles ); log.debug("Mapped v9 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime); diff --git a/processing/src/main/java/io/druid/segment/IndexMaker.java b/processing/src/main/java/io/druid/segment/IndexMaker.java new file mode 100644 index 00000000000..d054e0bedc2 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/IndexMaker.java @@ -0,0 +1,1739 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Objects; +import com.google.common.base.Predicate; +import com.google.common.base.Splitter; +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.Ordering; +import com.google.common.collect.Sets; +import com.google.common.io.ByteStreams; +import com.google.common.io.Files; +import com.google.common.primitives.Ints; +import com.google.inject.Injector; +import com.metamx.collections.spatial.ImmutableRTree; +import com.metamx.collections.spatial.RTree; +import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy; +import com.metamx.common.IAE; +import com.metamx.common.ISE; +import com.metamx.common.guava.FunctionalIterable; +import com.metamx.common.guava.MergeIterable; +import com.metamx.common.guava.nary.BinaryFn; +import com.metamx.common.io.smoosh.FileSmoosher; +import com.metamx.common.io.smoosh.SmooshedWriter; +import com.metamx.common.logger.Logger; +import io.druid.collections.CombiningIterable; +import io.druid.collections.ResourceHolder; +import io.druid.collections.StupidPool; +import io.druid.common.utils.JodaUtils; +import io.druid.common.utils.SerializerUtils; +import io.druid.guice.GuiceInjectors; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.ToLowerCaseAggregatorFactory; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ColumnCapabilitiesImpl; +import io.druid.segment.column.ColumnDescriptor; +import io.druid.segment.column.ValueType; +import io.druid.segment.data.CompressedFloatsIndexedSupplier; +import io.druid.segment.data.CompressedLongsIndexedSupplier; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.ConciseCompressedIndexedInts; +import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.IndexedIterable; +import io.druid.segment.data.VSizeIndexed; +import io.druid.segment.data.VSizeIndexedInts; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexAdapter; +import io.druid.segment.serde.ColumnPartSerde; +import io.druid.segment.serde.ComplexColumnPartSerde; +import io.druid.segment.serde.ComplexMetricSerde; +import io.druid.segment.serde.ComplexMetrics; +import io.druid.segment.serde.DictionaryEncodedColumnPartSerde; +import io.druid.segment.serde.FloatGenericColumnPartSerde; +import io.druid.segment.serde.LongGenericColumnPartSerde; +import it.uniroma3.mat.extendedset.intset.ConciseSet; +import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; +import org.apache.commons.io.FileUtils; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.FloatBuffer; +import java.nio.IntBuffer; +import java.nio.LongBuffer; +import java.util.AbstractList; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; + +/** + */ +public class IndexMaker +{ + private static final Logger log = new Logger(IndexMaker.class); + private static final SerializerUtils serializerUtils = new SerializerUtils(); + private static final int INVALID_ROW = -1; + private static final Splitter SPLITTER = Splitter.on(","); + private static final ObjectMapper mapper; + + static { + final Injector injector = GuiceInjectors.makeStartupInjector(); + mapper = injector.getInstance(ObjectMapper.class); + } + + public static File persist(final IncrementalIndex index, File outDir) throws IOException + { + return persist(index, index.getInterval(), outDir); + } + + /** + * This is *not* thread-safe and havok will ensue if this is called and writes are still occurring + * on the IncrementalIndex object. + * + * @param index the IncrementalIndex to persist + * @param dataInterval the Interval that the data represents + * @param outDir the directory to persist the data to + * + * @throws java.io.IOException + */ + public static File persist( + final IncrementalIndex index, + final Interval dataInterval, + File outDir + ) throws IOException + { + return persist(index, dataInterval, outDir, new LoggingProgressIndicator(outDir.toString())); + } + + public static File persist( + final IncrementalIndex index, + final Interval dataInterval, + File outDir, + ProgressIndicator progress + ) throws IOException + { + if (index.isEmpty()) { + throw new IAE("Trying to persist an empty index!"); + } + + final long firstTimestamp = index.getMinTime().getMillis(); + final long lastTimestamp = index.getMaxTime().getMillis(); + if (!(dataInterval.contains(firstTimestamp) && dataInterval.contains(lastTimestamp))) { + throw new IAE( + "interval[%s] does not encapsulate the full range of timestamps[%s, %s]", + dataInterval, + new DateTime(firstTimestamp), + new DateTime(lastTimestamp) + ); + } + + if (!outDir.exists()) { + outDir.mkdirs(); + } + if (!outDir.isDirectory()) { + throw new ISE("Can only persist to directories, [%s] wasn't a directory", outDir); + } + + log.info("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size()); + return merge( + Arrays.asList(new IncrementalIndexAdapter(dataInterval, index)), + index.getMetricAggs(), + outDir, + progress + ); + } + + public static File mergeQueryableIndex( + List indexes, final AggregatorFactory[] metricAggs, File outDir + ) throws IOException + { + return mergeQueryableIndex(indexes, metricAggs, outDir, new LoggingProgressIndicator(outDir.toString())); + } + + public static File mergeQueryableIndex( + List indexes, + final AggregatorFactory[] metricAggs, + File outDir, + ProgressIndicator progress + ) throws IOException + { + return merge( + Lists.transform( + indexes, + new Function() + { + @Override + public IndexableAdapter apply(final QueryableIndex input) + { + return new QueryableIndexIndexableAdapter(input); + } + } + ), + metricAggs, + outDir, + progress + ); + } + + public static File merge( + List adapters, final AggregatorFactory[] metricAggs, File outDir + ) throws IOException + { + return merge(adapters, metricAggs, outDir, new LoggingProgressIndicator(outDir.toString())); + } + + public static File merge( + List adapters, + final AggregatorFactory[] metricAggs, + File outDir, + ProgressIndicator progress + ) throws IOException + { + FileUtils.deleteDirectory(outDir); + if (!outDir.mkdirs()) { + throw new ISE("Couldn't make outdir[%s].", outDir); + } + + final AggregatorFactory[] lowerCaseMetricAggs = new AggregatorFactory[metricAggs.length]; + for (int i = 0; i < metricAggs.length; i++) { + lowerCaseMetricAggs[i] = new ToLowerCaseAggregatorFactory(metricAggs[i]); + } + + final List mergedDimensions = mergeIndexed( + Lists.transform( + adapters, + new Function>() + { + @Override + public Iterable apply(IndexableAdapter input) + { + return Iterables.transform( + input.getDimensionNames(), + new Function() + { + @Override + public String apply(String input) + { + return input.toLowerCase(); + } + } + ); + } + } + ) + ); + + final List mergedMetrics = Lists.transform( + mergeIndexed( + Lists.newArrayList( + FunctionalIterable + .create(adapters) + .transform( + new Function>() + { + @Override + public Iterable apply(IndexableAdapter input) + { + return Iterables.transform( + input.getMetricNames(), + new Function() + { + @Override + public String apply(String input) + { + return input.toLowerCase(); + } + } + ); + } + } + ) + .concat(Arrays.>asList(new AggFactoryStringIndexed(lowerCaseMetricAggs))) + ) + ), + new Function() + { + @Override + public String apply(String input) + { + return input.toLowerCase(); + } + } + ); + if (mergedMetrics.size() != lowerCaseMetricAggs.length) { + throw new IAE("Bad number of metrics[%d], expected [%d]", mergedMetrics.size(), lowerCaseMetricAggs.length); + } + + final AggregatorFactory[] sortedMetricAggs = new AggregatorFactory[mergedMetrics.size()]; + for (int i = 0; i < lowerCaseMetricAggs.length; i++) { + AggregatorFactory metricAgg = lowerCaseMetricAggs[i]; + sortedMetricAggs[mergedMetrics.indexOf(metricAgg.getName())] = metricAgg; + } + + for (int i = 0; i < mergedMetrics.size(); i++) { + if (!sortedMetricAggs[i].getName().equals(mergedMetrics.get(i))) { + throw new IAE( + "Metric mismatch, index[%d] [%s] != [%s]", + i, + lowerCaseMetricAggs[i].getName(), + mergedMetrics.get(i) + ); + } + } + + Function>, Iterable> rowMergerFn = new Function>, Iterable>() + { + @Override + public Iterable apply( + ArrayList> boats + ) + { + return CombiningIterable.create( + new MergeIterable<>( + Ordering.natural().nullsFirst(), + boats + ), + Ordering.natural().nullsFirst(), + new RowboatMergeFunction(sortedMetricAggs) + ); + } + }; + + return makeIndexFiles(adapters, outDir, progress, mergedDimensions, mergedMetrics, rowMergerFn); + } + + public static File append( + final List adapters, + File outDir + ) throws IOException + { + return append(adapters, outDir, new LoggingProgressIndicator(outDir.toString())); + } + + public static File append( + final List adapters, + final File outDir, + final ProgressIndicator progress + ) throws IOException + { + FileUtils.deleteDirectory(outDir); + if (!outDir.mkdirs()) { + throw new ISE("Couldn't make outdir[%s].", outDir); + } + + final List mergedDimensions = mergeIndexed( + Lists.transform( + adapters, + new Function>() + { + @Override + public Iterable apply(IndexableAdapter input) + { + return Iterables.transform( + input.getDimensionNames(), + new Function() + { + @Override + public String apply(String input) + { + return input.toLowerCase(); + } + } + ); + } + } + ) + ); + final List mergedMetrics = mergeIndexed( + Lists.transform( + adapters, + new Function>() + { + @Override + public Iterable apply(IndexableAdapter input) + { + return Iterables.transform( + input.getMetricNames(), + new Function() + { + @Override + public String apply(String input) + { + return input.toLowerCase(); + } + } + ); + } + } + ) + ); + + Function>, Iterable> rowMergerFn = new Function>, Iterable>() + { + @Override + public Iterable apply( + final ArrayList> boats + ) + { + return new MergeIterable<>( + Ordering.natural().nullsFirst(), + boats + ); + } + }; + + return makeIndexFiles(adapters, outDir, progress, mergedDimensions, mergedMetrics, rowMergerFn); + } + + private static File makeIndexFiles( + final List adapters, + final File outDir, + final ProgressIndicator progress, + final List mergedDimensions, + final List mergedMetrics, + final Function>, Iterable> rowMergerFn + ) throws IOException + { + progress.start(); + progress.progress(); + + final Map valueTypes = Maps.newTreeMap(Ordering.natural().nullsFirst()); + final Map metricTypeNames = Maps.newTreeMap(Ordering.natural().nullsFirst()); + final Map columnCapabilities = Maps.newHashMap(); + + for (IndexableAdapter adapter : adapters) { + for (String dimension : adapter.getDimensionNames()) { + ColumnCapabilitiesImpl mergedCapabilities = columnCapabilities.get(dimension); + ColumnCapabilities capabilities = adapter.getCapabilities(dimension); + if (mergedCapabilities == null) { + mergedCapabilities = new ColumnCapabilitiesImpl(); + mergedCapabilities.setType(ValueType.STRING); + } + columnCapabilities.put(dimension, mergedCapabilities.merge(capabilities)); + } + for (String metric : adapter.getMetricNames()) { + ColumnCapabilitiesImpl mergedCapabilities = columnCapabilities.get(metric); + ColumnCapabilities capabilities = adapter.getCapabilities(metric); + if (mergedCapabilities == null) { + mergedCapabilities = new ColumnCapabilitiesImpl(); + } + columnCapabilities.put(metric, mergedCapabilities.merge(capabilities)); + + valueTypes.put(metric, capabilities.getType()); + metricTypeNames.put(metric, adapter.getMetricType(metric)); + } + } + + outDir.mkdirs(); + final FileSmoosher v9Smoosher = new FileSmoosher(outDir); + + ByteStreams.write( + Ints.toByteArray(IndexIO.V9_VERSION), + Files.newOutputStreamSupplier(new File(outDir, "version.bin")) + ); + + final Map dimIndexes = Maps.newHashMap(); + final Map> dimensionValuesLookup = Maps.newHashMap(); + final ArrayList> dimConversions = Lists.newArrayListWithCapacity(adapters.size()); + final Set skippedDimensions = Sets.newHashSet(); + final List rowNumConversions = Lists.newArrayListWithCapacity(adapters.size()); + + progress.progress(); + setupDimConversion( + adapters, + progress, + mergedDimensions, + dimConversions, + dimIndexes, + skippedDimensions, + dimensionValuesLookup + ); + + progress.progress(); + final Iterable theRows = makeRowIterable( + adapters, + mergedDimensions, + mergedMetrics, + dimConversions, + rowMergerFn + ); + + progress.progress(); + final int rowCount = convertDims(adapters, progress, theRows, rowNumConversions); + + progress.progress(); + makeTimeColumn(v9Smoosher, progress, theRows, rowCount); + + progress.progress(); + makeDimColumns( + v9Smoosher, + adapters, + progress, + mergedDimensions, + skippedDimensions, + theRows, + columnCapabilities, + dimensionValuesLookup, + rowNumConversions + ); + + progress.progress(); + makeMetricColumns(v9Smoosher, progress, theRows, mergedMetrics, valueTypes, metricTypeNames, rowCount); + + progress.progress(); + makeIndexBinary(v9Smoosher, adapters, outDir, mergedDimensions, mergedMetrics, skippedDimensions, progress); + + v9Smoosher.close(); + + progress.stop(); + + return outDir; + } + + private static void setupDimConversion( + final List adapters, + final ProgressIndicator progress, + final List mergedDimensions, + final List> dimConversions, + final Map dimIndexes, + final Set skippedDimensions, + final Map> dimensionValuesLookup + ) + { + final String section = "setup dimension conversions"; + progress.startSection(section); + + for (IndexableAdapter adapter : adapters) { + dimConversions.add(Maps.newHashMap()); + } + + int dimIndex = 0; + for (String dimension : mergedDimensions) { + dimIndexes.put(dimension, dimIndex++); + + // lookups for all dimension values of this dimension + final List> dimValueLookups = Lists.newArrayListWithCapacity(adapters.size()); + + // each converter converts dim values of this dimension to global dictionary + final DimValueConverter[] converters = new DimValueConverter[adapters.size()]; + + for (int i = 0; i < adapters.size(); i++) { + Indexed dimValues = adapters.get(i).getDimValueLookup(dimension); + if (dimValues != null) { + dimValueLookups.add(dimValues); + converters[i] = new DimValueConverter(dimValues); + } + } + + // sort all dimension values and treat all null values as empty strings + final Iterable dimensionValues = CombiningIterable.createSplatted( + Iterables.transform( + dimValueLookups, + new Function, Iterable>() + { + @Override + public Iterable apply(Indexed indexed) + { + return Iterables.transform( + indexed, + new Function() + { + @Override + public String apply(@Nullable String input) + { + return (input == null) ? "" : input; + } + } + ); + } + } + ), + Ordering.natural() + ); + + int cardinality = 0; + for (String value : dimensionValues) { + for (int i = 0; i < adapters.size(); i++) { + DimValueConverter converter = converters[i]; + if (converter != null) { + converter.convert(value, cardinality); + } + } + + ++cardinality; + } + + if (cardinality == 0) { + log.info("Skipping [%s], it is empty!", dimension); + skippedDimensions.add(dimension); + continue; + } + + dimensionValuesLookup.put(dimension, dimensionValues); + + // make the dictionary + for (int i = 0; i < adapters.size(); ++i) { + DimValueConverter converter = converters[i]; + if (converter != null) { + dimConversions.get(i).put(dimension, converters[i].getConversionBuffer()); + } + } + } + + progress.stopSection(section); + } + + private static Iterable makeRowIterable( + final List adapters, + final List mergedDimensions, + final List mergedMetrics, + final ArrayList> dimConversions, + final Function>, Iterable> rowMergerFn + ) + { + ArrayList> boats = Lists.newArrayListWithCapacity(adapters.size()); + + for (int i = 0; i < adapters.size(); ++i) { + final IndexableAdapter adapter = adapters.get(i); + + final int[] dimLookup = new int[mergedDimensions.size()]; + int count = 0; + for (String dim : adapter.getDimensionNames()) { + dimLookup[count] = mergedDimensions.indexOf(dim.toLowerCase()); + count++; + } + + final int[] metricLookup = new int[mergedMetrics.size()]; + count = 0; + for (String metric : adapter.getMetricNames()) { + metricLookup[count] = mergedMetrics.indexOf(metric); + count++; + } + + boats.add( + new MMappedIndexRowIterable( + Iterables.transform( + adapters.get(i).getRows(), + new Function() + { + @Override + public Rowboat apply(Rowboat input) + { + int[][] newDims = new int[mergedDimensions.size()][]; + int j = 0; + for (int[] dim : input.getDims()) { + newDims[dimLookup[j]] = dim; + j++; + } + + Object[] newMetrics = new Object[mergedMetrics.size()]; + j = 0; + for (Object met : input.getMetrics()) { + newMetrics[metricLookup[j]] = met; + j++; + } + + return new Rowboat( + input.getTimestamp(), + newDims, + newMetrics, + input.getRowNum() + ); + } + } + ), + mergedDimensions, + dimConversions.get(i), + i + ) + ); + } + + return rowMergerFn.apply(boats); + } + + private static int convertDims( + final List adapters, + final ProgressIndicator progress, + final Iterable theRows, + final List rowNumConversions + ) throws IOException + { + final String section = "convert dims"; + progress.startSection(section); + + for (IndexableAdapter index : adapters) { + int[] arr = new int[index.getNumRows()]; + Arrays.fill(arr, INVALID_ROW); + rowNumConversions.add(IntBuffer.wrap(arr)); + } + + int rowCount = 0; + for (Rowboat theRow : theRows) { + for (Map.Entry> comprisedRow : theRow.getComprisedRows().entrySet()) { + final IntBuffer conversionBuffer = rowNumConversions.get(comprisedRow.getKey()); + + for (Integer rowNum : comprisedRow.getValue()) { + while (conversionBuffer.position() < rowNum) { + conversionBuffer.put(INVALID_ROW); + } + conversionBuffer.put(rowCount); + } + } + + if ((++rowCount % 500000) == 0) { + progress.progressSection(section, String.format("Walked 500,000/%,d rows", rowCount)); + } + } + + for (IntBuffer rowNumConversion : rowNumConversions) { + rowNumConversion.rewind(); + } + + progress.stopSection(section); + + return rowCount; + } + + private static void makeTimeColumn( + final FileSmoosher v9Smoosher, + final ProgressIndicator progress, + final Iterable theRows, + final int rowCount + ) throws IOException + { + final String section = "make time column"; + progress.startSection(section); + + long[] longs = new long[rowCount]; + + int rowNum = 0; + for (Rowboat theRow : theRows) { + longs[rowNum++] = theRow.getTimestamp(); + } + + CompressedLongsIndexedSupplier timestamps = CompressedLongsIndexedSupplier.fromLongBuffer( + LongBuffer.wrap(longs), + IndexIO.BYTE_ORDER, + CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY + + ); + + final ColumnDescriptor.Builder timeBuilder = ColumnDescriptor.builder(); + timeBuilder.setValueType(ValueType.LONG); + + writeColumn( + v9Smoosher, + new LongGenericColumnPartSerde(timestamps, IndexIO.BYTE_ORDER), + timeBuilder, + "__time" + ); + + progress.stopSection(section); + } + + private static void makeDimColumns( + final FileSmoosher v9Smoosher, + final List adapters, + final ProgressIndicator progress, + final List mergedDimensions, + final Set skippedDimensions, + final Iterable theRows, + final Map columnCapabilities, + final Map> dimensionValuesLookup, + final List rowNumConversions + ) throws IOException + { + final String dimSection = "make dimension columns"; + progress.startSection(dimSection); + + int dimIndex = 0; + for (String dimension : mergedDimensions) { + if (skippedDimensions.contains(dimension)) { + dimIndex++; + continue; + } + + makeDimColumn( + v9Smoosher, + adapters, + progress, + theRows, + dimIndex, + dimension, + columnCapabilities, + dimensionValuesLookup, + rowNumConversions + ); + dimIndex++; + } + progress.stopSection(dimSection); + } + + private static void makeDimColumn( + final FileSmoosher v9Smoosher, + final List adapters, + final ProgressIndicator progress, + final Iterable theRows, + final int dimIndex, + final String dimension, + final Map columnCapabilities, + final Map> dimensionValuesLookup, + final List rowNumConversions + ) throws IOException + { + final String section = String.format("make %s", dimension); + progress.startSection(section); + + final ColumnDescriptor.Builder dimBuilder = ColumnDescriptor.builder(); + dimBuilder.setValueType(ValueType.STRING); + + final List outParts = Lists.newArrayList(); + + ByteArrayOutputStream nameBAOS = new ByteArrayOutputStream(); + serializerUtils.writeString(nameBAOS, dimension); + outParts.add(ByteBuffer.wrap(nameBAOS.toByteArray())); + + boolean hasMultipleValues = columnCapabilities.get(dimension).hasMultipleValues(); + dimBuilder.setHasMultipleValues(hasMultipleValues); + + // make dimension columns + VSizeIndexedInts singleValCol = null; + VSizeIndexed multiValCol = null; + + ColumnDictionaryEntryStore adder = hasMultipleValues + ? new MultiValColumnDictionaryEntryStore() + : new SingleValColumnDictionaryEntryStore(); + + ConciseSet nullSet = null; + int rowCount = 0; + + for (Rowboat theRow : theRows) { + if (dimIndex > theRow.getDims().length) { + if (nullSet == null) { + nullSet = new ConciseSet(); + } + nullSet.add(rowCount); + adder.add(null); + } else { + int[] dimVals = theRow.getDims()[dimIndex]; + if (dimVals == null || dimVals.length == 0) { + if (nullSet == null) { + nullSet = new ConciseSet(); + } + nullSet.add(rowCount); + } + adder.add(dimVals); + } + rowCount++; + } + + final Iterable dimensionValues = dimensionValuesLookup.get(dimension); + GenericIndexed dictionary = GenericIndexed.fromIterable( + dimensionValues, + GenericIndexed.stringStrategy + ); + boolean bumpDictionary = false; + + if (hasMultipleValues) { + final List> vals = ((MultiValColumnDictionaryEntryStore) adder).get(); + if (nullSet != null) { + log.info("Dimension[%s] has null rows.", dimension); + + if (Iterables.getFirst(dimensionValues, "") != null) { + bumpDictionary = true; + log.info("Dimension[%s] has no null value in the dictionary, expanding...", dimension); + + final List nullList = Lists.newArrayList(); + nullList.add(null); + + dictionary = GenericIndexed.fromIterable( + Iterables.concat(nullList, dimensionValues), + GenericIndexed.stringStrategy + ); + + final int dictionarySize = dictionary.size(); + multiValCol = VSizeIndexed.fromIterable( + FunctionalIterable + .create(vals) + .transform( + new Function, VSizeIndexedInts>() + { + @Override + public VSizeIndexedInts apply(final List input) + { + if (input == null) { + return VSizeIndexedInts.fromList( + new AbstractList() + { + @Override + public Integer get(int index) + { + return 0; + } + + @Override + public int size() + { + return 1; + } + }, dictionarySize + ); + } + return VSizeIndexedInts.fromList( + new AbstractList() + { + @Override + public Integer get(int index) + { + Integer val = input.get(index); + if (val == null) { + return 0; + } + return val + 1; + } + + @Override + public int size() + { + return input.size(); + } + }, + dictionarySize + ); + } + } + ) + ); + } else { + final int dictionarySize = dictionary.size(); + multiValCol = VSizeIndexed.fromIterable( + FunctionalIterable + .create(vals) + .transform( + new Function, VSizeIndexedInts>() + { + @Override + public VSizeIndexedInts apply(List input) + { + if (input == null) { + return VSizeIndexedInts.fromList( + new AbstractList() + { + @Override + public Integer get(int index) + { + return 0; + } + + @Override + public int size() + { + return 1; + } + }, dictionarySize + ); + } + return VSizeIndexedInts.fromList( + input, + dictionarySize + ); + } + } + ) + ); + } + } else { + final int dictionarySize = dictionary.size(); + multiValCol = VSizeIndexed.fromIterable( + FunctionalIterable + .create(vals) + .transform( + new Function, VSizeIndexedInts>() + { + @Override + public VSizeIndexedInts apply(List input) + { + return VSizeIndexedInts.fromList( + input, + dictionarySize + ); + } + } + ) + ); + } + } else { + final List vals = ((SingleValColumnDictionaryEntryStore) adder).get(); + + if (nullSet != null) { + log.info("Dimension[%s] has null rows.", dimension); + + if (Iterables.getFirst(dimensionValues, "") != null) { + bumpDictionary = true; + log.info("Dimension[%s] has no null value in the dictionary, expanding...", dimension); + + final List nullList = Lists.newArrayList(); + nullList.add(null); + + dictionary = GenericIndexed.fromIterable( + Iterables.concat(nullList, dimensionValues), + GenericIndexed.stringStrategy + ); + singleValCol = VSizeIndexedInts.fromList( + new AbstractList() + { + @Override + public Integer get(int index) + { + Integer val = vals.get(index); + if (val == null) { + return 0; + } + return val + 1; + } + + @Override + public int size() + { + return vals.size(); + } + }, dictionary.size() + ); + } else { + singleValCol = VSizeIndexedInts.fromList( + new AbstractList() + { + @Override + public Integer get(int index) + { + Integer val = vals.get(index); + if (val == null) { + return 0; + } + return val; + } + + @Override + public int size() + { + return vals.size(); + } + }, dictionary.size() + ); + } + } else { + singleValCol = VSizeIndexedInts.fromList(vals, dictionary.size()); + } + } + + // Make bitmap indexes + List conciseSets = Lists.newArrayList(); + for (String dimVal : dimensionValues) { + List> convertedInverteds = Lists.newArrayListWithCapacity(adapters.size()); + for (int j = 0; j < adapters.size(); ++j) { + convertedInverteds.add( + new ConvertingIndexedInts( + adapters.get(j).getInverteds(dimension, dimVal), rowNumConversions.get(j) + ) + ); + } + + ConciseSet bitset = new ConciseSet(); + for (Integer row : CombiningIterable.createSplatted( + convertedInverteds, + Ordering.natural().nullsFirst() + )) { + if (row != INVALID_ROW) { + bitset.add(row); + } + } + + conciseSets.add(bitset); + } + + GenericIndexed bitmaps; + + if (nullSet != null) { + final ImmutableConciseSet theNullSet = ImmutableConciseSet.newImmutableFromMutable(nullSet); + if (bumpDictionary) { + bitmaps = GenericIndexed.fromIterable( + Iterables.concat( + Arrays.asList(theNullSet), + Iterables.transform( + conciseSets, + new Function() + { + @Override + public ImmutableConciseSet apply(ConciseSet input) + { + return ImmutableConciseSet.newImmutableFromMutable(input); + } + } + ) + ), + ConciseCompressedIndexedInts.objectStrategy + ); + } else { + Iterable immutableConciseSets = Iterables.transform( + conciseSets, + new Function() + { + @Override + public ImmutableConciseSet apply(ConciseSet input) + { + return ImmutableConciseSet.newImmutableFromMutable(input); + } + } + ); + + bitmaps = GenericIndexed.fromIterable( + Iterables.concat( + Arrays.asList( + ImmutableConciseSet.union( + theNullSet, + Iterables.getFirst(immutableConciseSets, null) + ) + ), + Iterables.skip(immutableConciseSets, 1) + ), + ConciseCompressedIndexedInts.objectStrategy + ); + } + } else { + bitmaps = GenericIndexed.fromIterable( + Iterables.transform( + conciseSets, + new Function() + { + @Override + public ImmutableConciseSet apply(ConciseSet input) + { + return ImmutableConciseSet.newImmutableFromMutable(input); + } + } + ), + ConciseCompressedIndexedInts.objectStrategy + ); + } + + // Make spatial indexes + ImmutableRTree spatialIndex = null; + boolean hasSpatialIndexes = columnCapabilities.get(dimension).hasSpatialIndexes(); + RTree tree = null; + if (hasSpatialIndexes) { + tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50)); + } + + int dimValIndex = 0; + for (String dimVal : dimensionValuesLookup.get(dimension)) { + if (hasSpatialIndexes) { + if (dimVal != null && !dimVal.isEmpty()) { + List stringCoords = Lists.newArrayList(SPLITTER.split(dimVal)); + float[] coords = new float[stringCoords.size()]; + for (int j = 0; j < coords.length; j++) { + coords[j] = Float.valueOf(stringCoords.get(j)); + } + tree.insert(coords, conciseSets.get(dimValIndex)); + } + dimValIndex++; + } + } + if (hasSpatialIndexes) { + spatialIndex = ImmutableRTree.newImmutableFromMutable(tree); + } + + log.info("Completed dimension[%s] with cardinality[%,d]. Starting write.", dimension, dictionary.size()); + + writeColumn( + v9Smoosher, + new DictionaryEncodedColumnPartSerde( + dictionary, + singleValCol, + multiValCol, + bitmaps, + spatialIndex + ), + dimBuilder, + dimension + ); + + progress.stopSection(section); + } + + private static void makeMetricColumns( + final FileSmoosher v9Smoosher, + final ProgressIndicator progress, + final Iterable theRows, + final List mergedMetrics, + final Map valueTypes, + final Map metricTypeNames, + final int rowCount + ) throws IOException + { + final String metSection = "make metric columns"; + progress.startSection(metSection); + + int metIndex = 0; + for (String metric : mergedMetrics) { + makeMetricColumn(v9Smoosher, progress, theRows, metIndex, metric, valueTypes, metricTypeNames, rowCount); + metIndex++; + } + progress.stopSection(metSection); + } + + private static void makeMetricColumn( + final FileSmoosher v9Smoosher, + final ProgressIndicator progress, + final Iterable theRows, + final int metricIndex, + final String metric, + final Map valueTypes, + final Map metricTypeNames, + final int rowCount + ) throws IOException + { + final String section = String.format("make column[%s]", metric); + progress.startSection(section); + + final ColumnDescriptor.Builder metBuilder = ColumnDescriptor.builder(); + ValueType type = valueTypes.get(metric); + + switch (type) { + case FLOAT: { + metBuilder.setValueType(ValueType.FLOAT); + + float[] arr = new float[rowCount]; + int rowNum = 0; + for (Rowboat theRow : theRows) { + Object obj = theRow.getMetrics()[metricIndex]; + arr[rowNum++] = (obj == null) ? 0 : ((Number) obj).floatValue(); + } + + CompressedFloatsIndexedSupplier compressedFloats = CompressedFloatsIndexedSupplier.fromFloatBuffer( + FloatBuffer.wrap(arr), + IndexIO.BYTE_ORDER, + CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY + ); + + writeColumn( + v9Smoosher, + new FloatGenericColumnPartSerde(compressedFloats, IndexIO.BYTE_ORDER), + metBuilder, + metric + ); + break; + } + case LONG: { + metBuilder.setValueType(ValueType.LONG); + + long[] arr = new long[rowCount]; + int rowNum = 0; + for (Rowboat theRow : theRows) { + Object obj = theRow.getMetrics()[metricIndex]; + arr[rowNum++] = (obj == null) ? 0 : ((Number) obj).longValue(); + } + + CompressedLongsIndexedSupplier compressedLongs = CompressedLongsIndexedSupplier.fromLongBuffer( + LongBuffer.wrap(arr), + IndexIO.BYTE_ORDER, + CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY + ); + + writeColumn( + v9Smoosher, + new LongGenericColumnPartSerde(compressedLongs, IndexIO.BYTE_ORDER), + metBuilder, + metric + ); + break; + } + case COMPLEX: + String complexType = metricTypeNames.get(metric); + + ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(complexType); + + if (serde == null) { + throw new ISE("Unknown type[%s]", complexType); + } + + final GenericIndexed metricColumn = GenericIndexed.fromIterable( + Iterables.transform( + theRows, + new Function() + { + @Override + public Object apply(Rowboat input) + { + return input.getMetrics()[metricIndex]; + } + } + ), + serde.getObjectStrategy() + ); + + metBuilder.setValueType(ValueType.COMPLEX); + writeColumn( + v9Smoosher, + new ComplexColumnPartSerde(metricColumn, complexType), + metBuilder, + metric + ); + break; + default: + throw new ISE("Unknown type[%s]", type); + } + + progress.stopSection(section); + } + + private static void makeIndexBinary( + final FileSmoosher v9Smoosher, + final List adapters, + final File outDir, + final List mergedDimensions, + final List mergedMetrics, + final Set skippedDimensions, + final ProgressIndicator progress + ) throws IOException + { + final String section = "building index.drd"; + progress.startSection(section); + + final Set finalColumns = Sets.newTreeSet(); + finalColumns.addAll(mergedDimensions); + finalColumns.addAll(mergedMetrics); + finalColumns.removeAll(skippedDimensions); + + final Iterable finalDimensions = Iterables.filter( + mergedDimensions, + new Predicate() + { + @Override + public boolean apply(String input) + { + return !skippedDimensions.contains(input); + } + } + ); + + GenericIndexed cols = GenericIndexed.fromIterable(finalColumns, GenericIndexed.stringStrategy); + GenericIndexed dims = GenericIndexed.fromIterable(finalDimensions, GenericIndexed.stringStrategy); + + final long numBytes = cols.getSerializedSize() + dims.getSerializedSize() + 16; + final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes); + + cols.writeToChannel(writer); + dims.writeToChannel(writer); + + DateTime minTime = new DateTime(Long.MAX_VALUE); + DateTime maxTime = new DateTime(0l); + + for (IndexableAdapter index : adapters) { + minTime = JodaUtils.minDateTime(minTime, index.getDataInterval().getStart()); + maxTime = JodaUtils.maxDateTime(maxTime, index.getDataInterval().getEnd()); + } + final Interval dataInterval = new Interval(minTime, maxTime); + + serializerUtils.writeLong(writer, dataInterval.getStartMillis()); + serializerUtils.writeLong(writer, dataInterval.getEndMillis()); + writer.close(); + + IndexIO.checkFileSize(new File(outDir, "index.drd")); + + progress.stopSection(section); + } + + private static void writeColumn( + FileSmoosher v9Smoosher, + ColumnPartSerde serde, + ColumnDescriptor.Builder builder, + String name + ) throws IOException + { + builder.addSerde(serde); + + final ColumnDescriptor descriptor = builder.build(); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + serializerUtils.writeString(baos, mapper.writeValueAsString(descriptor)); + byte[] specBytes = baos.toByteArray(); + + final SmooshedWriter channel = v9Smoosher.addWithSmooshedWriter( + name, descriptor.numBytes() + specBytes.length + ); + channel.write(ByteBuffer.wrap(specBytes)); + descriptor.write(channel); + channel.close(); + } + + private static ArrayList mergeIndexed(final List> indexedLists) + { + Set retVal = Sets.newTreeSet(Ordering.natural().nullsFirst()); + + for (Iterable indexedList : indexedLists) { + for (T val : indexedList) { + retVal.add(val); + } + } + + return Lists.newArrayList(retVal); + } + + private static interface ColumnDictionaryEntryStore + { + public void add(int[] vals); + } + + private static class DimValueConverter + { + private final Indexed dimSet; + private final IntBuffer conversionBuf; + private int currIndex; + private String lastVal = null; + + DimValueConverter( + Indexed dimSet + ) + { + this.dimSet = dimSet; + final int bufferSize = dimSet.size() * Ints.BYTES; + log.info("Allocating new dimension conversion buffer of size[%,d]", bufferSize); + this.conversionBuf = ByteBuffer.allocateDirect(bufferSize).asIntBuffer(); + + this.currIndex = 0; + } + + public void convert(String value, int index) + { + if (dimSet.size() == 0) { + return; + } + if (lastVal != null) { + if (value.compareTo(lastVal) <= 0) { + throw new ISE("Value[%s] is less than the last value[%s] I have, cannot be.", value, lastVal); + } + return; + } + String currValue = dimSet.get(currIndex); + + while (currValue == null) { + conversionBuf.position(conversionBuf.position() + 1); + ++currIndex; + if (currIndex == dimSet.size()) { + lastVal = value; + return; + } + currValue = dimSet.get(currIndex); + } + + if (Objects.equal(currValue, value)) { + conversionBuf.put(index); + ++currIndex; + if (currIndex == dimSet.size()) { + lastVal = value; + } + } else if (currValue.compareTo(value) < 0) { + throw new ISE( + "Skipped currValue[%s], currIndex[%,d]; incoming value[%s], index[%,d]", currValue, currIndex, value, index + ); + } + } + + public IntBuffer getConversionBuffer() + { + if (currIndex != conversionBuf.limit() || conversionBuf.hasRemaining()) { + throw new ISE( + "Asked for incomplete buffer. currIndex[%,d] != buf.limit[%,d]", currIndex, conversionBuf.limit() + ); + } + return (IntBuffer) conversionBuf.asReadOnlyBuffer().rewind(); + } + } + + private static class ConvertingIndexedInts implements Iterable + { + private final IndexedInts baseIndex; + private final IntBuffer conversionBuffer; + + public ConvertingIndexedInts( + IndexedInts baseIndex, + IntBuffer conversionBuffer + ) + { + this.baseIndex = baseIndex; + this.conversionBuffer = conversionBuffer; + } + + public int size() + { + return baseIndex.size(); + } + + public int get(int index) + { + return conversionBuffer.get(baseIndex.get(index)); + } + + @Override + public Iterator iterator() + { + return Iterators.transform( + baseIndex.iterator(), + new Function() + { + @Override + public Integer apply(Integer input) + { + return conversionBuffer.get(input); + } + } + ); + } + } + + private static class MMappedIndexRowIterable implements Iterable + { + private final Iterable index; + private final List convertedDims; + private final Map converters; + private final int indexNumber; + + MMappedIndexRowIterable( + Iterable index, + List convertedDims, + Map converters, + int indexNumber + ) + { + this.index = index; + this.convertedDims = convertedDims; + this.converters = converters; + this.indexNumber = indexNumber; + } + + public Iterable getIndex() + { + return index; + } + + public List getConvertedDims() + { + return convertedDims; + } + + public Map getConverters() + { + return converters; + } + + public int getIndexNumber() + { + return indexNumber; + } + + @Override + public Iterator iterator() + { + return Iterators.transform( + index.iterator(), + new Function() + { + @Override + public Rowboat apply(Rowboat input) + { + int[][] dims = input.getDims(); + int[][] newDims = new int[convertedDims.size()][]; + for (int i = 0; i < convertedDims.size(); ++i) { + IntBuffer converter = converters.get(convertedDims.get(i)); + + if (converter == null) { + continue; + } + + if (i >= dims.length || dims[i] == null) { + continue; + } + + newDims[i] = new int[dims[i].length]; + + for (int j = 0; j < dims[i].length; ++j) { + if (!converter.hasRemaining()) { + throw new ISE("Converter mismatch! wtfbbq!"); + } + newDims[i][j] = converter.get(dims[i][j]); + } + } + + final Rowboat retVal = new Rowboat( + input.getTimestamp(), + newDims, + input.getMetrics(), + input.getRowNum() + ); + + retVal.addRow(indexNumber, input.getRowNum()); + + return retVal; + } + } + ); + } + } + + private static class AggFactoryStringIndexed implements Indexed + { + private final AggregatorFactory[] metricAggs; + + public AggFactoryStringIndexed(AggregatorFactory[] metricAggs) {this.metricAggs = metricAggs;} + + @Override + public Class getClazz() + { + return String.class; + } + + @Override + public int size() + { + return metricAggs.length; + } + + @Override + public String get(int index) + { + return metricAggs[index].getName(); + } + + @Override + public int indexOf(String value) + { + throw new UnsupportedOperationException(); + } + + @Override + public Iterator iterator() + { + return IndexedIterable.create(this).iterator(); + } + } + + private static class RowboatMergeFunction implements BinaryFn + { + private final AggregatorFactory[] metricAggs; + + public RowboatMergeFunction(AggregatorFactory[] metricAggs) + { + this.metricAggs = metricAggs; + } + + @Override + public Rowboat apply(Rowboat lhs, Rowboat rhs) + { + if (lhs == null) { + return rhs; + } + if (rhs == null) { + return lhs; + } + + Object[] metrics = new Object[metricAggs.length]; + Object[] lhsMetrics = lhs.getMetrics(); + Object[] rhsMetrics = rhs.getMetrics(); + + for (int i = 0; i < metrics.length; ++i) { + metrics[i] = metricAggs[i].combine(lhsMetrics[i], rhsMetrics[i]); + } + + final Rowboat retVal = new Rowboat( + lhs.getTimestamp(), + lhs.getDims(), + metrics, + lhs.getRowNum() + ); + + for (Rowboat rowboat : Arrays.asList(lhs, rhs)) { + for (Map.Entry> entry : rowboat.getComprisedRows().entrySet()) { + for (Integer rowNum : entry.getValue()) { + retVal.addRow(entry.getKey(), rowNum); + } + } + } + + return retVal; + } + } + + private static class SingleValColumnDictionaryEntryStore implements ColumnDictionaryEntryStore + { + private final List data = Lists.newArrayList(); + + @Override + public void add(int[] vals) + { + if (vals == null || vals.length == 0) { + data.add(null); + } else { + data.add(vals[0]); + } + } + + public List get() + { + return data; + } + } + + private static class MultiValColumnDictionaryEntryStore implements ColumnDictionaryEntryStore + { + private final List> data = Lists.newArrayList(); + + public void add(int[] vals) + { + if (vals == null || vals.length == 0) { + data.add(null); + } else { + data.add(Ints.asList(vals)); + } + } + + public List> get() + { + return data; + } + } +} diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index 83b796d3a7d..d5947b21cab 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -50,8 +50,12 @@ import io.druid.common.utils.JodaUtils; import io.druid.common.utils.SerializerUtils; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.ToLowerCaseAggregatorFactory; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ColumnCapabilitiesImpl; +import io.druid.segment.column.ValueType; import io.druid.segment.data.ByteBufferWriter; import io.druid.segment.data.CompressedLongsSupplierSerializer; +import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.ConciseCompressedIndexedInts; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexedWriter; @@ -113,11 +117,12 @@ public class IndexMerger * @param outDir the directory to persist the data to * * @return the index output directory + * * @throws java.io.IOException if an IO error occurs persisting the index */ public static File persist(final IncrementalIndex index, final Interval dataInterval, File outDir) throws IOException { - return persist(index, dataInterval, outDir, new NoopProgressIndicator()); + return persist(index, dataInterval, outDir, new BaseProgressIndicator()); } public static File persist( @@ -159,7 +164,7 @@ public class IndexMerger List indexes, final AggregatorFactory[] metricAggs, File outDir ) throws IOException { - return mergeQueryableIndex(indexes, metricAggs, outDir, new NoopProgressIndicator()); + return mergeQueryableIndex(indexes, metricAggs, outDir, new BaseProgressIndicator()); } public static File mergeQueryableIndex( @@ -188,7 +193,7 @@ public class IndexMerger List indexes, final AggregatorFactory[] metricAggs, File outDir ) throws IOException { - return merge(indexes, metricAggs, outDir, new NoopProgressIndicator()); + return merge(indexes, metricAggs, outDir, new BaseProgressIndicator()); } public static File merge( @@ -214,7 +219,7 @@ public class IndexMerger public Iterable apply(@Nullable IndexableAdapter input) { return Iterables.transform( - input.getAvailableDimensions(), + input.getDimensionNames(), new Function() { @Override @@ -240,7 +245,7 @@ public class IndexMerger public Iterable apply(@Nullable IndexableAdapter input) { return Iterables.transform( - input.getAvailableMetrics(), + input.getMetricNames(), new Function() { @Override @@ -311,7 +316,7 @@ public class IndexMerger List indexes, File outDir ) throws IOException { - return append(indexes, outDir, new NoopProgressIndicator()); + return append(indexes, outDir, new BaseProgressIndicator()); } public static File append( @@ -332,7 +337,7 @@ public class IndexMerger public Iterable apply(@Nullable IndexableAdapter input) { return Iterables.transform( - input.getAvailableDimensions(), + input.getDimensionNames(), new Function() { @Override @@ -355,7 +360,7 @@ public class IndexMerger public Iterable apply(@Nullable IndexableAdapter input) { return Iterables.transform( - input.getAvailableMetrics(), + input.getMetricNames(), new Function() { @Override @@ -396,12 +401,34 @@ public class IndexMerger final Function>, Iterable> rowMergerFn ) throws IOException { - Map metricTypes = Maps.newTreeMap(Ordering.natural().nullsFirst()); + final Map valueTypes = Maps.newTreeMap(Ordering.natural().nullsFirst()); + final Map metricTypeNames = Maps.newTreeMap(Ordering.natural().nullsFirst()); + final Map columnCapabilities = Maps.newHashMap(); + for (IndexableAdapter adapter : indexes) { - for (String metric : adapter.getAvailableMetrics()) { - metricTypes.put(metric, adapter.getMetricType(metric)); + for (String dimension : adapter.getDimensionNames()) { + ColumnCapabilitiesImpl mergedCapabilities = columnCapabilities.get(dimension); + ColumnCapabilities capabilities = adapter.getCapabilities(dimension); + if (mergedCapabilities == null) { + mergedCapabilities = new ColumnCapabilitiesImpl(); + mergedCapabilities.setType(ValueType.STRING); + } + columnCapabilities.put(dimension, mergedCapabilities.merge(capabilities)); + } + for (String metric : adapter.getMetricNames()) { + ColumnCapabilitiesImpl mergedCapabilities = columnCapabilities.get(metric); + ColumnCapabilities capabilities = adapter.getCapabilities(metric); + if (mergedCapabilities == null) { + mergedCapabilities = new ColumnCapabilitiesImpl(); + } + columnCapabilities.put(metric, mergedCapabilities.merge(capabilities)); + + valueTypes.put(metric, capabilities.getType()); + metricTypeNames.put(metric, adapter.getMetricType(metric)); } } + + final Interval dataInterval; File v8OutDir = new File(outDir, "v8-tmp"); v8OutDir.mkdirs(); @@ -540,14 +567,14 @@ public class IndexMerger final int[] dimLookup = new int[mergedDimensions.size()]; int count = 0; - for (String dim : adapter.getAvailableDimensions()) { + for (String dim : adapter.getDimensionNames()) { dimLookup[count] = mergedDimensions.indexOf(dim.toLowerCase()); count++; } final int[] metricLookup = new int[mergedMetrics.size()]; count = 0; - for (String metric : adapter.getAvailableMetrics()) { + for (String metric : adapter.getMetricNames()) { metricLookup[count] = mergedMetrics.indexOf(metric); count++; } @@ -579,8 +606,7 @@ public class IndexMerger input.getTimestamp(), newDims, newMetrics, - input.getRowNum(), - input.getDescriptions() + input.getRowNum() ); } } @@ -593,7 +619,7 @@ public class IndexMerger Iterable theRows = rowMergerFn.apply(boats); CompressedLongsSupplierSerializer timeWriter = CompressedLongsSupplierSerializer.create( - ioPeon, "little_end_time", IndexIO.BYTE_ORDER + ioPeon, "little_end_time", IndexIO.BYTE_ORDER, CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY ); timeWriter.open(); @@ -606,21 +632,30 @@ public class IndexMerger } ArrayList metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size()); - for (Map.Entry entry : metricTypes.entrySet()) { - String metric = entry.getKey(); - String typeName = entry.getValue(); - if ("float".equals(typeName)) { - metWriters.add(new FloatMetricColumnSerializer(metric, v8OutDir, ioPeon)); - } else { - ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); + for (String metric : mergedMetrics) { + ValueType type = valueTypes.get(metric); + switch (type) { + case LONG: + metWriters.add(new LongMetricColumnSerializer(metric, v8OutDir, ioPeon)); + break; + case FLOAT: + metWriters.add(new FloatMetricColumnSerializer(metric, v8OutDir, ioPeon)); + break; + case COMPLEX: + final String typeName = metricTypeNames.get(metric); + ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); - if (serde == null) { - throw new ISE("Unknown type[%s]", typeName); - } + if (serde == null) { + throw new ISE("Unknown type[%s]", typeName); + } - metWriters.add(new ComplexMetricColumnSerializer(metric, v8OutDir, ioPeon, serde)); + metWriters.add(new ComplexMetricColumnSerializer(metric, v8OutDir, ioPeon, serde)); + break; + default: + throw new ISE("Unknown type[%s]", type); } } + for (MetricColumnSerializer metWriter : metWriters) { metWriter.open(); } @@ -634,7 +669,6 @@ public class IndexMerger rowNumConversions.add(IntBuffer.wrap(arr)); } - final Map descriptions = Maps.newHashMap(); for (Rowboat theRow : theRows) { progress.progress(); timeWriter.add(theRow.getTimestamp()); @@ -669,8 +703,6 @@ public class IndexMerger ); time = System.currentTimeMillis(); } - - descriptions.putAll(theRow.getDescriptions()); } for (IntBuffer rowNumConversion : rowNumConversions) { @@ -729,7 +761,7 @@ public class IndexMerger ); writer.open(); - boolean isSpatialDim = "spatial".equals(descriptions.get(dimension)); + boolean isSpatialDim = columnCapabilities.get(dimension).hasSpatialIndexes(); ByteBufferWriter spatialWriter = null; RTree tree = null; IOPeon spatialIoPeon = new TmpFileIOPeon(); @@ -1067,8 +1099,7 @@ public class IndexMerger input.getTimestamp(), newDims, input.getMetrics(), - input.getRowNum(), - input.getDescriptions() + input.getRowNum() ); retVal.addRow(indexNumber, input.getRowNum()); @@ -1148,8 +1179,7 @@ public class IndexMerger lhs.getTimestamp(), lhs.getDims(), metrics, - lhs.getRowNum(), - lhs.getDescriptions() + lhs.getRowNum() ); for (Rowboat rowboat : Arrays.asList(lhs, rhs)) { @@ -1163,15 +1193,4 @@ public class IndexMerger return retVal; } } - - public static interface ProgressIndicator - { - public void progress(); - } - - private static class NoopProgressIndicator implements ProgressIndicator - { - @Override - public void progress() {} - } } diff --git a/processing/src/main/java/io/druid/segment/IndexableAdapter.java b/processing/src/main/java/io/druid/segment/IndexableAdapter.java index a9c6f07f7a9..e676cee6e19 100644 --- a/processing/src/main/java/io/druid/segment/IndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/IndexableAdapter.java @@ -19,11 +19,14 @@ package io.druid.segment; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ValueType; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; import org.joda.time.Interval; /** + * An adapter to an index */ public interface IndexableAdapter { @@ -31,9 +34,9 @@ public interface IndexableAdapter int getNumRows(); - Indexed getAvailableDimensions(); + Indexed getDimensionNames(); - Indexed getAvailableMetrics(); + Indexed getMetricNames(); Indexed getDimValueLookup(String dimension); @@ -42,4 +45,6 @@ public interface IndexableAdapter IndexedInts getInverteds(String dimension, String value); String getMetricType(String metric); + + ColumnCapabilities getCapabilities(String column); } diff --git a/processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java b/processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java new file mode 100644 index 00000000000..39b1bcbd976 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java @@ -0,0 +1,99 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment; + +import com.google.common.base.Stopwatch; +import com.google.common.collect.Maps; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; + +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + */ +public class LoggingProgressIndicator extends BaseProgressIndicator +{ + private static Logger log = new Logger(LoggingProgressIndicator.class); + + private final String progressName; + private final Stopwatch global; + + private final Map sections = Maps.newHashMap(); + + public LoggingProgressIndicator(String progressName) + { + this.progressName = progressName; + this.global = Stopwatch.createUnstarted(); + } + + @Override + public void start() + { + log.info("Starting [%s]", progressName); + global.start(); + } + + @Override + public void stop() + { + long time = global.elapsed(TimeUnit.MILLISECONDS); + global.stop(); + + log.info("[%s] complete. Elapsed time: [%,d] millis", progressName, time); + } + + @Override + public void startSection(String section) + { + log.info("[%s]: Starting [%s]", progressName, section); + + Stopwatch sectionWatch = sections.get(section); + if (sectionWatch != null) { + throw new ISE("[%s]: Cannot start progress tracker for [%s]. It is already started.", progressName, section); + } + sectionWatch = Stopwatch.createStarted(); + sections.put(section, sectionWatch); + } + + @Override + public void progressSection(String section, String message) + { + Stopwatch sectionWatch = sections.get(section); + if (sectionWatch == null) { + throw new ISE("[%s]: Cannot progress tracker for [%s]. Nothing started.", progressName, section); + } + long time = sectionWatch.elapsed(TimeUnit.MILLISECONDS); + log.info("[%s]: [%s] : %s. Elapsed time: [%,d] millis", progressName, section, message, time); + } + + @Override + public void stopSection(String section) + { + Stopwatch sectionWatch = sections.remove(section); + if (sectionWatch == null) { + throw new ISE("[%s]: Cannot stop progress tracker for [%s]. Nothing started.", progressName, section); + } + long time = sectionWatch.elapsed(TimeUnit.MILLISECONDS); + sectionWatch.stop(); + + log.info("[%s]: [%s] has completed. Elapsed time: [%,d] millis", progressName, section, time); + } +} diff --git a/processing/src/main/java/io/druid/segment/TimestampColumnSelector.java b/processing/src/main/java/io/druid/segment/LongColumnSelector.java similarity index 92% rename from processing/src/main/java/io/druid/segment/TimestampColumnSelector.java rename to processing/src/main/java/io/druid/segment/LongColumnSelector.java index dc36f6a4064..a9db4734cd8 100644 --- a/processing/src/main/java/io/druid/segment/TimestampColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/LongColumnSelector.java @@ -21,7 +21,7 @@ package io.druid.segment; /** */ -public interface TimestampColumnSelector +public interface LongColumnSelector { - public long getTimestamp(); + public long get(); } diff --git a/processing/src/main/java/io/druid/segment/LongMetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/LongMetricColumnSerializer.java new file mode 100644 index 00000000000..bcb1a39d007 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/LongMetricColumnSerializer.java @@ -0,0 +1,81 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment; + +import com.google.common.io.Files; +import io.druid.segment.data.CompressedLongsSupplierSerializer; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.IOPeon; + +import java.io.File; +import java.io.IOException; + +/** + */ +public class LongMetricColumnSerializer implements MetricColumnSerializer +{ + private final String metricName; + private final IOPeon ioPeon; + private final File outDir; + + private CompressedLongsSupplierSerializer writer; + + public LongMetricColumnSerializer( + String metricName, + File outDir, + IOPeon ioPeon + ) + { + this.metricName = metricName; + this.ioPeon = ioPeon; + this.outDir = outDir; + } + + @Override + public void open() throws IOException + { + writer = CompressedLongsSupplierSerializer.create( + ioPeon, String.format("%s_little", metricName), IndexIO.BYTE_ORDER, + CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY + ); + + writer.open(); + } + + @Override + public void serialize(Object obj) throws IOException + { + long val = (obj == null) ? 0 : ((Number) obj).longValue(); + writer.add(val); + } + + @Override + public void close() throws IOException + { + final File outFile = IndexIO.makeMetricFile(outDir, metricName, IndexIO.BYTE_ORDER); + outFile.delete(); + MetricHolder.writeLongMetric( + Files.newOutputStreamSupplier(outFile, true), metricName, writer + ); + IndexIO.checkFileSize(outFile); + + writer = null; + } +} diff --git a/processing/src/main/java/io/druid/segment/MMappedIndexAdapter.java b/processing/src/main/java/io/druid/segment/MMappedIndexAdapter.java deleted file mode 100644 index 0af0a82bbc8..00000000000 --- a/processing/src/main/java/io/druid/segment/MMappedIndexAdapter.java +++ /dev/null @@ -1,197 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012, 2013 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package io.druid.segment; - -import com.google.common.collect.Maps; -import com.metamx.common.guava.CloseQuietly; -import io.druid.segment.data.ConciseCompressedIndexedInts; -import io.druid.segment.data.Indexed; -import io.druid.segment.data.IndexedFloats; -import io.druid.segment.data.IndexedInts; -import io.druid.segment.data.IndexedLongs; -import org.joda.time.Interval; - -import java.util.Iterator; -import java.util.Map; -import java.util.NoSuchElementException; - -/** - */ -public class MMappedIndexAdapter implements IndexableAdapter -{ - private final MMappedIndex index; - private final int numRows; - - public MMappedIndexAdapter(MMappedIndex index) - { - this.index = index; - - numRows = index.getReadOnlyTimestamps().size(); - } - - @Override - public Interval getDataInterval() - { - return index.getDataInterval(); - } - - @Override - public int getNumRows() - { - return numRows; - } - - @Override - public Indexed getAvailableDimensions() - { - return index.getAvailableDimensions(); - } - - @Override - public Indexed getAvailableMetrics() - { - return index.getAvailableMetrics(); - } - - @Override - public Indexed getDimValueLookup(String dimension) - { - return index.getDimValueLookup(dimension); - } - - @Override - public Iterable getRows() - { - return new Iterable() - { - @Override - public Iterator iterator() - { - return new Iterator() - { - final IndexedLongs timestamps = index.getReadOnlyTimestamps(); - final MetricHolder[] metrics; - final IndexedFloats[] floatMetrics; - final Map> dimensions; - - final int numMetrics = index.getAvailableMetrics().size(); - - int currRow = 0; - boolean done = false; - - { - dimensions = Maps.newLinkedHashMap(); - for (String dim : index.getAvailableDimensions()) { - dimensions.put(dim, index.getDimColumn(dim)); - } - - final Indexed availableMetrics = index.getAvailableMetrics(); - metrics = new MetricHolder[availableMetrics.size()]; - floatMetrics = new IndexedFloats[availableMetrics.size()]; - for (int i = 0; i < metrics.length; ++i) { - metrics[i] = index.getMetricHolder(availableMetrics.get(i)); - if (metrics[i].getType() == MetricHolder.MetricType.FLOAT) { - floatMetrics[i] = metrics[i].getFloatType(); - } - } - } - - @Override - public boolean hasNext() - { - final boolean hasNext = currRow < numRows; - if (!hasNext && !done) { - CloseQuietly.close(timestamps); - for (IndexedFloats floatMetric : floatMetrics) { - CloseQuietly.close(floatMetric); - } - 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()) { - IndexedInts dimVals = dimensions.get(dim).get(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) { - switch (metrics[i].getType()) { - case FLOAT: - metricArray[i] = floatMetrics[i].get(currRow); - break; - case COMPLEX: - metricArray[i] = metrics[i].getComplexType().get(currRow); - } - } - - Map descriptions = Maps.newHashMap(); - for (String spatialDim : index.getSpatialIndexes().keySet()) { - descriptions.put(spatialDim, "spatial"); - } - final Rowboat retVal = new Rowboat(timestamps.get(currRow), dims, metricArray, currRow, descriptions); - - ++currRow; - - return retVal; - } - - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - }; - } - }; - } - - @Override - public IndexedInts getInverteds(String dimension, String value) - { - return new ConciseCompressedIndexedInts(index.getInvertedIndex(dimension, value)); - } - - @Override - public String getMetricType(String metric) - { - MetricHolder holder = index.getMetricHolder(metric); - if (holder == null) { - return null; - } - return holder.getTypeName(); - } -} diff --git a/processing/src/main/java/io/druid/segment/MetricHolder.java b/processing/src/main/java/io/druid/segment/MetricHolder.java index 2627444e758..f5ccc83e190 100644 --- a/processing/src/main/java/io/druid/segment/MetricHolder.java +++ b/processing/src/main/java/io/druid/segment/MetricHolder.java @@ -28,10 +28,13 @@ import com.metamx.common.guava.CloseQuietly; import io.druid.common.utils.SerializerUtils; import io.druid.segment.data.CompressedFloatsIndexedSupplier; import io.druid.segment.data.CompressedFloatsSupplierSerializer; +import io.druid.segment.data.CompressedLongsIndexedSupplier; +import io.druid.segment.data.CompressedLongsSupplierSerializer; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedFloats; +import io.druid.segment.data.IndexedLongs; import io.druid.segment.data.ObjectStrategy; import io.druid.segment.serde.ComplexMetricSerde; import io.druid.segment.serde.ComplexMetrics; @@ -99,6 +102,16 @@ public class MetricHolder column.closeAndConsolidate(outSupplier); } + public static void writeLongMetric( + OutputSupplier outSupplier, String name, CompressedLongsSupplierSerializer column + ) throws IOException + { + ByteStreams.write(version, outSupplier); + serializerUtils.writeString(outSupplier, name); + serializerUtils.writeString(outSupplier, "long"); + column.closeAndConsolidate(outSupplier); + } + public static void writeToChannel(MetricHolder holder, WritableByteChannel out) throws IOException { out.write(ByteBuffer.wrap(version)); @@ -136,6 +149,9 @@ public class MetricHolder MetricHolder holder = new MetricHolder(metricName, typeName); switch (holder.type) { + case LONG: + holder.longType = CompressedLongsIndexedSupplier.fromByteBuffer(buf, ByteOrder.nativeOrder()); + break; case FLOAT: holder.floatType = CompressedFloatsIndexedSupplier.fromByteBuffer(buf, ByteOrder.nativeOrder()); break; @@ -163,18 +179,22 @@ public class MetricHolder public enum MetricType { + LONG, FLOAT, COMPLEX; static MetricType determineType(String typeName) { - if ("float".equalsIgnoreCase(typeName)) { + if ("long".equalsIgnoreCase(typeName)) { + return LONG; + } else if ("float".equalsIgnoreCase(typeName)) { return FLOAT; } return COMPLEX; } } + CompressedLongsIndexedSupplier longType = null; CompressedFloatsIndexedSupplier floatType = null; Indexed complexType = null; @@ -203,6 +223,12 @@ public class MetricHolder return type; } + public IndexedLongs getLongType() + { + assertType(MetricType.LONG); + return longType.get(); + } + public IndexedFloats getFloatType() { assertType(MetricType.FLOAT); @@ -217,9 +243,14 @@ public class MetricHolder public MetricHolder convertByteOrder(ByteOrder order) { + MetricHolder retVal; switch (type) { + case LONG: + retVal = new MetricHolder(name, typeName); + retVal.longType = longType.convertByteOrder(order); + return retVal; case FLOAT: - MetricHolder retVal = new MetricHolder(name, typeName); + retVal = new MetricHolder(name, typeName); retVal.floatType = floatType.convertByteOrder(order); return retVal; case COMPLEX: diff --git a/processing/src/main/java/io/druid/segment/ProgressIndicator.java b/processing/src/main/java/io/druid/segment/ProgressIndicator.java new file mode 100644 index 00000000000..619d69dab56 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/ProgressIndicator.java @@ -0,0 +1,37 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment; + +/** + */ +public interface ProgressIndicator +{ + public void progress(); + + public void start(); + + public void stop(); + + public void startSection(String section); + + public void progressSection(String section, String message); + + public void stopSection(String section); +} diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java index ef6dc2f6adc..6b20722360f 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java @@ -27,9 +27,12 @@ import com.metamx.common.guava.CloseQuietly; import com.metamx.common.logger.Logger; import io.druid.segment.column.BitmapIndex; import io.druid.segment.column.Column; +import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ComplexColumn; import io.druid.segment.column.DictionaryEncodedColumn; import io.druid.segment.column.GenericColumn; +import io.druid.segment.column.IndexedFloatsGenericColumn; +import io.druid.segment.column.IndexedLongsGenericColumn; import io.druid.segment.column.ValueType; import io.druid.segment.data.ArrayBasedIndexedInts; import io.druid.segment.data.ConciseCompressedIndexedInts; @@ -53,10 +56,8 @@ import java.util.Set; public class QueryableIndexIndexableAdapter implements IndexableAdapter { private static final Logger log = new Logger(QueryableIndexIndexableAdapter.class); - private final int numRows; private final QueryableIndex input; - private final List availableDimensions; public QueryableIndexIndexableAdapter(QueryableIndex input) @@ -94,18 +95,18 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter } @Override - public Indexed getAvailableDimensions() + public Indexed getDimensionNames() { - return new ListIndexed(availableDimensions, String.class); + return new ListIndexed<>(availableDimensions, String.class); } @Override - public Indexed getAvailableMetrics() + public Indexed getMetricNames() { final Set columns = Sets.newLinkedHashSet(input.getColumnNames()); - final HashSet dimensions = Sets.newHashSet(getAvailableDimensions()); + final HashSet dimensions = Sets.newHashSet(getDimensionNames()); - return new ListIndexed( + return new ListIndexed<>( Lists.newArrayList(Sets.difference(columns, dimensions)), String.class ); @@ -170,28 +171,30 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter { return new Iterator() { - final GenericColumn timestamps = input.getTimeColumn().getGenericColumn(); + final GenericColumn timestamps = input.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn(); final Object[] metrics; + final Map dimensions; - final int numMetrics = getAvailableMetrics().size(); + final int numMetrics = getMetricNames().size(); int currRow = 0; boolean done = false; { dimensions = Maps.newLinkedHashMap(); - for (String dim : getAvailableDimensions()) { + for (String dim : getDimensionNames()) { dimensions.put(dim, input.getColumn(dim).getDictionaryEncoding()); } - final Indexed availableMetrics = getAvailableMetrics(); + final Indexed availableMetrics = getMetricNames(); 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: + case LONG: metrics[i] = column.getGenericColumn(); break; case COMPLEX: @@ -247,21 +250,17 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter Object[] metricArray = new Object[numMetrics]; for (int i = 0; i < metricArray.length; ++i) { - if (metrics[i] instanceof GenericColumn) { + if (metrics[i] instanceof IndexedFloatsGenericColumn) { metricArray[i] = ((GenericColumn) metrics[i]).getFloatSingleValueRow(currRow); + } else if (metrics[i] instanceof IndexedLongsGenericColumn) { + metricArray[i] = ((GenericColumn) metrics[i]).getLongSingleValueRow(currRow); } else if (metrics[i] instanceof ComplexColumn) { metricArray[i] = ((ComplexColumn) metrics[i]).getRowValue(currRow); } } - Map descriptions = Maps.newHashMap(); - for (String columnName : input.getColumnNames()) { - if (input.getColumn(columnName).getSpatialIndex() != null) { - descriptions.put(columnName, "spatial"); - } - } final Rowboat retVal = new Rowboat( - timestamps.getLongSingleValueRow(currRow), dims, metricArray, currRow, descriptions + timestamps.getLongSingleValueRow(currRow), dims, metricArray, currRow ); ++currRow; @@ -305,10 +304,18 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter switch (type) { case FLOAT: return "float"; + case LONG: + return "long"; case COMPLEX: return column.getComplexColumn().getTypeName(); default: throw new ISE("Unknown type[%s]", type); } } + + @Override + public ColumnCapabilities getCapabilities(String column) + { + return input.getColumn(column).getCapabilities(); + } } diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index c174a64e20e..3af73449ec5 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -106,7 +106,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter { GenericColumn column = null; try { - column = index.getTimeColumn().getGenericColumn(); + column = index.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn(); return new DateTime(column.getLongSingleValueRow(0)); } finally { @@ -119,7 +119,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter { GenericColumn column = null; try { - column = index.getTimeColumn().getGenericColumn(); + column = index.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn(); return new DateTime(column.getLongSingleValueRow(column.length() - 1)); } finally { @@ -195,7 +195,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter final Map complexColumnCache = Maps.newHashMap(); final Map objectColumnCache = Maps.newHashMap(); - final GenericColumn timestamps = index.getTimeColumn().getGenericColumn(); + final GenericColumn timestamps = index.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn(); return Sequences.withBaggage( Sequences.map( @@ -258,19 +258,6 @@ public class QueryableIndexStorageAdapter implements StorageAdapter cursorOffset = initOffset.clone(); } - @Override - public TimestampColumnSelector makeTimestampColumnSelector() - { - return new TimestampColumnSelector() - { - @Override - public long getTimestamp() - { - return timestamps.getLongSingleValueRow(cursorOffset.getOffset()); - } - }; - } - @Override public DimensionSelector makeDimensionSelector(String dimension) { @@ -374,7 +361,8 @@ public class QueryableIndexStorageAdapter implements StorageAdapter if (cachedMetricVals == null) { Column holder = index.getColumn(metricName); - if (holder != null && holder.getCapabilities().getType() == ValueType.FLOAT) { + if (holder != null && (holder.getCapabilities().getType() == ValueType.FLOAT + || holder.getCapabilities().getType() == ValueType.LONG)) { cachedMetricVals = holder.getGenericColumn(); genericColumnCache.put(metricName, cachedMetricVals); } @@ -402,6 +390,43 @@ public class QueryableIndexStorageAdapter implements StorageAdapter }; } + @Override + public LongColumnSelector makeLongColumnSelector(String columnName) + { + final String metricName = columnName.toLowerCase(); + GenericColumn cachedMetricVals = genericColumnCache.get(metricName); + + if (cachedMetricVals == null) { + Column holder = index.getColumn(metricName); + if (holder != null && (holder.getCapabilities().getType() == ValueType.LONG + || holder.getCapabilities().getType() == ValueType.FLOAT)) { + cachedMetricVals = holder.getGenericColumn(); + genericColumnCache.put(metricName, cachedMetricVals); + } + } + + if (cachedMetricVals == null) { + return new LongColumnSelector() + { + @Override + public long get() + { + return 0L; + } + }; + } + + final GenericColumn metricVals = cachedMetricVals; + return new LongColumnSelector() + { + @Override + public long get() + { + return metricVals.getLongSingleValueRow(cursorOffset.getOffset()); + } + }; + } + @Override public ObjectColumnSelector makeObjectColumnSelector(String column) { @@ -657,7 +682,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter final Map complexColumnCache = Maps.newHashMap(); final Map objectColumnCache = Maps.newHashMap(); - final GenericColumn timestamps = index.getTimeColumn().getGenericColumn(); + final GenericColumn timestamps = index.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn(); return Sequences.withBaggage( Sequences.map( @@ -713,19 +738,6 @@ public class QueryableIndexStorageAdapter implements StorageAdapter currRow = initRow; } - @Override - public TimestampColumnSelector makeTimestampColumnSelector() - { - return new TimestampColumnSelector() - { - @Override - public long getTimestamp() - { - return timestamps.getLongSingleValueRow(currRow); - } - }; - } - @Override public DimensionSelector makeDimensionSelector(String dimension) { @@ -829,7 +841,8 @@ public class QueryableIndexStorageAdapter implements StorageAdapter if (cachedMetricVals == null) { Column holder = index.getColumn(metricName); - if (holder != null && holder.getCapabilities().getType() == ValueType.FLOAT) { + if (holder != null && (holder.getCapabilities().getType() == ValueType.LONG + || holder.getCapabilities().getType() == ValueType.FLOAT)) { cachedMetricVals = holder.getGenericColumn(); genericColumnCache.put(metricName, cachedMetricVals); } @@ -857,6 +870,43 @@ public class QueryableIndexStorageAdapter implements StorageAdapter }; } + @Override + public LongColumnSelector makeLongColumnSelector(String columnName) + { + final String metricName = columnName.toLowerCase(); + GenericColumn cachedMetricVals = genericColumnCache.get(metricName); + + if (cachedMetricVals == null) { + Column holder = index.getColumn(metricName); + if (holder != null && (holder.getCapabilities().getType() == ValueType.LONG + || holder.getCapabilities().getType() == ValueType.FLOAT)) { + cachedMetricVals = holder.getGenericColumn(); + genericColumnCache.put(metricName, cachedMetricVals); + } + } + + if (cachedMetricVals == null) { + return new LongColumnSelector() + { + @Override + public long get() + { + return 0L; + } + }; + } + + final GenericColumn metricVals = cachedMetricVals; + return new LongColumnSelector() + { + @Override + public long get() + { + return metricVals.getLongSingleValueRow(currRow); + } + }; + } + @Override public ObjectColumnSelector makeObjectColumnSelector(String column) { diff --git a/processing/src/main/java/io/druid/segment/Rowboat.java b/processing/src/main/java/io/druid/segment/Rowboat.java index f6367ffcd61..600ff370b0f 100644 --- a/processing/src/main/java/io/druid/segment/Rowboat.java +++ b/processing/src/main/java/io/druid/segment/Rowboat.java @@ -37,21 +37,17 @@ public class Rowboat implements Comparable private final int rowNum; private final Map> comprisedRows; - private Map columnDescriptor; - public Rowboat( long timestamp, int[][] dims, Object[] metrics, - int rowNum, - Map columnDescriptor + int rowNum ) { this.timestamp = timestamp; this.dims = dims; this.metrics = metrics; this.rowNum = rowNum; - this.columnDescriptor = columnDescriptor; this.comprisedRows = Maps.newHashMap(); } @@ -91,11 +87,6 @@ public class Rowboat implements Comparable return rowNum; } - public Map getDescriptions() - { - return columnDescriptor; - } - @Override public int compareTo(Rowboat rhs) { diff --git a/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java b/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java index 7cfa6c9c217..b2d122098f9 100644 --- a/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java @@ -21,6 +21,7 @@ package io.druid.segment; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; +import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; import org.joda.time.Interval; @@ -51,15 +52,15 @@ public class RowboatFilteringIndexAdapter implements IndexableAdapter } @Override - public Indexed getAvailableDimensions() + public Indexed getDimensionNames() { - return baseAdapter.getAvailableDimensions(); + return baseAdapter.getDimensionNames(); } @Override - public Indexed getAvailableMetrics() + public Indexed getMetricNames() { - return baseAdapter.getAvailableMetrics(); + return baseAdapter.getMetricNames(); } @Override @@ -85,4 +86,10 @@ public class RowboatFilteringIndexAdapter implements IndexableAdapter { return baseAdapter.getMetricType(metric); } + + @Override + public ColumnCapabilities getCapabilities(String column) + { + return baseAdapter.getCapabilities(column); + } } diff --git a/processing/src/main/java/io/druid/segment/SegmentMissingException.java b/processing/src/main/java/io/druid/segment/SegmentMissingException.java new file mode 100644 index 00000000000..aade5e560ca --- /dev/null +++ b/processing/src/main/java/io/druid/segment/SegmentMissingException.java @@ -0,0 +1,29 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment; + +import com.metamx.common.ISE; + +public class SegmentMissingException extends ISE +{ + public SegmentMissingException(String formatText, Object... arguments) { + super(String.format(formatText, arguments)); + } +} diff --git a/processing/src/main/java/io/druid/segment/SimpleQueryableIndex.java b/processing/src/main/java/io/druid/segment/SimpleQueryableIndex.java index 2e3bc85958f..c2b569dda78 100644 --- a/processing/src/main/java/io/druid/segment/SimpleQueryableIndex.java +++ b/processing/src/main/java/io/druid/segment/SimpleQueryableIndex.java @@ -19,6 +19,7 @@ package io.druid.segment; +import com.google.common.base.Preconditions; import com.metamx.common.io.smoosh.SmooshedFileMapper; import io.druid.segment.column.Column; import io.druid.segment.data.Indexed; @@ -34,24 +35,22 @@ 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; + private final Map columns; private final SmooshedFileMapper fileMapper; public SimpleQueryableIndex( Interval dataInterval, Indexed columnNames, Indexed dimNames, - Column timeColumn, - Map otherColumns, + Map columns, SmooshedFileMapper fileMapper ) { + Preconditions.checkNotNull(columns.get(Column.TIME_COLUMN_NAME)); this.dataInterval = dataInterval; this.columnNames = columnNames; this.availableDimensions = dimNames; - this.timeColumn = timeColumn; - this.otherColumns = otherColumns; + this.columns = columns; this.fileMapper = fileMapper; } @@ -64,7 +63,7 @@ public class SimpleQueryableIndex implements QueryableIndex @Override public int getNumRows() { - return timeColumn.getLength(); + return columns.get(Column.TIME_COLUMN_NAME).getLength(); } @Override @@ -79,16 +78,10 @@ public class SimpleQueryableIndex implements QueryableIndex return availableDimensions; } - @Override - public Column getTimeColumn() - { - return timeColumn; - } - @Override public Column getColumn(String columnName) { - return otherColumns.get(columnName); + return columns.get(columnName); } @Override diff --git a/processing/src/main/java/io/druid/segment/column/Column.java b/processing/src/main/java/io/druid/segment/column/Column.java index 14de694e149..9519b11978a 100644 --- a/processing/src/main/java/io/druid/segment/column/Column.java +++ b/processing/src/main/java/io/druid/segment/column/Column.java @@ -23,6 +23,7 @@ package io.druid.segment.column; */ public interface Column { + public static final String TIME_COLUMN_NAME = "__time"; public ColumnCapabilities getCapabilities(); public int getLength(); diff --git a/processing/src/main/java/io/druid/segment/column/ColumnCapabilities.java b/processing/src/main/java/io/druid/segment/column/ColumnCapabilities.java index fbb3e87fd31..d7f1b1efdca 100644 --- a/processing/src/main/java/io/druid/segment/column/ColumnCapabilities.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnCapabilities.java @@ -30,4 +30,6 @@ public interface ColumnCapabilities public boolean hasBitmapIndexes(); public boolean hasSpatialIndexes(); public boolean hasMultipleValues(); + + public ColumnCapabilitiesImpl merge(ColumnCapabilities other); } diff --git a/processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java b/processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java index f7b09d698c1..e0692d55765 100644 --- a/processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java @@ -20,6 +20,7 @@ package io.druid.segment.column; import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.common.ISE; /** */ @@ -109,4 +110,28 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities this.hasMultipleValues = hasMultipleValues; return this; } + + @Override + public ColumnCapabilitiesImpl merge(ColumnCapabilities other) + { + if (other == null) { + return this; + } + + if (type == null) { + type = other.getType(); + } + + if (!type.equals(other.getType())) { + throw new ISE("Cannot merge columns of type[%s] and [%s]", type, other.getType()); + } + + this.dictionaryEncoded |= other.isDictionaryEncoded(); + this.runLengthEncoded |= other.isRunLengthEncoded(); + this.hasInvertedIndexes |= other.hasBitmapIndexes(); + this.hasSpatialIndexes |= other.hasSpatialIndexes(); + this.hasMultipleValues |= other.hasMultipleValues(); + + return this; + } } diff --git a/processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java b/processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java index 8aa260e4e53..c66fac93d09 100644 --- a/processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java +++ b/processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java @@ -82,7 +82,7 @@ public class IndexedFloatsGenericColumn implements GenericColumn @Override public long getLongSingleValueRow(int rowNum) { - throw new UnsupportedOperationException(); + return (long) column.get(rowNum); } @Override diff --git a/processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java b/processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java index 368e7b11ebe..29c599ff08e 100644 --- a/processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java +++ b/processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java @@ -70,7 +70,7 @@ public class IndexedLongsGenericColumn implements GenericColumn @Override public float getFloatSingleValueRow(int rowNum) { - throw new UnsupportedOperationException(); + return (float) column.get(rowNum); } @Override diff --git a/processing/src/main/java/io/druid/segment/data/CompressedFloatBufferObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedFloatBufferObjectStrategy.java index 7a9906cd364..b0680b2ec23 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedFloatBufferObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedFloatBufferObjectStrategy.java @@ -28,14 +28,14 @@ import java.nio.FloatBuffer; /** */ -public class CompressedFloatBufferObjectStrategy extends CompressedObjectStrategy +public class CompressedFloatBufferObjectStrategy extends FixedSizeCompressedObjectStrategy { - public static CompressedFloatBufferObjectStrategy getBufferForOrder(ByteOrder order) + public static CompressedFloatBufferObjectStrategy getBufferForOrder(final ByteOrder order, final CompressionStrategy compression, final int sizePer) { - return new CompressedFloatBufferObjectStrategy(order); + return new CompressedFloatBufferObjectStrategy(order, compression, sizePer); } - private CompressedFloatBufferObjectStrategy(final ByteOrder order) + private CompressedFloatBufferObjectStrategy(final ByteOrder order, final CompressionStrategy compression, final int sizePer) { super( order, @@ -64,7 +64,9 @@ public class CompressedFloatBufferObjectStrategy extends CompressedObjectStrateg { return into.asFloatBuffer().put(from); } - } + }, + compression, + sizePer ); } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java index 5d003763225..58f57d1191a 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java @@ -40,22 +40,26 @@ import java.util.Iterator; */ public class CompressedFloatsIndexedSupplier implements Supplier { - public static final byte version = 0x1; + public static final byte LZF_VERSION = 0x1; + public static final byte version = 0x2; public static final int MAX_FLOATS_IN_BUFFER = (0xFFFF >> 2); private final int totalSize; private final int sizePer; private final GenericIndexed> baseFloatBuffers; + private final CompressedObjectStrategy.CompressionStrategy compression; CompressedFloatsIndexedSupplier( int totalSize, int sizePer, - GenericIndexed> baseFloatBuffers + GenericIndexed> baseFloatBuffers, + CompressedObjectStrategy.CompressionStrategy compression ) { this.totalSize = totalSize; this.sizePer = sizePer; this.baseFloatBuffers = baseFloatBuffers; + this.compression = compression; } public int size() @@ -151,7 +155,7 @@ public class CompressedFloatsIndexedSupplier implements Supplier public long getSerializedSize() { - return baseFloatBuffers.getSerializedSize() + 1 + 4 + 4; + return baseFloatBuffers.getSerializedSize() + 1 + 4 + 4 + 1; } public void writeToChannel(WritableByteChannel channel) throws IOException @@ -159,6 +163,7 @@ public class CompressedFloatsIndexedSupplier implements Supplier channel.write(ByteBuffer.wrap(new byte[]{version})); channel.write(ByteBuffer.wrap(Ints.toByteArray(totalSize))); channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer))); + channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); baseFloatBuffers.writeToChannel(channel); } @@ -167,7 +172,8 @@ public class CompressedFloatsIndexedSupplier implements Supplier return new CompressedFloatsIndexedSupplier( totalSize, sizePer, - GenericIndexed.fromIterable(baseFloatBuffers, CompressedFloatBufferObjectStrategy.getBufferForOrder(order)) + GenericIndexed.fromIterable(baseFloatBuffers, CompressedFloatBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)), + compression ); } @@ -189,23 +195,53 @@ public class CompressedFloatsIndexedSupplier implements Supplier byte versionFromBuffer = buffer.get(); if (versionFromBuffer == version) { + final int totalSize = buffer.getInt(); + final int sizePer = buffer.getInt(); + final CompressedObjectStrategy.CompressionStrategy compression = + CompressedObjectStrategy.CompressionStrategy.forId(buffer.get()); + return new CompressedFloatsIndexedSupplier( - buffer.getInt(), - buffer.getInt(), - GenericIndexed.read(buffer, CompressedFloatBufferObjectStrategy.getBufferForOrder(order)) + totalSize, + sizePer, + GenericIndexed.read( + buffer, + CompressedFloatBufferObjectStrategy.getBufferForOrder( + order, + compression, + sizePer + ) + ), + compression + ); + } else if (versionFromBuffer == LZF_VERSION) { + final int totalSize = buffer.getInt(); + final int sizePer = buffer.getInt(); + final CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.LZF; + return new CompressedFloatsIndexedSupplier( + totalSize, + sizePer, + GenericIndexed.read( + buffer, + CompressedFloatBufferObjectStrategy.getBufferForOrder( + order, + compression, + sizePer + ) + ), + compression ); } throw new IAE("Unknown version[%s]", versionFromBuffer); } - public static CompressedFloatsIndexedSupplier fromFloatBuffer(FloatBuffer buffer, final ByteOrder order) + public static CompressedFloatsIndexedSupplier fromFloatBuffer(FloatBuffer buffer, final ByteOrder order, CompressedObjectStrategy.CompressionStrategy compression) { - return fromFloatBuffer(buffer, MAX_FLOATS_IN_BUFFER, order); + return fromFloatBuffer(buffer, MAX_FLOATS_IN_BUFFER, order, compression); } public static CompressedFloatsIndexedSupplier fromFloatBuffer( - final FloatBuffer buffer, final int chunkFactor, final ByteOrder order + final FloatBuffer buffer, final int chunkFactor, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression ) { Preconditions.checkArgument( @@ -252,8 +288,9 @@ public class CompressedFloatsIndexedSupplier implements Supplier }; } }, - CompressedFloatBufferObjectStrategy.getBufferForOrder(order) - ) + CompressedFloatBufferObjectStrategy.getBufferForOrder(order, compression, chunkFactor) + ), + compression ); } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java index 6cb06631c78..c982f566cba 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java @@ -35,27 +35,29 @@ import java.nio.FloatBuffer; public class CompressedFloatsSupplierSerializer { public static CompressedFloatsSupplierSerializer create( - IOPeon ioPeon, final String filenameBase, final ByteOrder order + IOPeon ioPeon, final String filenameBase, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression ) throws IOException { - return create(ioPeon, filenameBase, CompressedFloatsIndexedSupplier.MAX_FLOATS_IN_BUFFER, order); + return create(ioPeon, filenameBase, CompressedFloatsIndexedSupplier.MAX_FLOATS_IN_BUFFER, order, compression); } public static CompressedFloatsSupplierSerializer create( - IOPeon ioPeon, final String filenameBase, final int sizePer, final ByteOrder order + IOPeon ioPeon, final String filenameBase, final int sizePer, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression ) throws IOException { final CompressedFloatsSupplierSerializer retVal = new CompressedFloatsSupplierSerializer( sizePer, new GenericIndexedWriter>( - ioPeon, filenameBase, CompressedFloatBufferObjectStrategy.getBufferForOrder(order) - ) + ioPeon, filenameBase, CompressedFloatBufferObjectStrategy.getBufferForOrder(order, compression, sizePer) + ), + compression ); return retVal; } private final int sizePer; private final GenericIndexedWriter> flattener; + private final CompressedObjectStrategy.CompressionStrategy compression; private int numInserted = 0; @@ -63,11 +65,13 @@ public class CompressedFloatsSupplierSerializer public CompressedFloatsSupplierSerializer( int sizePer, - GenericIndexedWriter> flattener + GenericIndexedWriter> flattener, + CompressedObjectStrategy.CompressionStrategy compression ) { this.sizePer = sizePer; this.flattener = flattener; + this.compression = compression; endBuffer = FloatBuffer.allocate(sizePer); endBuffer.mark(); @@ -109,6 +113,7 @@ public class CompressedFloatsSupplierSerializer out.write(CompressedFloatsIndexedSupplier.version); out.write(Ints.toByteArray(numInserted)); out.write(Ints.toByteArray(sizePer)); + out.write(new byte[]{compression.getId()}); ByteStreams.copy(flattener.combineStreams(), out); } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedLongBufferObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedLongBufferObjectStrategy.java index 823c86a0516..13fd264eba3 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongBufferObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongBufferObjectStrategy.java @@ -28,14 +28,14 @@ import java.nio.LongBuffer; /** */ -public class CompressedLongBufferObjectStrategy extends CompressedObjectStrategy +public class CompressedLongBufferObjectStrategy extends FixedSizeCompressedObjectStrategy { - public static CompressedLongBufferObjectStrategy getBufferForOrder(ByteOrder order) + public static CompressedLongBufferObjectStrategy getBufferForOrder(final ByteOrder order, final CompressionStrategy compression, final int sizePer) { - return new CompressedLongBufferObjectStrategy(order); + return new CompressedLongBufferObjectStrategy(order, compression, sizePer); } - private CompressedLongBufferObjectStrategy(final ByteOrder order) + private CompressedLongBufferObjectStrategy(final ByteOrder order, final CompressionStrategy compression, final int sizePer) { super( order, @@ -64,8 +64,9 @@ public class CompressedLongBufferObjectStrategy extends CompressedObjectStrategy { return into.asLongBuffer().put(from); } - } + }, + compression, + sizePer ); } - } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java index aa8b9590f54..578712d16f3 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java @@ -40,21 +40,25 @@ import java.util.Iterator; */ public class CompressedLongsIndexedSupplier implements Supplier { - public static final byte version = 0x1; + public static final byte LZF_VERSION = 0x1; + public static final byte version = 0x2; private final int totalSize; private final int sizePer; private final GenericIndexed> baseLongBuffers; + private final CompressedObjectStrategy.CompressionStrategy compression; CompressedLongsIndexedSupplier( int totalSize, int sizePer, - GenericIndexed> baseLongBuffers + GenericIndexed> baseLongBuffers, + CompressedObjectStrategy.CompressionStrategy compression ) { this.totalSize = totalSize; this.sizePer = sizePer; this.baseLongBuffers = baseLongBuffers; + this.compression = compression; } public int size() @@ -162,7 +166,7 @@ public class CompressedLongsIndexedSupplier implements Supplier public long getSerializedSize() { - return baseLongBuffers.getSerializedSize() + 1 + 4 + 4; + return baseLongBuffers.getSerializedSize() + 1 + 4 + 4 + 1; } public void writeToChannel(WritableByteChannel channel) throws IOException @@ -170,6 +174,7 @@ public class CompressedLongsIndexedSupplier implements Supplier channel.write(ByteBuffer.wrap(new byte[]{version})); channel.write(ByteBuffer.wrap(Ints.toByteArray(totalSize))); channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer))); + channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); baseLongBuffers.writeToChannel(channel); } @@ -178,7 +183,8 @@ public class CompressedLongsIndexedSupplier implements Supplier return new CompressedLongsIndexedSupplier( totalSize, sizePer, - GenericIndexed.fromIterable(baseLongBuffers, CompressedLongBufferObjectStrategy.getBufferForOrder(order)) + GenericIndexed.fromIterable(baseLongBuffers, CompressedLongBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)), + compression ); } @@ -195,23 +201,37 @@ public class CompressedLongsIndexedSupplier implements Supplier byte versionFromBuffer = buffer.get(); if (versionFromBuffer == version) { + final int totalSize = buffer.getInt(); + final int sizePer = buffer.getInt(); + final CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.forId(buffer.get()); return new CompressedLongsIndexedSupplier( - buffer.getInt(), - buffer.getInt(), - GenericIndexed.read(buffer, CompressedLongBufferObjectStrategy.getBufferForOrder(order)) + totalSize, + sizePer, + GenericIndexed.read(buffer, CompressedLongBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)), + compression + ); + } else if (versionFromBuffer == LZF_VERSION) { + final int totalSize = buffer.getInt(); + final int sizePer = buffer.getInt(); + final CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.LZF; + return new CompressedLongsIndexedSupplier( + totalSize, + sizePer, + GenericIndexed.read(buffer, CompressedLongBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)), + compression ); } throw new IAE("Unknown version[%s]", versionFromBuffer); } - public static CompressedLongsIndexedSupplier fromLongBuffer(LongBuffer buffer, final ByteOrder byteOrder) + public static CompressedLongsIndexedSupplier fromLongBuffer(LongBuffer buffer, final ByteOrder byteOrder, CompressedObjectStrategy.CompressionStrategy compression) { - return fromLongBuffer(buffer, 0xFFFF / Longs.BYTES, byteOrder); + return fromLongBuffer(buffer, 0xFFFF / Longs.BYTES, byteOrder, compression); } public static CompressedLongsIndexedSupplier fromLongBuffer( - final LongBuffer buffer, final int chunkFactor, final ByteOrder byteOrder + final LongBuffer buffer, final int chunkFactor, final ByteOrder byteOrder, CompressedObjectStrategy.CompressionStrategy compression ) { Preconditions.checkArgument( @@ -258,8 +278,9 @@ public class CompressedLongsIndexedSupplier implements Supplier }; } }, - CompressedLongBufferObjectStrategy.getBufferForOrder(byteOrder) - ) + CompressedLongBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkFactor) + ), + compression ); } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java index 4d89d8604d2..cf04cc44767 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java @@ -36,20 +36,23 @@ import java.nio.LongBuffer; public class CompressedLongsSupplierSerializer { public static CompressedLongsSupplierSerializer create( - IOPeon ioPeon, final String filenameBase, final ByteOrder order + IOPeon ioPeon, final String filenameBase, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression ) throws IOException { + final int sizePer = 0xFFFF / Longs.BYTES; final CompressedLongsSupplierSerializer retVal = new CompressedLongsSupplierSerializer( - 0xFFFF / Longs.BYTES, + sizePer, new GenericIndexedWriter>( - ioPeon, filenameBase, CompressedLongBufferObjectStrategy.getBufferForOrder(order) - ) + ioPeon, filenameBase, CompressedLongBufferObjectStrategy.getBufferForOrder(order, compression, sizePer) + ), + compression ); return retVal; } private final int sizePer; private final GenericIndexedWriter> flattener; + private final CompressedObjectStrategy.CompressionStrategy compression; private int numInserted = 0; @@ -57,11 +60,13 @@ public class CompressedLongsSupplierSerializer public CompressedLongsSupplierSerializer( int sizePer, - GenericIndexedWriter> flattener + GenericIndexedWriter> flattener, + CompressedObjectStrategy.CompressionStrategy compression ) { this.sizePer = sizePer; this.flattener = flattener; + this.compression = compression; endBuffer = LongBuffer.allocate(sizePer); endBuffer.mark(); @@ -103,6 +108,7 @@ public class CompressedLongsSupplierSerializer out.write(CompressedLongsIndexedSupplier.version); out.write(Ints.toByteArray(numInserted)); out.write(Ints.toByteArray(sizePer)); + out.write(new byte[]{compression.getId()}); ByteStreams.copy(flattener.combineStreams(), out); } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java index 4a0b5723b41..0796b60f66b 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java @@ -20,32 +20,218 @@ package io.druid.segment.data; import com.google.common.base.Throwables; +import com.google.common.collect.Maps; import com.metamx.common.guava.CloseQuietly; import com.ning.compress.lzf.ChunkEncoder; import com.ning.compress.lzf.LZFChunk; import com.ning.compress.lzf.LZFDecoder; import io.druid.collections.ResourceHolder; import io.druid.segment.CompressedPools; +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.lz4.LZ4FastDecompressor; +import net.jpountz.lz4.LZ4SafeDecompressor; import java.io.IOException; import java.nio.Buffer; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.util.Map; /** */ public class CompressedObjectStrategy implements ObjectStrategy> { - private final ByteOrder order; - private final BufferConverter converter; + public static final CompressionStrategy DEFAULT_COMPRESSION_STRATEGY = CompressionStrategy.LZ4; + + public static enum CompressionStrategy { + LZF ((byte)0x0) + { + @Override + public Decompressor getDecompressor() + { + return new LZFDecompressor(); + } + + @Override + public Compressor getCompressor() + { + return new LZFCompressor(); + } + }, + + LZ4 ((byte)0x1) { + @Override + public Decompressor getDecompressor() + { + return new LZ4Decompressor(); + } + + @Override + public Compressor getCompressor() + { + return new LZ4Compressor(); + } + }; + + final byte id; + + CompressionStrategy(byte id) { + this.id = id; + } + + public byte getId() + { + return id; + } + public abstract Compressor getCompressor(); + public abstract Decompressor getDecompressor(); + + static final Map idMap = Maps.newHashMap(); + static { + for(CompressionStrategy strategy : CompressionStrategy.values()) idMap.put(strategy.getId(), strategy); + } + + public static CompressionStrategy forId(byte id) + { + return idMap.get(id); + } + } + + public static interface Decompressor + { + /** + * Implementations of this method are expected to call out.flip() after writing to the output buffer + * + * @param in + * @param numBytes + * @param out + */ + public void decompress(ByteBuffer in, int numBytes, ByteBuffer out); + public void decompress(ByteBuffer in, int numBytes, ByteBuffer out, int decompressedSize); + } + + public static interface Compressor + { + /** + * Currently assumes buf is an array backed ByteBuffer + * + * @param bytes + * @return + */ + public byte[] compress(byte[] bytes); + } + + public static class LZFDecompressor implements Decompressor + { + @Override + public void decompress(ByteBuffer in, int numBytes, ByteBuffer out) + { + final byte[] bytes = new byte[numBytes]; + in.get(bytes); + + try (final ResourceHolder outputBytesHolder = CompressedPools.getOutputBytes()) { + final byte[] outputBytes = outputBytesHolder.get(); + final int numDecompressedBytes = LZFDecoder.decode(bytes, outputBytes); + out.put(outputBytes, 0, numDecompressedBytes); + out.flip(); + } + catch (IOException e) { + Throwables.propagate(e); + } + } + + @Override + public void decompress(ByteBuffer in, int numBytes, ByteBuffer out, int decompressedSize) + { + decompress(in, numBytes, out); + } + } + + public static class LZFCompressor implements Compressor + { + @Override + public byte[] compress(byte[] bytes) + { + final ResourceHolder encoder = CompressedPools.getChunkEncoder(); + LZFChunk chunk = encoder.get().encodeChunk(bytes, 0, bytes.length); + CloseQuietly.close(encoder); + + return chunk.getData(); + } + } + + public static class LZ4Decompressor implements Decompressor + { + private final LZ4SafeDecompressor lz4 = LZ4Factory.fastestJavaInstance().safeDecompressor(); + private final LZ4FastDecompressor lz4Fast = LZ4Factory.fastestJavaInstance().fastDecompressor(); + + @Override + public void decompress(ByteBuffer in, int numBytes, ByteBuffer out) + { + final byte[] bytes = new byte[numBytes]; + in.get(bytes); + + try (final ResourceHolder outputBytesHolder = CompressedPools.getOutputBytes()) { + final byte[] outputBytes = outputBytesHolder.get(); + final int numDecompressedBytes = lz4.decompress(bytes, 0, bytes.length, outputBytes, 0, outputBytes.length); + + out.put(outputBytes, 0, numDecompressedBytes); + out.flip(); + } + catch (IOException e) { + Throwables.propagate(e); + } + } + + @Override + public void decompress(ByteBuffer in, int numBytes, ByteBuffer out, int decompressedSize) + { + final byte[] bytes = new byte[numBytes]; + in.get(bytes); + + try (final ResourceHolder outputBytesHolder = CompressedPools.getOutputBytes()) { + final byte[] outputBytes = outputBytesHolder.get(); + lz4Fast.decompress(bytes, 0, outputBytes, 0, decompressedSize); + + out.put(outputBytes, 0, decompressedSize); + out.flip(); + } + catch (IOException e) { + Throwables.propagate(e); + } + } + } + + public static class LZ4Compressor implements Compressor + { + private final net.jpountz.lz4.LZ4Compressor lz4 = LZ4Factory.fastestJavaInstance().highCompressor(); + + @Override + public byte[] compress(byte[] bytes) + { + final byte[] intermediate = new byte[lz4.maxCompressedLength(bytes.length)]; + final int outputBytes = lz4.compress(bytes, 0, bytes.length, intermediate, 0, intermediate.length); + final byte[] out = new byte[outputBytes]; + System.arraycopy(intermediate, 0, out, 0, outputBytes); + return out; + } + } + + protected final ByteOrder order; + protected final BufferConverter converter; + protected final Decompressor decompressor; + private final Compressor compressor; protected CompressedObjectStrategy( final ByteOrder order, - final BufferConverter converter + final BufferConverter converter, + final CompressionStrategy compression ) { this.order = order; this.converter = converter; + this.decompressor = compression.getDecompressor(); + this.compressor = compression.getCompressor(); } @Override @@ -58,56 +244,49 @@ public class CompressedObjectStrategy implements ObjectStrateg @Override public ResourceHolder fromByteBuffer(ByteBuffer buffer, int numBytes) { - byte[] bytes = new byte[numBytes]; - buffer.get(bytes); - final ResourceHolder bufHolder = CompressedPools.getByteBuf(order); final ByteBuffer buf = bufHolder.get(); buf.position(0); buf.limit(buf.capacity()); - try { - final ResourceHolder outputBytesHolder = CompressedPools.getOutputBytes(); - - byte[] outputBytes = outputBytesHolder.get(); - int numDecompressedBytes = LZFDecoder.decode(bytes, outputBytes); - buf.put(outputBytes, 0, numDecompressedBytes); - buf.flip(); - - CloseQuietly.close(outputBytesHolder); - - return new ResourceHolder() + decompress(buffer, numBytes, buf); + return new ResourceHolder() + { + @Override + public T get() { - @Override - public T get() - { - return converter.convert(buf); - } + return converter.convert(buf); + } - @Override - public void close() throws IOException - { - bufHolder.close(); - } - }; - } - catch (IOException e) { - throw Throwables.propagate(e); - } + @Override + public void close() throws IOException + { + bufHolder.close(); + } + }; + } + + protected void decompress( + ByteBuffer buffer, + int numBytes, + ByteBuffer buf + ) + { + decompressor.decompress(buffer, numBytes, buf); } @Override public byte[] toBytes(ResourceHolder holder) { T val = holder.get(); - ByteBuffer buf = ByteBuffer.allocate(converter.sizeOf(val.remaining())).order(order); + ByteBuffer buf = bufferFor(val); converter.combine(buf, val); + return compressor.compress(buf.array()); + } - final ResourceHolder encoder = CompressedPools.getChunkEncoder(); - LZFChunk chunk = encoder.get().encodeChunk(buf.array(), 0, buf.array().length); - CloseQuietly.close(encoder); - - return chunk.getData(); + protected ByteBuffer bufferFor(T val) + { + return ByteBuffer.allocate(converter.sizeOf(val.remaining())).order(order); } @Override diff --git a/processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java new file mode 100644 index 00000000000..3efc1ba06ac --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java @@ -0,0 +1,56 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment.data; + +import java.nio.Buffer; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public abstract class FixedSizeCompressedObjectStrategy extends CompressedObjectStrategy +{ + private final int sizePer; + + protected FixedSizeCompressedObjectStrategy( + ByteOrder order, + BufferConverter converter, + CompressionStrategy compression, + int sizePer + ) + { + super(order, converter, compression); + this.sizePer = sizePer; + } + + public int getSize() { + return sizePer; + } + + @Override + protected ByteBuffer bufferFor(T val) + { + return ByteBuffer.allocate(converter.sizeOf(getSize())).order(order); + } + + @Override + protected void decompress(ByteBuffer buffer, int numBytes, ByteBuffer buf) + { + decompressor.decompress(buffer, numBytes, buf, converter.sizeOf(getSize())); + } +} diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java index bd6258b2a52..8f1a43b7b41 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java @@ -38,8 +38,6 @@ import java.util.Arrays; */ public class GenericIndexedWriter implements Closeable { - private static final byte[] EMPTY_ARRAY = new byte[]{}; - private final IOPeon ioPeon; private final String filenameBase; private final ObjectStrategy strategy; diff --git a/processing/src/main/java/io/druid/segment/data/InMemoryCompressedFloats.java b/processing/src/main/java/io/druid/segment/data/InMemoryCompressedFloats.java index d056158fa7d..d86a0f4364e 100644 --- a/processing/src/main/java/io/druid/segment/data/InMemoryCompressedFloats.java +++ b/processing/src/main/java/io/druid/segment/data/InMemoryCompressedFloats.java @@ -37,6 +37,7 @@ import java.util.List; */ public class InMemoryCompressedFloats implements IndexedFloats { + public static final CompressedObjectStrategy.CompressionStrategy COMPRESSION = CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY; private final CompressedFloatBufferObjectStrategy strategy; private final int sizePer; @@ -56,7 +57,11 @@ public class InMemoryCompressedFloats implements IndexedFloats ) { this.sizePer = sizePer; - strategy = CompressedFloatBufferObjectStrategy.getBufferForOrder(order); + strategy = CompressedFloatBufferObjectStrategy.getBufferForOrder( + order, + COMPRESSION, + sizePer + ); endBuffer = FloatBuffer.allocate(sizePer); endBuffer.mark(); @@ -184,7 +189,8 @@ public class InMemoryCompressedFloats implements IndexedFloats Arrays.>asList(StupidResourceHolder.create(endBufCopy)) ), strategy - ) + ), + COMPRESSION ); } diff --git a/processing/src/main/java/io/druid/segment/data/InMemoryCompressedLongs.java b/processing/src/main/java/io/druid/segment/data/InMemoryCompressedLongs.java index e0ef6fac375..266475636d3 100644 --- a/processing/src/main/java/io/druid/segment/data/InMemoryCompressedLongs.java +++ b/processing/src/main/java/io/druid/segment/data/InMemoryCompressedLongs.java @@ -38,6 +38,7 @@ import java.util.List; */ public class InMemoryCompressedLongs implements IndexedLongs { + public static final CompressedObjectStrategy.CompressionStrategy COMPRESSION = CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY; private final CompressedLongBufferObjectStrategy strategy; private final int sizePer; @@ -57,7 +58,11 @@ public class InMemoryCompressedLongs implements IndexedLongs ) { this.sizePer = sizePer; - strategy = CompressedLongBufferObjectStrategy.getBufferForOrder(order); + strategy = CompressedLongBufferObjectStrategy.getBufferForOrder( + order, + COMPRESSION, + sizePer + ); endBuffer = LongBuffer.allocate(sizePer); endBuffer.mark(); @@ -195,7 +200,8 @@ public class InMemoryCompressedLongs implements IndexedLongs Arrays.>asList(StupidResourceHolder.create(longBufCopy)) ), strategy - ) + ), + COMPRESSION ); } diff --git a/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java b/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java index a2c106ff309..c2f70b3a191 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java @@ -19,6 +19,7 @@ package io.druid.segment.data; +import com.google.common.collect.Lists; import com.google.common.primitives.Ints; import com.metamx.common.IAE; @@ -44,6 +45,11 @@ public class VSizeIndexedInts implements IndexedInts, ComparablenewArrayList(), 0); + } + public static VSizeIndexedInts fromList(List list, int maxValue) { int numBytes = getNumBytesForMax(maxValue); @@ -57,7 +63,7 @@ public class VSizeIndexedInts implements IndexedInts, Comparable maxValue) { throw new IAE("val[%d] > maxValue[%d], please don't lie about maxValue. i[%d]", val, maxValue, i); } - + byte[] intAsBytes = Ints.toByteArray(val); buffer.put(intAsBytes, intAsBytes.length - numBytes, numBytes); ++i; @@ -138,8 +144,8 @@ public class VSizeIndexedInts implements IndexedInts, Comparable +public class IncrementalIndex implements Iterable, Closeable { - private static final Logger log = new Logger(IncrementalIndex.class); - private static final Joiner JOINER = Joiner.on(","); private final long minTimestamp; private final QueryGranularity gran; + private final List> rowTransformers; private final AggregatorFactory[] metrics; private final Map metricIndexes; private final Map metricTypes; private final ImmutableList metricNames; + private final BufferAggregator[] aggs; + private final int[] aggPositionOffsets; + private final int totalAggSize; private final LinkedHashMap dimensionOrder; - private final CopyOnWriteArrayList dimensions; - private final List spatialDimensions; - private final SpatialDimensionRowFormatter spatialDimensionRowFormatter; + protected final CopyOnWriteArrayList dimensions; private final DimensionHolder dimValues; - private final ConcurrentSkipListMap facts; + private final Map columnCapabilities; + private final ConcurrentNavigableMap facts; + private final ResourceHolder bufferHolder; private volatile AtomicInteger numEntries = new AtomicInteger(); // This is modified on add() in a critical section. - private InputRow in; + private ThreadLocal in = new ThreadLocal<>(); - public IncrementalIndex(IncrementalIndexSchema incrementalIndexSchema) + /** + * Setting deserializeComplexMetrics to false is necessary for intermediate aggregation such as groupBy that + * should not deserialize input columns using ComplexMetricSerde for aggregators that return complex metrics. + * + * @param incrementalIndexSchema + * @param bufferPool + * @param deserializeComplexMetrics flag whether or not to call ComplexMetricExtractor.extractValue() on the input + * value for aggregators that return metrics other than float. + */ + public IncrementalIndex( + IncrementalIndexSchema incrementalIndexSchema, + StupidPool bufferPool, + final boolean deserializeComplexMetrics + ) { this.minTimestamp = incrementalIndexSchema.getMinTimestamp(); this.gran = incrementalIndexSchema.getGran(); this.metrics = incrementalIndexSchema.getMetrics(); + this.rowTransformers = Lists.newCopyOnWriteArrayList(); final ImmutableList.Builder metricNamesBuilder = ImmutableList.builder(); final ImmutableMap.Builder metricIndexesBuilder = ImmutableMap.builder(); final ImmutableMap.Builder metricTypesBuilder = ImmutableMap.builder(); + this.aggs = new BufferAggregator[metrics.length]; + this.aggPositionOffsets = new int[metrics.length]; + int currAggSize = 0; for (int i = 0; i < metrics.length; i++) { + final AggregatorFactory agg = metrics[i]; + aggs[i] = agg.factorizeBuffered( + new ColumnSelectorFactory() + { + @Override + public LongColumnSelector makeLongColumnSelector(String columnName) + { + if(columnName.equals(Column.TIME_COLUMN_NAME)){ + return new LongColumnSelector() + { + @Override + public long get() + { + return in.get().getTimestampFromEpoch(); + } + }; + } + final String metricName = columnName.toLowerCase(); + return new LongColumnSelector() + { + @Override + public long get() + { + return in.get().getLongMetric(metricName); + } + }; + } + + @Override + public FloatColumnSelector makeFloatColumnSelector(String columnName) + { + final String metricName = columnName.toLowerCase(); + return new FloatColumnSelector() + { + @Override + public float get() + { + return in.get().getFloatMetric(metricName); + } + }; + } + + @Override + public ObjectColumnSelector makeObjectColumnSelector(String column) + { + final String typeName = agg.getTypeName(); + final String columnName = column.toLowerCase(); + + final ObjectColumnSelector rawColumnSelector = new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return Object.class; + } + + @Override + public Object get() + { + return in.get().getRaw(columnName); + } + }; + + if (!deserializeComplexMetrics) { + return rawColumnSelector; + } else { + if (typeName.equals("float")) { + return rawColumnSelector; + } + + final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); + if (serde == null) { + throw new ISE("Don't know how to handle type[%s]", typeName); + } + + final ComplexMetricExtractor extractor = serde.getExtractor(); + return new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return extractor.extractedClass(); + } + + @Override + public Object get() + { + return extractor.extractValue(in.get(), columnName); + } + }; + } + } + + @Override + public DimensionSelector makeDimensionSelector(final String dimension) + { + final String dimensionName = dimension.toLowerCase(); + return new DimensionSelector() + { + @Override + public IndexedInts getRow() + { + final List dimensionValues = in.get().getDimension(dimensionName); + final ArrayList vals = Lists.newArrayList(); + if (dimensionValues != null) { + for (int i = 0; i < dimensionValues.size(); ++i) { + vals.add(i); + } + } + + return new IndexedInts() + { + @Override + public int size() + { + return vals.size(); + } + + @Override + public int get(int index) + { + return vals.get(index); + } + + @Override + public Iterator iterator() + { + return vals.iterator(); + } + }; + } + + @Override + public int getValueCardinality() + { + throw new UnsupportedOperationException("value cardinality is unknown in incremental index"); + } + + @Override + public String lookupName(int id) + { + return in.get().getDimension(dimensionName).get(id); + } + + @Override + public int lookupId(String name) + { + return in.get().getDimension(dimensionName).indexOf(name); + } + }; + } + } + ); + aggPositionOffsets[i] = currAggSize; + currAggSize += agg.getMaxIntermediateSize(); final String metricName = metrics[i].getName().toLowerCase(); metricNamesBuilder.add(metricName); metricIndexesBuilder.put(metricName, i); @@ -107,60 +289,120 @@ public class IncrementalIndex implements Iterable metricIndexes = metricIndexesBuilder.build(); metricTypes = metricTypesBuilder.build(); - this.dimensionOrder = Maps.newLinkedHashMap(); - this.dimensions = new CopyOnWriteArrayList(); - int index = 0; - for (String dim : incrementalIndexSchema.getDimensions()) { - dimensionOrder.put(dim, index++); - dimensions.add(dim); - } - this.spatialDimensions = incrementalIndexSchema.getSpatialDimensions(); - this.spatialDimensionRowFormatter = new SpatialDimensionRowFormatter(spatialDimensions); + this.totalAggSize = currAggSize; + this.dimensionOrder = Maps.newLinkedHashMap(); + this.dimensions = new CopyOnWriteArrayList<>(); + // This should really be more generic + List spatialDimensions = incrementalIndexSchema.getDimensionsSpec().getSpatialDimensions(); + if (!spatialDimensions.isEmpty()) { + this.rowTransformers.add(new SpatialDimensionRowTransformer(spatialDimensions)); + } + + this.columnCapabilities = Maps.newHashMap(); + for (Map.Entry entry : metricTypes.entrySet()) { + ValueType type; + if (entry.getValue().equalsIgnoreCase("float")) { + type = ValueType.FLOAT; + } else if (entry.getValue().equalsIgnoreCase("long")) { + type = ValueType.LONG; + } else { + type = ValueType.COMPLEX; + } + ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl(); + capabilities.setType(type); + columnCapabilities.put(entry.getKey(), capabilities); + } + for (String dimension : dimensions) { + ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl(); + capabilities.setType(ValueType.STRING); + columnCapabilities.put(dimension, capabilities); + } + for (SpatialDimensionSchema spatialDimension : spatialDimensions) { + ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl(); + capabilities.setType(ValueType.STRING); + capabilities.setHasSpatialIndexes(true); + columnCapabilities.put(spatialDimension.getDimName(), capabilities); + } + this.bufferHolder = bufferPool.take(); this.dimValues = new DimensionHolder(); - this.facts = new ConcurrentSkipListMap(); + this.facts = createFactsTable(); + } + + protected ConcurrentNavigableMap createFactsTable() { + return new ConcurrentSkipListMap<>(); } public IncrementalIndex( long minTimestamp, QueryGranularity gran, - final AggregatorFactory[] metrics + final AggregatorFactory[] metrics, + StupidPool bufferPool ) { this( new IncrementalIndexSchema.Builder().withMinTimestamp(minTimestamp) .withQueryGranularity(gran) .withMetrics(metrics) - .build() + .build(), + bufferPool, + true ); } - public int add(InputRow row) + public IncrementalIndex( + IncrementalIndexSchema incrementalIndexSchema, + StupidPool bufferPool + ) { - // this is an ugly workaround to call ComplexMetricExtractor.extractValue at ingestion time - return add(row, true); + this(incrementalIndexSchema, bufferPool, true); + } + + public IncrementalIndex( + long minTimestamp, + QueryGranularity gran, + final AggregatorFactory[] metrics, + StupidPool bufferPool, + boolean deserializeComplexMetrics + ) + { + this( + new IncrementalIndexSchema.Builder().withMinTimestamp(minTimestamp) + .withQueryGranularity(gran) + .withMetrics(metrics) + .build(), + bufferPool, + deserializeComplexMetrics + ); + } + + public InputRow formatRow(InputRow row) + { + for (Function rowTransformer : rowTransformers) { + row = rowTransformer.apply(row); + } + + if (row == null) { + throw new IAE("Row is null? How can this be?!"); + } + return row; } /** * Adds a new row. The row might correspond with another row that already exists, in which case this will * update that row instead of inserting a new one. - * - * + *

+ *

* Calls to add() are thread safe. - * - * Setting deserializeComplexMetrics to false is necessary for intermediate aggregation such as groupBy that - * should not deserialize input columns using ComplexMetricSerde for aggregators that return complex metrics. + *

* * @param row the row of data to add - * @param deserializeComplexMetrics flag whether or not to call ComplexMetricExtractor.extractValue() on the input - * value for aggregators that return metrics other than float. * * @return the number of rows in the data set after adding the InputRow */ - public int add(InputRow row, final boolean deserializeComplexMetrics) + public int add(InputRow row) { - row = spatialDimensionRowFormatter.formatRow(row); - + row = formatRow(row); if (row.getTimestampFromEpoch() < minTimestamp) { throw new IAE("Cannot add row[%s] because it is below the minTimestamp[%s]", row, new DateTime(minTimestamp)); } @@ -174,6 +416,18 @@ public class IncrementalIndex implements Iterable for (String dimension : rowDimensions) { dimension = dimension.toLowerCase(); List dimensionValues = row.getDimension(dimension); + + // Set column capabilities as data is coming in + ColumnCapabilitiesImpl capabilities = columnCapabilities.get(dimension); + if (capabilities == null) { + capabilities = new ColumnCapabilitiesImpl(); + capabilities.setType(ValueType.STRING); + columnCapabilities.put(dimension, capabilities); + } + if (dimensionValues.size() > 1) { + capabilities.setHasMultipleValues(true); + } + Integer index = dimensionOrder.get(dimension); if (index == null) { dimensionOrder.put(dimension, dimensionOrder.size()); @@ -201,173 +455,30 @@ public class IncrementalIndex implements Iterable } final TimeAndDims key = new TimeAndDims(Math.max(gran.truncate(row.getTimestampFromEpoch()), minTimestamp), dims); - - Aggregator[] aggs = facts.get(key); - if (aggs == null) { - aggs = new Aggregator[metrics.length]; - - for (int i = 0; i < metrics.length; ++i) { - final AggregatorFactory agg = metrics[i]; - aggs[i] = - agg.factorize( - new ColumnSelectorFactory() - { - @Override - public TimestampColumnSelector makeTimestampColumnSelector() - { - return new TimestampColumnSelector() - { - @Override - public long getTimestamp() - { - return in.getTimestampFromEpoch(); - } - }; - } - - @Override - public FloatColumnSelector makeFloatColumnSelector(String columnName) - { - final String metricName = columnName.toLowerCase(); - return new FloatColumnSelector() - { - @Override - public float get() - { - return in.getFloatMetric(metricName); - } - }; - } - - @Override - public ObjectColumnSelector makeObjectColumnSelector(String column) - { - final String typeName = agg.getTypeName(); - final String columnName = column.toLowerCase(); - - final ObjectColumnSelector rawColumnSelector = new ObjectColumnSelector() - { - @Override - public Class classOfObject() - { - return Object.class; - } - - @Override - public Object get() - { - return in.getRaw(columnName); - } - }; - - if(!deserializeComplexMetrics) { - return rawColumnSelector; - } else { - if (typeName.equals("float")) { - return rawColumnSelector; - } - - final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); - if (serde == null) { - throw new ISE("Don't know how to handle type[%s]", typeName); - } - - final ComplexMetricExtractor extractor = serde.getExtractor(); - return new ObjectColumnSelector() - { - @Override - public Class classOfObject() - { - return extractor.extractedClass(); - } - - @Override - public Object get() - { - return extractor.extractValue(in, columnName); - } - }; - } - } - - @Override - public DimensionSelector makeDimensionSelector(final String dimension) - { - final String dimensionName = dimension.toLowerCase(); - return new DimensionSelector() - { - @Override - public IndexedInts getRow() - { - final List dimensionValues = in.getDimension(dimensionName); - final ArrayList vals = Lists.newArrayList(); - if (dimensionValues != null) { - for (int i = 0; i < dimensionValues.size(); ++i) { - vals.add(i); - } - } - - return new IndexedInts() - { - @Override - public int size() - { - return vals.size(); - } - - @Override - public int get(int index) - { - return vals.get(index); - } - - @Override - public Iterator iterator() - { - return vals.iterator(); - } - }; - } - - @Override - public int getValueCardinality() - { - throw new UnsupportedOperationException("value cardinality is unknown in incremental index"); - } - - @Override - public String lookupName(int id) - { - return in.getDimension(dimensionName).get(id); - } - - @Override - public int lookupId(String name) - { - return in.getDimension(dimensionName).indexOf(name); - } - }; - } - } - - ); - } - - Aggregator[] prev = facts.putIfAbsent(key, aggs); - if (prev != null) { - aggs = prev; - } else { - numEntries.incrementAndGet(); - } - } - + Integer rowOffset; synchronized (this) { - in = row; - for (Aggregator agg : aggs) { - agg.aggregate(); + rowOffset = totalAggSize * numEntries.get(); + final Integer prev = facts.putIfAbsent(key, rowOffset); + if (prev != null) { + rowOffset = prev; + } else { + if (rowOffset + totalAggSize > bufferHolder.get().limit()) { + facts.remove(key); + throw new ISE("Buffer full, cannot add more rows! Current rowSize[%,d].", numEntries.get()); + } + numEntries.incrementAndGet(); + for (int i = 0; i < aggs.length; i++) { + aggs[i].init(bufferHolder.get(), getMetricPosition(rowOffset, i)); + } } - in = null; } + in.set(row); + for (int i = 0; i < aggs.length; i++) { + synchronized (aggs[i]) { + aggs[i].aggregate(bufferHolder.get(), getMetricPosition(rowOffset, i)); + } + } + in.set(null); return numEntries.get(); } @@ -398,11 +509,9 @@ public class IncrementalIndex implements Iterable int count = 0; for (String dimValue : dimValues) { String canonicalDimValue = dimLookup.get(dimValue); - if (canonicalDimValue == null) { - canonicalDimValue = dimValue; + if (!dimLookup.contains(canonicalDimValue)) { dimLookup.add(dimValue); } - retVal[count] = canonicalDimValue; count++; } @@ -421,16 +530,6 @@ public class IncrementalIndex implements Iterable return dimensions; } - public List getSpatialDimensions() - { - return spatialDimensions; - } - - public SpatialDimensionRowFormatter getSpatialDimensionRowFormatter() - { - return spatialDimensionRowFormatter; - } - public String getMetricType(String metric) { return metricTypes.get(metric); @@ -481,12 +580,32 @@ public class IncrementalIndex implements Iterable return metricIndexes.get(metricName); } - ConcurrentSkipListMap getFacts() + int getMetricPosition(int rowOffset, int metricIndex) + { + return rowOffset + aggPositionOffsets[metricIndex]; + } + + ByteBuffer getMetricBuffer() + { + return bufferHolder.get(); + } + + BufferAggregator getAggregator(int metricIndex) + { + return aggs[metricIndex]; + } + + ColumnCapabilities getCapabilities(String column) + { + return columnCapabilities.get(column); + } + + ConcurrentNavigableMap getFacts() { return facts; } - ConcurrentNavigableMap getSubMap(TimeAndDims start, TimeAndDims end) + ConcurrentNavigableMap getSubMap(TimeAndDims start, TimeAndDims end) { return facts.subMap(start, end); } @@ -506,13 +625,13 @@ public class IncrementalIndex implements Iterable { return Iterators.transform( facts.entrySet().iterator(), - new Function, Row>() + new Function, Row>() { @Override - public Row apply(final Map.Entry input) + public Row apply(final Map.Entry input) { final TimeAndDims timeAndDims = input.getKey(); - final Aggregator[] aggregators = input.getValue(); + final int rowOffset = input.getValue(); String[][] theDims = timeAndDims.getDims(); @@ -524,8 +643,8 @@ public class IncrementalIndex implements Iterable } } - for (int i = 0; i < aggregators.length; ++i) { - theVals.put(metrics[i].getName(), aggregators[i].get()); + for (int i = 0; i < aggs.length; ++i) { + theVals.put(metrics[i].getName(), aggs[i].get(bufferHolder.get(), getMetricPosition(rowOffset, i))); } if (postAggs != null) { @@ -542,7 +661,18 @@ public class IncrementalIndex implements Iterable }; } - static class DimensionHolder + @Override + public void close() + { + try { + bufferHolder.close(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + class DimensionHolder { private final Map dimensions; @@ -560,7 +690,7 @@ public class IncrementalIndex implements Iterable { DimDim holder = dimensions.get(dimension); if (holder == null) { - holder = new DimDim(); + holder = createDimDim(dimension); dimensions.put(dimension, holder); } else { throw new ISE("dimension[%s] already existed even though add() was called!?", dimension); @@ -574,6 +704,10 @@ public class IncrementalIndex implements Iterable } } + protected DimDim createDimDim(String dimension){ + return new DimDimImpl(); + } + static class TimeAndDims implements Comparable { private final long timestamp; @@ -659,26 +793,51 @@ public class IncrementalIndex implements Iterable } } - static class DimDim + static interface DimDim { - private final Map poorMansInterning = Maps.newConcurrentMap(); + public String get(String value); + + public int getId(String value); + + public String getValue(int id); + + public boolean contains(String value); + + public int size(); + + public int add(String value); + + public int getSortedId(String value); + + public String getSortedValue(int index); + + public void sort(); + + public boolean compareCannonicalValues(String s1, String s2); + } + + private static class DimDimImpl implements DimDim{ private final Map falseIds; private final Map falseIdsReverse; private volatile String[] sortedVals = null; + final ConcurrentMap poorMansInterning = Maps.newConcurrentMap(); - public DimDim() + + public DimDimImpl() { - BiMap biMap = Maps.synchronizedBiMap(HashBiMap.create()); - falseIds = biMap; - falseIdsReverse = biMap.inverse(); + BiMap biMap = Maps.synchronizedBiMap(HashBiMap.create()); + falseIds = biMap; + falseIdsReverse = biMap.inverse(); } - - // Returns the interned String value to allow fast comparisons using `==` instead of `.equals()` - // see io.druid.segment.incremental.IncrementalIndexStorageAdapter.EntryHolderValueMatcherFactory#makeValueMatcher(String, String) - public String get(String value) + /** + * Returns the interned String value to allow fast comparisons using `==` instead of `.equals()` + * @see io.druid.segment.incremental.IncrementalIndexStorageAdapter.EntryHolderValueMatcherFactory#makeValueMatcher(String, String) + */ + public String get(String str) { - return value == null ? null : poorMansInterning.get(value); + String prev = poorMansInterning.putIfAbsent(str, str); + return prev != null ? prev : str; } public int getId(String value) @@ -691,20 +850,21 @@ public class IncrementalIndex implements Iterable return falseIdsReverse.get(id); } + public boolean contains(String value) + { + return falseIds.containsKey(value); + } + public int size() { - return poorMansInterning.size(); + return falseIds.size(); } - public Set keySet() + public synchronized int add(String value) { - return poorMansInterning.keySet(); - } - - public synchronized void add(String value) - { - poorMansInterning.put(value, value); - falseIds.put(value, falseIds.size()); + int id = falseIds.size(); + falseIds.put(value, id); + return id; } public int getSortedId(String value) @@ -738,5 +898,10 @@ public class IncrementalIndex implements Iterable throw new ISE("Call sort() before calling the getSorted* methods."); } } + + public boolean compareCannonicalValues(String s1, String s2) + { + return s1 ==s2; + } } } diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java index b72530ba47a..1addc0551e1 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java @@ -23,10 +23,9 @@ import com.google.common.base.Function; import com.google.common.collect.Maps; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; -import io.druid.data.input.impl.SpatialDimensionSchema; -import io.druid.query.aggregation.Aggregator; import io.druid.segment.IndexableAdapter; import io.druid.segment.Rowboat; +import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.data.EmptyIndexedInts; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; @@ -45,10 +44,8 @@ import java.util.Map; public class IncrementalIndexAdapter implements IndexableAdapter { private static final Logger log = new Logger(IncrementalIndexAdapter.class); - private final Interval dataInterval; private final IncrementalIndex index; - private final Map> invertedIndexes; public IncrementalIndexAdapter( @@ -114,13 +111,13 @@ public class IncrementalIndexAdapter implements IndexableAdapter } @Override - public Indexed getAvailableDimensions() + public Indexed getDimensionNames() { return new ListIndexed(index.getDimensions(), String.class); } @Override - public Indexed getAvailableMetrics() + public Indexed getMetricNames() { return new ListIndexed(index.getMetricNames(), String.class); } @@ -171,18 +168,18 @@ public class IncrementalIndexAdapter implements IndexableAdapter return FunctionalIterable .create(index.getFacts().entrySet()) .transform( - new Function, Rowboat>() + new Function, Rowboat>() { int count = 0; @Override public Rowboat apply( - @Nullable Map.Entry input + @Nullable Map.Entry input ) { final IncrementalIndex.TimeAndDims timeAndDims = input.getKey(); final String[][] dimValues = timeAndDims.getDims(); - final Aggregator[] aggs = input.getValue(); + final int rowOffset = input.getValue(); int[][] dims = new int[dimValues.length][]; for (String dimension : index.getDimensions()) { @@ -205,21 +202,17 @@ public class IncrementalIndexAdapter implements IndexableAdapter } } - Object[] metrics = new Object[aggs.length]; - for (int i = 0; i < aggs.length; i++) { - metrics[i] = aggs[i].get(); + Object[] metrics = new Object[index.getMetricAggs().length]; + for (int i = 0; i < metrics.length; i++) { + metrics[i] = index.getAggregator(i) + .get(index.getMetricBuffer(), index.getMetricPosition(rowOffset, i)); } - Map description = Maps.newHashMap(); - for (SpatialDimensionSchema spatialDimensionSchema : index.getSpatialDimensions()) { - description.put(spatialDimensionSchema.getDimName(), "spatial"); - } return new Rowboat( timeAndDims.getTimestamp(), dims, metrics, - count++, - description + count++ ); } } @@ -289,4 +282,10 @@ public class IncrementalIndexAdapter implements IndexableAdapter { return index.getMetricType(metric); } + + @Override + public ColumnCapabilities getCapabilities(String column) + { + return index.getCapabilities(column); + } } diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexSchema.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexSchema.java index 4438116b638..7c97d69c5db 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexSchema.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexSchema.java @@ -19,39 +19,30 @@ package io.druid.segment.incremental; -import com.google.common.base.Function; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; +import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.InputRowParser; -import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; -import java.util.Collections; -import java.util.List; - /** */ public class IncrementalIndexSchema { private final long minTimestamp; private final QueryGranularity gran; - private final List dimensions; - private final List spatialDimensions; + private final DimensionsSpec dimensionsSpec; private final AggregatorFactory[] metrics; public IncrementalIndexSchema( long minTimestamp, QueryGranularity gran, - List dimensions, - List spatialDimensions, + DimensionsSpec dimensionsSpec, AggregatorFactory[] metrics ) { this.minTimestamp = minTimestamp; this.gran = gran; - this.dimensions = dimensions; - this.spatialDimensions = spatialDimensions; + this.dimensionsSpec = dimensionsSpec; this.metrics = metrics; } @@ -65,14 +56,9 @@ public class IncrementalIndexSchema return gran; } - public List getDimensions() + public DimensionsSpec getDimensionsSpec() { - return dimensions; - } - - public List getSpatialDimensions() - { - return spatialDimensions; + return dimensionsSpec; } public AggregatorFactory[] getMetrics() @@ -84,16 +70,14 @@ public class IncrementalIndexSchema { private long minTimestamp; private QueryGranularity gran; - private List dimensions; - private List spatialDimensions; + private DimensionsSpec dimensionsSpec; private AggregatorFactory[] metrics; public Builder() { this.minTimestamp = 0L; this.gran = QueryGranularity.NONE; - this.dimensions = Lists.newArrayList(); - this.spatialDimensions = Lists.newArrayList(); + this.dimensionsSpec = new DimensionsSpec(null, null, null); this.metrics = new AggregatorFactory[]{}; } @@ -109,44 +93,25 @@ public class IncrementalIndexSchema return this; } - public Builder withDimensions(Iterable dimensions) + public Builder withDimensionsSpec(DimensionsSpec dimensionsSpec) { - this.dimensions = Lists.newArrayList( - Iterables.transform( - dimensions, new Function() - { - @Override - public String apply(String input) - { - return input.toLowerCase(); - } - } - ) - ); - Collections.sort(this.dimensions); + this.dimensionsSpec = dimensionsSpec; return this; } - public Builder withSpatialDimensions(InputRowParser parser) + public Builder withDimensionsSpec(InputRowParser parser) { if (parser != null && parser.getParseSpec() != null - && parser.getParseSpec().getDimensionsSpec() != null - && parser.getParseSpec().getDimensionsSpec().getSpatialDimensions() != null) { - this.spatialDimensions = parser.getParseSpec().getDimensionsSpec().getSpatialDimensions(); + && parser.getParseSpec().getDimensionsSpec() != null) { + this.dimensionsSpec = parser.getParseSpec().getDimensionsSpec(); } else { - this.spatialDimensions = Lists.newArrayList(); + this.dimensionsSpec = new DimensionsSpec(null, null, null); } return this; } - public Builder withSpatialDimensions(List spatialDimensions) - { - this.spatialDimensions = spatialDimensions; - return this; - } - public Builder withMetrics(AggregatorFactory[] metrics) { this.metrics = metrics; @@ -156,7 +121,7 @@ public class IncrementalIndexSchema public IncrementalIndexSchema build() { return new IncrementalIndexSchema( - minTimestamp, gran, dimensions, spatialDimensions, metrics + minTimestamp, gran, dimensionsSpec, metrics ); } } diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java index 22d25fa579b..3d5fe33e4c3 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -30,7 +30,7 @@ import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.granularity.QueryGranularity; import io.druid.query.QueryInterruptedException; -import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.BufferAggregator; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherFactory; @@ -38,9 +38,10 @@ import io.druid.segment.Capabilities; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; import io.druid.segment.FloatColumnSelector; +import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.StorageAdapter; -import io.druid.segment.TimestampColumnSelector; +import io.druid.segment.column.Column; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.ListIndexed; @@ -62,7 +63,6 @@ import java.util.concurrent.ConcurrentNavigableMap; public class IncrementalIndexStorageAdapter implements StorageAdapter { private static final Splitter SPLITTER = Splitter.on(","); - private final IncrementalIndex index; public IncrementalIndexStorageAdapter( @@ -169,8 +169,8 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter return new Cursor() { - private Iterator> baseIter; - private ConcurrentNavigableMap cursorMap; + private Iterator> baseIter; + private ConcurrentNavigableMap cursorMap; final DateTime time; int numAdvanced = -1; boolean done; @@ -265,19 +265,6 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter done = !foundMatched && (cursorMap.size() == 0 || !baseIter.hasNext()); } - @Override - public TimestampColumnSelector makeTimestampColumnSelector() - { - return new TimestampColumnSelector() - { - @Override - public long getTimestamp() - { - return currEntry.getKey().getTimestamp(); - } - }; - } - @Override public DimensionSelector makeDimensionSelector(String dimension) { @@ -367,13 +354,60 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter } final int metricIndex = metricIndexInt; + final BufferAggregator agg = index.getAggregator(metricIndex); return new FloatColumnSelector() { @Override public float get() { - return currEntry.getValue()[metricIndex].getFloat(); + return agg.getFloat( + index.getMetricBuffer(), + index.getMetricPosition(currEntry.getValue(), metricIndex) + ); + } + }; + } + + @Override + public LongColumnSelector makeLongColumnSelector(String columnName) + { + final String metricName = columnName.toLowerCase(); + + if(metricName.equals(Column.TIME_COLUMN_NAME)){ + return new LongColumnSelector() + { + @Override + public long get() + { + return currEntry.getKey().getTimestamp(); + } + }; + } + final Integer metricIndexInt = index.getMetricIndex(metricName); + if (metricIndexInt == null) { + return new LongColumnSelector() + { + @Override + public long get() + { + return 0L; + } + }; + } + + final int metricIndex = metricIndexInt; + final BufferAggregator agg = index.getAggregator(metricIndex); + + return new LongColumnSelector() + { + @Override + public long get() + { + return agg.getLong( + index.getMetricBuffer(), + index.getMetricPosition(currEntry.getValue(), metricIndex) + ); } }; } @@ -388,7 +422,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter final int metricIndex = metricIndexInt; final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(index.getMetricType(columnName)); - + final BufferAggregator agg = index.getAggregator(metricIndex); return new ObjectColumnSelector() { @Override @@ -400,7 +434,10 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter @Override public Object get() { - return currEntry.getValue()[metricIndex].get(); + return agg.get( + index.getMetricBuffer(), + index.getMetricPosition(currEntry.getValue(), metricIndex) + ); } }; } @@ -427,11 +464,9 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter final String[] dimVals = dims[dimensionIndex]; if (dimVals.length == 1) { return dimVals[0]; - } - else if (dimVals.length == 0) { + } else if (dimVals.length == 0) { return null; - } - else { + } else { return dimVals; } } @@ -455,14 +490,14 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter private static class EntryHolder { - Map.Entry currEntry = null; + Map.Entry currEntry = null; - public Map.Entry get() + public Map.Entry get() { return currEntry; } - public void set(Map.Entry currEntry) + public void set(Map.Entry currEntry) { this.currEntry = currEntry; this.currEntry = currEntry; @@ -473,7 +508,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter return currEntry.getKey(); } - public Aggregator[] getValue() + public Integer getValue() { return currEntry.getValue(); } @@ -497,8 +532,8 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter if (dimIndexObject == null) { return new BooleanValueMatcher(false); } - String idObject = index.getDimension(dimension.toLowerCase()).get(value); - if (idObject == null) { + final IncrementalIndex.DimDim dimDim = index.getDimension(dimension.toLowerCase()); + if (!dimDim.contains(value)) { if (value == null || "".equals(value)) { final int dimIndex = dimIndexObject; @@ -519,7 +554,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter } final int dimIndex = dimIndexObject; - final String id = idObject; + final String id = dimDim.get(value); return new ValueMatcher() { @@ -532,9 +567,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter } for (String dimVal : dims[dimIndex]) { - // using == here instead of .equals() to speed up lookups made possible by - // io.druid.segment.incremental.IncrementalIndex.DimDim#poorMansInterning - if (id == dimVal) { + if (dimDim.compareCannonicalValues(id,dimVal)) { return true; } } diff --git a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java new file mode 100644 index 00000000000..7e048c37e70 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java @@ -0,0 +1,274 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment.incremental; + + +import com.metamx.common.ISE; +import io.druid.collections.StupidPool; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; +import org.mapdb.BTreeKeySerializer; +import org.mapdb.DB; +import org.mapdb.DBMaker; +import org.mapdb.Serializer; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.io.Serializable; +import java.lang.ref.WeakReference; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Map; +import java.util.UUID; +import java.util.WeakHashMap; +import java.util.concurrent.ConcurrentNavigableMap; + +public class OffheapIncrementalIndex extends IncrementalIndex +{ + private volatile DB db; + private volatile DB factsDb; + + public OffheapIncrementalIndex( + IncrementalIndexSchema incrementalIndexSchema, + StupidPool bufferPool + ) + { + super(incrementalIndexSchema, bufferPool); + } + + public OffheapIncrementalIndex( + long minTimestamp, + QueryGranularity gran, + final AggregatorFactory[] metrics, + StupidPool bufferPool, + boolean deserializeComplexMetrics + + ) + { + super(minTimestamp, gran, metrics, bufferPool, deserializeComplexMetrics); + } + + @Override + protected synchronized ConcurrentNavigableMap createFactsTable() + { + if (factsDb == null) { + final DBMaker dbMaker = DBMaker.newMemoryDirectDB() + .transactionDisable() + .asyncWriteEnable() + .cacheSoftRefEnable(); + factsDb = dbMaker.make(); + db = dbMaker.make(); + } + final TimeAndDimsSerializer timeAndDimsSerializer = new TimeAndDimsSerializer(this); + return factsDb.createTreeMap("__facts" + UUID.randomUUID()) + .keySerializer(timeAndDimsSerializer) + .comparator(timeAndDimsSerializer.getComparator()) + .valueSerializer(Serializer.INTEGER) + .make(); + } + + @Override + protected DimDim createDimDim(String dimension) + { + return new OffheapDimDim(dimension); + } + + public static class TimeAndDimsSerializer extends BTreeKeySerializer implements Serializable + { + private final TimeAndDimsComparator comparator; + private final transient IncrementalIndex incrementalIndex; + + TimeAndDimsSerializer(IncrementalIndex incrementalIndex) + { + this.comparator = new TimeAndDimsComparator(); + this.incrementalIndex = incrementalIndex; + } + + @Override + public void serialize(DataOutput out, int start, int end, Object[] keys) throws IOException + { + for (int i = start; i < end; i++) { + TimeAndDims timeAndDim = (TimeAndDims) keys[i]; + out.writeLong(timeAndDim.getTimestamp()); + out.writeInt(timeAndDim.getDims().length); + int index = 0; + for (String[] dims : timeAndDim.getDims()) { + if (dims == null) { + out.write(-1); + } else { + DimDim dimDim = incrementalIndex.getDimension(incrementalIndex.dimensions.get(index)); + out.writeInt(dims.length); + for (String value : dims) { + out.writeInt(dimDim.getId(value)); + } + } + index++; + } + } + } + + @Override + public Object[] deserialize(DataInput in, int start, int end, int size) throws IOException + { + Object[] ret = new Object[size]; + for (int i = start; i < end; i++) { + final long timeStamp = in.readLong(); + final String[][] dims = new String[in.readInt()][]; + for (int k = 0; k < dims.length; k++) { + int len = in.readInt(); + if (len != -1) { + DimDim dimDim = incrementalIndex.getDimension(incrementalIndex.dimensions.get(k)); + String[] col = new String[len]; + for (int l = 0; l < col.length; l++) { + col[l] = dimDim.get(dimDim.getValue(in.readInt())); + } + dims[k] = col; + } + } + ret[i] = new TimeAndDims(timeStamp, dims); + } + return ret; + } + + @Override + public Comparator getComparator() + { + return comparator; + } + } + + public static class TimeAndDimsComparator implements Comparator, Serializable + { + @Override + public int compare(Object o1, Object o2) + { + return ((TimeAndDims) o1).compareTo((TimeAndDims) o2); + } + } + + private class OffheapDimDim implements DimDim + { + private final Map falseIds; + private final Map falseIdsReverse; + private final WeakHashMap> cache = + new WeakHashMap(); + private volatile String[] sortedVals = null; + // size on MapDB is slow so maintain a count here + private volatile int size = 0; + + public OffheapDimDim(String dimension) + { + falseIds = db.createHashMap(dimension) + .keySerializer(Serializer.STRING) + .valueSerializer(Serializer.INTEGER) + .make(); + falseIdsReverse = db.createHashMap(dimension + "_inverse") + .keySerializer(Serializer.INTEGER) + .valueSerializer(Serializer.STRING) + .make(); + } + + /** + * Returns the interned String value to allow fast comparisons using `==` instead of `.equals()` + * + * @see io.druid.segment.incremental.IncrementalIndexStorageAdapter.EntryHolderValueMatcherFactory#makeValueMatcher(String, String) + */ + public String get(String str) + { + final WeakReference cached = cache.get(str); + if (cached != null) { + final String value = cached.get(); + if (value != null) { + return value; + } + } + cache.put(str, new WeakReference(str)); + return str; + } + + public int getId(String value) + { + return falseIds.get(value); + } + + public String getValue(int id) + { + return falseIdsReverse.get(id); + } + + public boolean contains(String value) + { + return falseIds.containsKey(value); + } + + public int size() + { + return size; + } + + public synchronized int add(String value) + { + int id = size++; + falseIds.put(value, id); + falseIdsReverse.put(id, value); + return id; + } + + public int getSortedId(String value) + { + assertSorted(); + return Arrays.binarySearch(sortedVals, value); + } + + public String getSortedValue(int index) + { + assertSorted(); + return sortedVals[index]; + } + + public void sort() + { + if (sortedVals == null) { + sortedVals = new String[falseIds.size()]; + + int index = 0; + for (String value : falseIds.keySet()) { + sortedVals[index++] = value; + } + Arrays.sort(sortedVals); + } + } + + private void assertSorted() + { + if (sortedVals == null) { + throw new ISE("Call sort() before calling the getSorted* methods."); + } + } + + public boolean compareCannonicalValues(String s1, String s2) + { + return s1.equals(s2); + } + } + +} diff --git a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java similarity index 94% rename from processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java rename to processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java index b9b4ce9cc74..cd21bb777be 100644 --- a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java +++ b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java @@ -44,7 +44,7 @@ import java.util.Set; /** * We throw away all invalid spatial dimensions */ -public class SpatialDimensionRowFormatter +public class SpatialDimensionRowTransformer implements Function { private static final Joiner JOINER = Joiner.on(","); private static final Splitter SPLITTER = Splitter.on(","); @@ -52,7 +52,7 @@ public class SpatialDimensionRowFormatter private final Map spatialDimensionMap; private final Set spatialPartialDimNames; - public SpatialDimensionRowFormatter(List spatialDimensions) + public SpatialDimensionRowTransformer(List spatialDimensions) { this.spatialDimensionMap = Maps.newHashMap(); for (SpatialDimensionSchema spatialDimension : spatialDimensions) { @@ -77,7 +77,8 @@ public class SpatialDimensionRowFormatter ); } - public InputRow formatRow(final InputRow row) + @Override + public InputRow apply(final InputRow row) { final Map> spatialLookup = Maps.newHashMap(); @@ -140,6 +141,12 @@ public class SpatialDimensionRowFormatter return row.getRaw(dimension); } + @Override + public long getLongMetric(String metric) + { + return row.getLongMetric(metric); + } + @Override public float getFloatMetric(String metric) { @@ -222,4 +229,4 @@ public class SpatialDimensionRowFormatter } return true; } -} +} \ No newline at end of file diff --git a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java index ccd08ddac62..c632085a5b0 100644 --- a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -43,13 +43,7 @@ import java.nio.channels.WritableByteChannel; */ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde { - @JsonCreator - public static DictionaryEncodedColumnPartSerde createDeserializer( - boolean singleValued - ) - { - return new DictionaryEncodedColumnPartSerde(); - } + private final boolean isSingleValued; private final GenericIndexed dictionary; private final VSizeIndexedInts singleValuedColumn; @@ -67,6 +61,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde ImmutableRTree spatialIndex ) { + this.isSingleValued = multiValCol == null; this.dictionary = dictionary; this.singleValuedColumn = singleValCol; this.multiValuedColumn = multiValCol; @@ -89,20 +84,25 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde this.size = size; } - private DictionaryEncodedColumnPartSerde() + @JsonCreator + public DictionaryEncodedColumnPartSerde( + @JsonProperty("isSingleValued") boolean isSingleValued + ) { - dictionary = null; - singleValuedColumn = null; - multiValuedColumn = null; - bitmaps = null; - spatialIndex = null; - size = 0; + this.isSingleValued = isSingleValued; + + this.dictionary = null; + this.singleValuedColumn = null; + this.multiValuedColumn = null; + this.bitmaps = null; + this.spatialIndex = null; + this.size = 0; } @JsonProperty private boolean isSingleValued() { - return singleValuedColumn != null; + return isSingleValued; } @Override @@ -114,14 +114,26 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde @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); + channel.write(ByteBuffer.wrap(new byte[]{(byte) (isSingleValued ? 0x0 : 0x1)})); + + if (dictionary != null) { + dictionary.writeToChannel(channel); } - bitmaps.writeToChannel(channel); + + if (isSingleValued()) { + if (singleValuedColumn != null) { + singleValuedColumn.writeToChannel(channel); + } + } else { + if (multiValuedColumn != null) { + multiValuedColumn.writeToChannel(channel); + } + } + + if (bitmaps != null) { + bitmaps.writeToChannel(channel); + } + if (spatialIndex != null) { ByteBufferSerializer.writeToChannel(spatialIndex, IndexedRTree.objectStrategy, channel); } diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index b6469893e4b..489c36e799f 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -35,6 +35,8 @@ import org.easymock.IAnswer; import org.junit.Assert; import org.junit.Test; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -100,13 +102,14 @@ public class ChainedExecutionQueryRunnerTest runner3 ) ); - + HashMap context = new HashMap(); final Sequence seq = chainedRunner.run( Druids.newTimeseriesQueryBuilder() .dataSource("test") .intervals("2014/2015") .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) - .build() + .build(), + context ); Future resultFuture = Executors.newFixedThreadPool(1).submit( @@ -207,14 +210,15 @@ public class ChainedExecutionQueryRunnerTest runner3 ) ); - + HashMap context = new HashMap(); final Sequence seq = chainedRunner.run( Druids.newTimeseriesQueryBuilder() .dataSource("test") .intervals("2014/2015") .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) .context(ImmutableMap.of("timeout", 100, "queryId", "test")) - .build() + .build(), + context ); Future resultFuture = Executors.newFixedThreadPool(1).submit( @@ -275,7 +279,7 @@ public class ChainedExecutionQueryRunnerTest } @Override - public Sequence run(Query query) + public Sequence run(Query query, Map context) { hasStarted = true; start.countDown(); diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index bb0556f4757..298b0afa343 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -194,9 +194,10 @@ public class QueryRunnerTestHelper ) throws IOException { - final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex(); + final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex(false); final QueryableIndex mMappedTestIndex = TestIndex.getMMappedTestIndex(); final QueryableIndex mergedRealtimeIndex = TestIndex.mergedRealtimeIndex(); + final IncrementalIndex rtIndexOffheap = TestIndex.getIncrementalTestIndex(true); return Arrays.asList( new Object[][]{ { @@ -207,6 +208,9 @@ public class QueryRunnerTestHelper }, { makeQueryRunner(factory, new QueryableIndexSegment(segmentId, mergedRealtimeIndex)) + }, + { + makeQueryRunner(factory, new IncrementalIndexSegment(rtIndexOffheap, segmentId)) } } ); @@ -218,9 +222,11 @@ public class QueryRunnerTestHelper ) throws IOException { - final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex(); + final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex(false); final QueryableIndex mMappedTestIndex = TestIndex.getMMappedTestIndex(); final QueryableIndex mergedRealtimeIndex = TestIndex.mergedRealtimeIndex(); + final IncrementalIndex rtIndexOffheap = TestIndex.getIncrementalTestIndex(true); + return Arrays.asList( new Object[][]{ { @@ -231,6 +237,9 @@ public class QueryRunnerTestHelper }, { makeUnionQueryRunner(factory, new QueryableIndexSegment(segmentId, mergedRealtimeIndex)) + }, + { + makeUnionQueryRunner(factory, new IncrementalIndexSegment(rtIndexOffheap, segmentId)) } } ); diff --git a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java new file mode 100644 index 00000000000..3adfed65c06 --- /dev/null +++ b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java @@ -0,0 +1,266 @@ +package io.druid.query; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; +import com.google.common.collect.MapMaker; +import com.google.common.collect.Maps; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.segment.SegmentMissingException; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +public class RetryQueryRunnerTest +{ + private final ObjectMapper jsonMapper = new DefaultObjectMapper(); + + final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory( + "idx", + "index" + ), + QueryRunnerTestHelper.qualityUniques + ) + ) + .build(); + + + @Test + public void testRunWithMissingSegments() throws Exception + { + Map context = new MapMaker().makeMap(); + context.put(RetryQueryRunner.MISSING_SEGMENTS_KEY, Lists.newArrayList()); + RetryQueryRunner> runner = new RetryQueryRunner<>( + new QueryRunner>() + { + @Override + public Sequence> run(Query query, Map context) + { + ((List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY)).add( + new SegmentDescriptor( + new Interval( + 178888, + 1999999 + ), "test", 1 + ) + ); + return Sequences.empty(); + } + }, + new RetryQueryRunnerConfig() + { + private int numTries = 0; + private boolean returnPartialResults = true; + + public int numTries() { return numTries; } + + public boolean returnPartialResults() { return returnPartialResults; } + }, + jsonMapper + ); + + Iterable> actualResults = Sequences.toList( + runner.run(query, context), + Lists.>newArrayList() + ); + + Assert.assertTrue( + "Should have one entry in the list of missing segments", + ((List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY)).size() == 1 + ); + Assert.assertTrue("Should return an empty sequence as a result", ((List) actualResults).size() == 0); + } + + + @Test + public void testRetry() throws Exception + { + Map context = new MapMaker().makeMap(); + context.put("count", 0); + context.put(RetryQueryRunner.MISSING_SEGMENTS_KEY, Lists.newArrayList()); + RetryQueryRunner> runner = new RetryQueryRunner<>( + new QueryRunner>() + { + @Override + public Sequence> run( + Query> query, + Map context + ) + { + if ((int) context.get("count") == 0) { + ((List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY)).add( + new SegmentDescriptor( + new Interval( + 178888, + 1999999 + ), "test", 1 + ) + ); + context.put("count", 1); + return Sequences.empty(); + } else { + return Sequences.simple( + Arrays.asList( + new Result<>( + new DateTime(), + new TimeseriesResultValue( + Maps.newHashMap() + ) + ) + ) + ); + } + } + }, + new RetryQueryRunnerConfig() + { + private int numTries = 1; + private boolean returnPartialResults = true; + + public int numTries() { return numTries; } + + public boolean returnPartialResults() { return returnPartialResults; } + }, + jsonMapper + ); + + Iterable> actualResults = Sequences.toList( + runner.run(query, context), + Lists.>newArrayList() + ); + + Assert.assertTrue("Should return a list with one element", ((List) actualResults).size() == 1); + Assert.assertTrue( + "Should have nothing in missingSegment list", + ((List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY)).size() == 0 + ); + } + + @Test + public void testRetryMultiple() throws Exception + { + Map context = new MapMaker().makeMap(); + context.put("count", 0); + context.put(RetryQueryRunner.MISSING_SEGMENTS_KEY, Lists.newArrayList()); + RetryQueryRunner> runner = new RetryQueryRunner<>( + new QueryRunner>() + { + @Override + public Sequence> run( + Query> query, + Map context + ) + { + if ((int) context.get("count") < 3) { + ((List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY)).add( + new SegmentDescriptor( + new Interval( + 178888, + 1999999 + ), "test", 1 + ) + ); + context.put("count", (int) context.get("count") + 1); + return Sequences.empty(); + } else { + return Sequences.simple( + Arrays.asList( + new Result<>( + new DateTime(), + new TimeseriesResultValue( + Maps.newHashMap() + ) + ) + ) + ); + } + } + }, + new RetryQueryRunnerConfig() + { + private int numTries = 4; + private boolean returnPartialResults = true; + + public int numTries() { return numTries; } + + public boolean returnPartialResults() { return returnPartialResults; } + }, + jsonMapper + ); + + Iterable> actualResults = Sequences.toList( + runner.run(query, context), + Lists.>newArrayList() + ); + + Assert.assertTrue("Should return a list with one element", ((List) actualResults).size() == 1); + Assert.assertTrue( + "Should have nothing in missingSegment list", + ((List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY)).size() == 0 + ); + } + + @Test(expected = SegmentMissingException.class) + public void testException() throws Exception + { + Map context = new MapMaker().makeMap(); + context.put(RetryQueryRunner.MISSING_SEGMENTS_KEY, Lists.newArrayList()); + RetryQueryRunner> runner = new RetryQueryRunner<>( + new QueryRunner>() + { + @Override + public Sequence> run( + Query> query, + Map context + ) + { + ((List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY)).add( + new SegmentDescriptor( + new Interval( + 178888, + 1999999 + ), "test", 1 + ) + ); + return Sequences.empty(); + } + }, + new RetryQueryRunnerConfig() + { + private int numTries = 1; + private boolean returnPartialResults = false; + + public int numTries() { return numTries; } + + public boolean returnPartialResults() { return returnPartialResults; } + }, + jsonMapper + ); + + Iterable> actualResults = Sequences.toList( + runner.run(query, context), + Lists.>newArrayList() + ); + + Assert.assertTrue( + "Should have one entry in the list of missing segments", + ((List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY)).size() == 1 + ); + } +} diff --git a/processing/src/test/java/io/druid/query/TestQueryRunners.java b/processing/src/test/java/io/druid/query/TestQueryRunners.java index 1ff948e15ea..d5cdb079954 100644 --- a/processing/src/test/java/io/druid/query/TestQueryRunners.java +++ b/processing/src/test/java/io/druid/query/TestQueryRunners.java @@ -26,11 +26,10 @@ public class TestQueryRunners @Override public ByteBuffer get() { - return ByteBuffer.allocate(1024 * 10); + return ByteBuffer.allocate(1024 * 1024 * 10); } } ); - public static final TopNQueryConfig topNConfig = new TopNQueryConfig(); public static StupidPool getPool() diff --git a/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java b/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java index dc17ba61cb3..73f5c39dcb6 100644 --- a/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java +++ b/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java @@ -22,6 +22,7 @@ package io.druid.query; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.query.aggregation.AggregatorFactory; @@ -35,9 +36,13 @@ import org.junit.Assert; import org.junit.Test; import java.util.Arrays; +import java.util.Map; + public class TimewarpOperatorTest { + public static final ImmutableMap CONTEXT = ImmutableMap.of(); + TimewarpOperator> testOperator = new TimewarpOperator<>( new Interval(new DateTime("2014-01-01"), new DateTime("2014-01-15")), new Period("P1W"), @@ -75,7 +80,10 @@ public class TimewarpOperatorTest new QueryRunner>() { @Override - public Sequence> run(Query> query) + public Sequence> run( + Query> query, + Map context + ) { return Sequences.simple( ImmutableList.of( @@ -120,7 +128,7 @@ public class TimewarpOperatorTest new TimeseriesResultValue(ImmutableMap.of("metric", 5)) ) ), - Sequences.toList(queryRunner.run(query), Lists.>newArrayList()) + Sequences.toList(queryRunner.run(query, CONTEXT), Lists.>newArrayList()) ); @@ -134,13 +142,21 @@ public class TimewarpOperatorTest new QueryRunner>() { @Override - public Sequence> run(Query> query) + public Sequence> run( + Query> query, + Map context + ) { return Sequences.simple( ImmutableList.of( new Result<>( new DateTime("2014-01-12"), - new TimeBoundaryResultValue(ImmutableMap.of("maxTime", new DateTime("2014-01-12"))) + new TimeBoundaryResultValue( + ImmutableMap.of( + "maxTime", + new DateTime("2014-01-12") + ) + ) ) ) ); @@ -161,7 +177,10 @@ public class TimewarpOperatorTest new TimeBoundaryResultValue(ImmutableMap.of("maxTime", new DateTime("2014-08-02"))) ) ), - Sequences.toList(timeBoundaryRunner.run(timeBoundaryQuery), Lists.>newArrayList()) + Sequences.toList( + timeBoundaryRunner.run(timeBoundaryQuery, CONTEXT), + Lists.>newArrayList() + ) ); } @@ -173,7 +192,10 @@ public class TimewarpOperatorTest new QueryRunner>() { @Override - public Sequence> run(Query> query) + public Sequence> run( + Query> query, + Map context + ) { return Sequences.simple( ImmutableList.of( @@ -210,7 +232,7 @@ public class TimewarpOperatorTest new TimeseriesResultValue(ImmutableMap.of("metric", 3)) ) ), - Sequences.toList(queryRunner.run(query), Lists.>newArrayList()) + Sequences.toList(queryRunner.run(query, Maps.newHashMap()), Lists.>newArrayList()) ); } } diff --git a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java index b50ebb810f6..d1e0cf11dde 100644 --- a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java @@ -24,8 +24,8 @@ import io.druid.query.filter.SelectorDimFilter; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionSelector; import io.druid.segment.FloatColumnSelector; +import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; -import io.druid.segment.TimestampColumnSelector; import io.druid.segment.data.ArrayBasedIndexedInts; import io.druid.segment.data.IndexedInts; import org.junit.Assert; @@ -68,12 +68,6 @@ public class FilteredAggregatorTest return new ColumnSelectorFactory() { - @Override - public TimestampColumnSelector makeTimestampColumnSelector() - { - throw new UnsupportedOperationException(); - } - @Override public DimensionSelector makeDimensionSelector(String dimensionName) { @@ -127,6 +121,12 @@ public class FilteredAggregatorTest } } + @Override + public LongColumnSelector makeLongColumnSelector(String columnName) + { + throw new UnsupportedOperationException(); + } + @Override public FloatColumnSelector makeFloatColumnSelector(String columnName) { diff --git a/processing/src/test/java/io/druid/query/aggregation/LongSumAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/LongSumAggregatorTest.java index 294eef07b24..046e836248b 100644 --- a/processing/src/test/java/io/druid/query/aggregation/LongSumAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/LongSumAggregatorTest.java @@ -28,7 +28,7 @@ import java.util.Comparator; */ public class LongSumAggregatorTest { - private void aggregate(TestFloatColumnSelector selector, LongSumAggregator agg) + private void aggregate(TestLongColumnSelector selector, LongSumAggregator agg) { agg.aggregate(); selector.increment(); @@ -37,7 +37,7 @@ public class LongSumAggregatorTest @Test public void testAggregate() { - final TestFloatColumnSelector selector = new TestFloatColumnSelector(new float[]{24.15f, 20f}); + final TestLongColumnSelector selector = new TestLongColumnSelector(new long[]{24L, 20L}); LongSumAggregator agg = new LongSumAggregator("billy", selector); Assert.assertEquals("billy", agg.getName()); @@ -58,7 +58,7 @@ public class LongSumAggregatorTest @Test public void testComparator() { - final TestFloatColumnSelector selector = new TestFloatColumnSelector(new float[]{18293f}); + final TestLongColumnSelector selector = new TestLongColumnSelector(new long[]{18293L}); LongSumAggregator agg = new LongSumAggregator("billy", selector); Assert.assertEquals("billy", agg.getName()); diff --git a/processing/src/test/java/io/druid/query/aggregation/TestLongColumnSelector.java b/processing/src/test/java/io/druid/query/aggregation/TestLongColumnSelector.java new file mode 100644 index 00000000000..398409b2717 --- /dev/null +++ b/processing/src/test/java/io/druid/query/aggregation/TestLongColumnSelector.java @@ -0,0 +1,48 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.aggregation; + +import io.druid.segment.FloatColumnSelector; +import io.druid.segment.LongColumnSelector; + +/** + */ +public class TestLongColumnSelector implements LongColumnSelector +{ + private final long[] longs; + + private int index = 0; + + public TestLongColumnSelector(long[] longs) + { + this.longs = longs; + } + + @Override + public long get() + { + return longs[index]; + } + + public void increment() + { + ++index; + } +} diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 772d3b0d18d..2babeadb78a 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -26,6 +26,7 @@ import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; @@ -37,6 +38,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.JavaScriptAggregatorFactory; @@ -79,6 +81,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.Comparator; import java.util.List; +import java.util.Map; @RunWith(Parameterized.class) public class GroupByQueryRunnerTest @@ -118,7 +121,8 @@ public class GroupByQueryRunnerTest engine, QueryRunnerTestHelper.NOOP_QUERYWATCHER, configSupplier, - new GroupByQueryQueryToolChest(configSupplier, mapper, engine) + new GroupByQueryQueryToolChest(configSupplier, mapper, engine, TestQueryRunners.pool), + TestQueryRunners.pool ); GroupByQueryConfig singleThreadedConfig = new GroupByQueryConfig() @@ -138,7 +142,8 @@ public class GroupByQueryRunnerTest singleThreadEngine, QueryRunnerTestHelper.NOOP_QUERYWATCHER, singleThreadedConfigSupplier, - new GroupByQueryQueryToolChest(singleThreadedConfigSupplier, mapper, singleThreadEngine) + new GroupByQueryQueryToolChest(singleThreadedConfigSupplier, mapper, singleThreadEngine, pool), + pool ); @@ -358,25 +363,169 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "automotive", "rows", 1L, "idx", 135L), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "business", "rows", 1L, "idx", 118L), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "entertainment", "rows", 1L, "idx", 158L), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "health", "rows", 1L, "idx", 120L), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "mezzanine", "rows", 3L, "idx", 2870L), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "news", "rows", 1L, "idx", 121L), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "premium", "rows", 3L, "idx", 2900L), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "technology", "rows", 1L, "idx", 78L), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-03-31", tz), "alias", "travel", "rows", 1L, "idx", 119L), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-03-31", tz), + "alias", + "automotive", + "rows", + 1L, + "idx", + 135L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-03-31", tz), + "alias", + "business", + "rows", + 1L, + "idx", + 118L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-03-31", tz), + "alias", + "entertainment", + "rows", + 1L, + "idx", + 158L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-03-31", tz), + "alias", + "health", + "rows", + 1L, + "idx", + 120L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-03-31", tz), + "alias", + "mezzanine", + "rows", + 3L, + "idx", + 2870L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-03-31", tz), + "alias", + "news", + "rows", + 1L, + "idx", + 121L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-03-31", tz), + "alias", + "premium", + "rows", + 3L, + "idx", + 2900L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-03-31", tz), + "alias", + "technology", + "rows", + 1L, + "idx", + 78L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-03-31", tz), + "alias", + "travel", + "rows", + 1L, + "idx", + 119L + ), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "automotive", "rows", 1L, "idx", 147L), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "business", "rows", 1L, "idx", 112L), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "entertainment", "rows", 1L, "idx", 166L), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "health", "rows", 1L, "idx", 113L), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "mezzanine", "rows", 3L, "idx", 2447L), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "news", "rows", 1L, "idx", 114L), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "premium", "rows", 3L, "idx", 2505L), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "technology", "rows", 1L, "idx", 97L), - GroupByQueryRunnerTestHelper.createExpectedRow(new DateTime("2011-04-01", tz), "alias", "travel", "rows", 1L, "idx", 126L) + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-04-01", tz), + "alias", + "automotive", + "rows", + 1L, + "idx", + 147L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-04-01", tz), + "alias", + "business", + "rows", + 1L, + "idx", + 112L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-04-01", tz), + "alias", + "entertainment", + "rows", + 1L, + "idx", + 166L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-04-01", tz), + "alias", + "health", + "rows", + 1L, + "idx", + 113L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-04-01", tz), + "alias", + "mezzanine", + "rows", + 3L, + "idx", + 2447L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-04-01", tz), + "alias", + "news", + "rows", + 1L, + "idx", + 114L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-04-01", tz), + "alias", + "premium", + "rows", + 3L, + "idx", + 2505L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-04-01", tz), + "alias", + "technology", + "rows", + 1L, + "idx", + 97L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + new DateTime("2011-04-01", tz), + "alias", + "travel", + "rows", + 1L, + "idx", + 126L + ) ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -406,7 +555,9 @@ public class GroupByQueryRunnerTest new QueryRunner() { @Override - public Sequence run(Query query) + public Sequence run( + Query query, Map context + ) { // simulate two daily segments final Query query1 = query.withQuerySegmentSpec( @@ -415,7 +566,7 @@ public class GroupByQueryRunnerTest 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)); + return Sequences.concat(runner.run(query1, context), runner.run(query2, context)); } } ); @@ -432,8 +583,9 @@ public class GroupByQueryRunnerTest GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) ); - TestHelper.assertExpectedObjects(expectedResults, runner.run(fullQuery), "direct"); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery), "merged"); + Map context = Maps.newHashMap(); + TestHelper.assertExpectedObjects(expectedResults, runner.run(fullQuery, context), "direct"); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged"); List allGranExpectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L), @@ -447,8 +599,8 @@ public class GroupByQueryRunnerTest GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) ); - TestHelper.assertExpectedObjects(allGranExpectedResults, runner.run(allGranQuery), "direct"); - TestHelper.assertExpectedObjects(allGranExpectedResults, mergedRunner.run(allGranQuery), "merged"); + TestHelper.assertExpectedObjects(allGranExpectedResults, runner.run(allGranQuery, context), "direct"); + TestHelper.assertExpectedObjects(allGranExpectedResults, mergedRunner.run(allGranQuery, context), "merged"); } @Test @@ -491,8 +643,9 @@ public class GroupByQueryRunnerTest QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); + Map context = Maps.newHashMap(); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery), String.format("limit: %d", limit) + Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery, context), String.format("limit: %d", limit) ); } @@ -598,7 +751,9 @@ public class GroupByQueryRunnerTest new QueryRunner() { @Override - public Sequence run(Query query) + public Sequence run( + Query query, Map context + ) { // simulate two daily segments final Query query1 = query.withQuerySegmentSpec( @@ -607,12 +762,13 @@ public class GroupByQueryRunnerTest 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)); + return Sequences.concat(runner.run(query1, context), runner.run(query2, context)); } } ); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery), "merged"); + Map context = Maps.newHashMap(); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged"); } @Test @@ -647,11 +803,12 @@ public class GroupByQueryRunnerTest GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L) ); + Map context = Maps.newHashMap(); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build()), "limited" + Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), context), "limited" ); } @@ -687,10 +844,11 @@ public class GroupByQueryRunnerTest GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L) ); + Map context = Maps.newHashMap(); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build()), "limited" + Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), context), "limited" ); } @@ -715,21 +873,94 @@ public class GroupByQueryRunnerTest final GroupByQuery query = builder.build(); List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4423.6533203125D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4418.61865234375D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 319.94403076171875D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 270.3977966308594D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243.65843200683594D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 222.20980834960938D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 218.7224884033203D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 216.97836303710938D), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 178.24917602539062D) + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "mezzanine", + "rows", + 6L, + "idx", + 4423.6533203125D + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "premium", + "rows", + 6L, + "idx", + 4418.61865234375D + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "entertainment", + "rows", + 2L, + "idx", + 319.94403076171875D + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "automotive", + "rows", + 2L, + "idx", + 270.3977966308594D + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "travel", + "rows", + 2L, + "idx", + 243.65843200683594D + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "news", + "rows", + 2L, + "idx", + 222.20980834960938D + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "business", + "rows", + 2L, + "idx", + 218.7224884033203D + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "health", + "rows", + 2L, + "idx", + 216.97836303710938D + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "technology", + "rows", + 2L, + "idx", + 178.24917602539062D + ) ); + Map context = Maps.newHashMap(); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build()), "limited" + Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), context), "limited" ); } @@ -766,9 +997,27 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "marketalias", "upfront", "rows", 186L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "marketalias", "total_market", "rows", 186L), - GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "marketalias", "spot", "rows", 837L) + GroupByQueryRunnerTestHelper.createExpectedRow( + "1970-01-01T00:00:00.000Z", + "marketalias", + "upfront", + "rows", + 186L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "1970-01-01T00:00:00.000Z", + "marketalias", + "total_market", + "rows", + 186L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "1970-01-01T00:00:00.000Z", + "marketalias", + "spot", + "rows", + 837L + ) ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -811,7 +1060,9 @@ public class GroupByQueryRunnerTest new QueryRunner() { @Override - public Sequence run(Query query) + public Sequence run( + Query query, Map context + ) { // simulate two daily segments final Query query1 = query.withQuerySegmentSpec( @@ -820,12 +1071,13 @@ public class GroupByQueryRunnerTest 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)); + return Sequences.concat(runner.run(query1, context), runner.run(query2, context)); } } ); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery), "merged"); + Map context = Maps.newHashMap(); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged"); } @Test @@ -867,9 +1119,11 @@ public class GroupByQueryRunnerTest QueryRunner mergeRunner = new GroupByQueryQueryToolChest( configSupplier, new DefaultObjectMapper(), - engine + engine, + TestQueryRunners.pool ).mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); + Map context = Maps.newHashMap(); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); } @Test @@ -902,7 +1156,8 @@ public class GroupByQueryRunnerTest GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L) ); - TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); + Map context = Maps.newHashMap(); + TestHelper.assertExpectedObjects(expectedResults, runner.run(query, context), "normal"); final GroupByQueryEngine engine = new GroupByQueryEngine( configSupplier, new StupidPool( @@ -920,9 +1175,10 @@ public class GroupByQueryRunnerTest QueryRunner mergeRunner = new GroupByQueryQueryToolChest( configSupplier, new DefaultObjectMapper(), - engine + engine, + TestQueryRunners.pool ).mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); } @Test @@ -955,7 +1211,8 @@ public class GroupByQueryRunnerTest GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L) ); - TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); + Map context = Maps.newHashMap(); + TestHelper.assertExpectedObjects(expectedResults, runner.run(query, context), "normal"); final GroupByQueryEngine engine = new GroupByQueryEngine( configSupplier, new StupidPool( @@ -973,9 +1230,10 @@ public class GroupByQueryRunnerTest QueryRunner mergeRunner = new GroupByQueryQueryToolChest( configSupplier, new DefaultObjectMapper(), - engine + engine, + TestQueryRunners.pool ).mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); } // A subquery identical to the query should yield identical results @@ -1203,7 +1461,15 @@ public class GroupByQueryRunnerTest List expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "entertainment", + "rows", + 1L, + "idx", + 11158.0 + ), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 13870.0), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), @@ -1213,7 +1479,15 @@ public class GroupByQueryRunnerTest GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "entertainment", + "rows", + 1L, + "idx", + 11166.0 + ), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), @@ -1303,7 +1577,15 @@ public class GroupByQueryRunnerTest List expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "entertainment", + "rows", + 1L, + "idx", + 11158.0 + ), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), @@ -1311,7 +1593,15 @@ public class GroupByQueryRunnerTest GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + "entertainment", + "rows", + 1L, + "idx", + 11166.0 + ), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), @@ -1514,15 +1804,105 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 282L, "uniq", 1.0002442201269182), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 230L, "uniq", 1.0002442201269182), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 324L, "uniq", 1.0002442201269182), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 2L, "idx", 233L, "uniq", 1.0002442201269182), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 5317L, "uniq", 1.0002442201269182), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 2L, "idx", 235L, "uniq", 1.0002442201269182), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 5405L, "uniq", 1.0002442201269182), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 175L, "uniq", 1.0002442201269182), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 245L, "uniq", 1.0002442201269182) + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "automotive", + "rows", + 2L, + "idx", + 282L, + "uniq", + 1.0002442201269182 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "business", + "rows", + 2L, + "idx", + 230L, + "uniq", + 1.0002442201269182 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "entertainment", + "rows", + 2L, + "idx", + 324L, + "uniq", + 1.0002442201269182 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "health", + "rows", + 2L, + "idx", + 233L, + "uniq", + 1.0002442201269182 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "mezzanine", + "rows", + 6L, + "idx", + 5317L, + "uniq", + 1.0002442201269182 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "news", + "rows", + 2L, + "idx", + 235L, + "uniq", + 1.0002442201269182 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "premium", + "rows", + 6L, + "idx", + 5405L, + "uniq", + 1.0002442201269182 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "technology", + "rows", + 2L, + "idx", + 175L, + "uniq", + 1.0002442201269182 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + "travel", + "rows", + 2L, + "idx", + 245L, + "uniq", + 1.0002442201269182 + ) ); // Subqueries are handled by the ToolChest diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java index 74dc5091a9d..a60335e214c 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java @@ -47,7 +47,7 @@ public class GroupByQueryRunnerTestHelper toolChest ); - Sequence queryResult = theRunner.run(query); + Sequence queryResult = theRunner.run(query, Maps.newHashMap()); return Sequences.toList(queryResult, Lists.newArrayList()); } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java index 1ad4d2baf0e..78c936e0a57 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -33,6 +33,7 @@ import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; +import io.druid.query.TestQueryRunners; import io.druid.query.timeseries.TimeseriesQuery; import io.druid.query.timeseries.TimeseriesQueryRunnerTest; import io.druid.query.timeseries.TimeseriesResultValue; @@ -43,6 +44,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collection; +import java.util.Map; /** */ @@ -75,7 +77,8 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest engine, QueryRunnerTestHelper.NOOP_QUERYWATCHER, configSupplier, - new GroupByQueryQueryToolChest(configSupplier, new DefaultObjectMapper(), engine) + new GroupByQueryQueryToolChest(configSupplier, new DefaultObjectMapper(), engine, TestQueryRunners.pool), + TestQueryRunners.pool ); final Collection objects = QueryRunnerTestHelper.makeQueryRunners(factory); @@ -92,20 +95,21 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest QueryRunner timeseriesRunner = new QueryRunner() { @Override - public Sequence run(Query query) + public Sequence run(Query query, Map metadata) { TimeseriesQuery tsQuery = (TimeseriesQuery) query; return Sequences.map( groupByRunner.run( GroupByQuery.builder() - .setDataSource(tsQuery.getDataSource()) - .setQuerySegmentSpec(tsQuery.getQuerySegmentSpec()) - .setGranularity(tsQuery.getGranularity()) - .setDimFilter(tsQuery.getDimensionsFilter()) - .setAggregatorSpecs(tsQuery.getAggregatorSpecs()) - .setPostAggregatorSpecs(tsQuery.getPostAggregatorSpecs()) - .build() + .setDataSource(tsQuery.getDataSource()) + .setQuerySegmentSpec(tsQuery.getQuerySegmentSpec()) + .setGranularity(tsQuery.getGranularity()) + .setDimFilter(tsQuery.getDimensionsFilter()) + .setAggregatorSpecs(tsQuery.getAggregatorSpecs()) + .setPostAggregatorSpecs(tsQuery.getPostAggregatorSpecs()) + .build(), + metadata ), new Function>() { diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java index 6130b318757..cb40f8e78d9 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java @@ -38,6 +38,7 @@ import io.druid.segment.column.ValueType; import junit.framework.Assert; import org.junit.Test; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -49,7 +50,7 @@ public class SegmentAnalyzerTest public void testIncrementalDoesNotWork() throws Exception { final List results = getSegmentAnalysises( - new IncrementalIndexSegment(TestIndex.getIncrementalTestIndex(), null) + new IncrementalIndexSegment(TestIndex.getIncrementalTestIndex(false), null) ); Assert.assertEquals(0, results.size()); @@ -106,6 +107,7 @@ public class SegmentAnalyzerTest final SegmentMetadataQuery query = new SegmentMetadataQuery( new LegacyDataSource("test"), QuerySegmentSpecs.create("2011/2012"), null, null, null ); - return Sequences.toList(query.run(runner), Lists.newArrayList()); + HashMap context = new HashMap(); + return Sequences.toList(query.run(runner, context), Lists.newArrayList()); } } diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java index 70336573432..64fc8e9453a 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java @@ -41,6 +41,8 @@ import org.junit.Assert; import org.junit.Test; import java.util.Arrays; +import java.util.HashMap; +import java.util.List; public class SegmentMetadataQueryTest { @@ -73,9 +75,9 @@ public class SegmentMetadataQueryTest .toInclude(new ListColumnIncluderator(Arrays.asList("placement"))) .merge(true) .build(); - + HashMap context = new HashMap(); Iterable results = Sequences.toList( - runner.run(query), + runner.run(query, context), Lists.newArrayList() ); SegmentAnalysis val = results.iterator().next(); diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java index ee833db7385..56842f16ec2 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java @@ -44,6 +44,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; @@ -386,8 +387,9 @@ public class SearchQueryRunnerTest private void checkSearchQuery(SearchQuery searchQuery, Map> expectedResults) { + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(searchQuery), + runner.run(searchQuery, context), Lists.>newArrayList() ); diff --git a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java index e4779fb3df5..ed000dfa77d 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java @@ -44,6 +44,7 @@ import org.junit.runners.Parameterized; import java.io.IOException; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -89,9 +90,9 @@ public class SelectQueryRunnerTest new PagingSpec(null, 3), null ); - + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, context), Lists.>newArrayList() ); @@ -158,9 +159,9 @@ public class SelectQueryRunnerTest new PagingSpec(null, 3), null ); - + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, context), Lists.>newArrayList() ); @@ -218,9 +219,9 @@ public class SelectQueryRunnerTest new PagingSpec(Maps.newLinkedHashMap(ImmutableMap.of(QueryRunnerTestHelper.segmentId, 3)), 3), null ); - + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, context), Lists.>newArrayList() ); @@ -278,9 +279,9 @@ public class SelectQueryRunnerTest new PagingSpec(Maps.newLinkedHashMap(ImmutableMap.of(QueryRunnerTestHelper.segmentId, 3)), 3), null ); - + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, context), Lists.>newArrayList() ); @@ -380,7 +381,7 @@ public class SelectQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, Maps.newHashMap()), Lists.>newArrayList() ); @@ -413,7 +414,7 @@ public class SelectQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, Maps.newHashMap()), Lists.>newArrayList() ); diff --git a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java index 3c39b3274d6..ba93287b828 100644 --- a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java @@ -21,11 +21,13 @@ package io.druid.query.timeboundary; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.google.common.collect.MapMaker; import com.metamx.common.guava.Sequences; import io.druid.query.Druids; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; +import io.druid.query.RetryQueryRunner; import io.druid.query.TableDataSource; import org.joda.time.DateTime; import org.junit.Assert; @@ -36,7 +38,9 @@ import org.junit.runners.Parameterized; import java.io.IOException; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.List; +import java.util.Map; /** */ @@ -67,9 +71,9 @@ public class TimeBoundaryQueryRunnerTest TimeBoundaryQuery timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder() .dataSource("testing") .build(); - + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(timeBoundaryQuery), + runner.run(timeBoundaryQuery, context), Lists.>newArrayList() ); TimeBoundaryResultValue val = results.iterator().next().getValue(); @@ -88,9 +92,10 @@ public class TimeBoundaryQueryRunnerTest .dataSource("testing") .bound(TimeBoundaryQuery.MAX_TIME) .build(); - + Map context = new MapMaker().makeMap(); + context.put(RetryQueryRunner.MISSING_SEGMENTS_KEY, Lists.newArrayList()); Iterable> results = Sequences.toList( - runner.run(timeBoundaryQuery), + runner.run(timeBoundaryQuery, context), Lists.>newArrayList() ); TimeBoundaryResultValue val = results.iterator().next().getValue(); @@ -109,9 +114,10 @@ public class TimeBoundaryQueryRunnerTest .dataSource("testing") .bound(TimeBoundaryQuery.MIN_TIME) .build(); - + Map context = new MapMaker().makeMap(); + context.put(RetryQueryRunner.MISSING_SEGMENTS_KEY, Lists.newArrayList()); Iterable> results = Sequences.toList( - runner.run(timeBoundaryQuery), + runner.run(timeBoundaryQuery, context), Lists.>newArrayList() ); TimeBoundaryResultValue val = results.iterator().next().getValue(); diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java index 84ace0a798f..57b06a31ff5 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java @@ -21,6 +21,7 @@ package io.druid.query.timeseries; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.query.Druids; @@ -44,7 +45,9 @@ import org.junit.runners.Parameterized; import java.io.IOException; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.List; +import java.util.Map; @RunWith(Parameterized.class) public class TimeSeriesUnionQueryRunnerTest @@ -103,9 +106,9 @@ public class TimeSeriesUnionQueryRunnerTest ) ) ); - + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, context), Lists.>newArrayList() ); @@ -142,7 +145,8 @@ public class TimeSeriesUnionQueryRunnerTest new QueryRunner>() { @Override - public Sequence> run(Query> query) + public Sequence> run(Query> query, + Map context) { if (query.getDataSource().equals(new TableDataSource("ds1"))) { return Sequences.simple( @@ -244,7 +248,7 @@ public class TimeSeriesUnionQueryRunnerTest ); Iterable> results = Sequences.toList( - mergingrunner.run(query), + mergingrunner.run(query, Maps.newHashMap()), Lists.>newArrayList() ); diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java index 67c91b4be40..95d654b9153 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java @@ -33,6 +33,7 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; +import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.IncrementalIndexSegment; @@ -43,6 +44,7 @@ import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Test; +import java.util.HashMap; import java.util.List; public class TimeseriesQueryRunnerBonusTest @@ -51,7 +53,8 @@ public class TimeseriesQueryRunnerBonusTest public void testOneRowAtATime() throws Exception { final IncrementalIndex oneRowIndex = new IncrementalIndex( - new DateTime("2012-01-01T00:00:00Z").getMillis(), QueryGranularity.NONE, new AggregatorFactory[]{} + new DateTime("2012-01-01T00:00:00Z").getMillis(), QueryGranularity.NONE, new AggregatorFactory[]{}, + TestQueryRunners.pool ); List> results; @@ -110,9 +113,9 @@ public class TimeseriesQueryRunnerBonusTest ) ) .build(); - + HashMap context = new HashMap(); return Sequences.toList( - runner.run(query), + runner.run(query, context), Lists.>newArrayList() ); } diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java index d4110c7b266..ea0e0e7060d 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -53,6 +53,7 @@ import org.junit.runners.Parameterized; import java.io.IOException; import java.util.Arrays; import java.util.Collection; +import java.util.Map; import java.util.List; /** @@ -60,6 +61,9 @@ import java.util.List; @RunWith(Parameterized.class) public class TimeseriesQueryRunnerTest { + + public static final Map CONTEXT = ImmutableMap.of(); + @Parameterized.Parameters public static Collection constructorFeeder() throws IOException { @@ -103,7 +107,7 @@ public class TimeseriesQueryRunnerTest DateTime expectedLast = new DateTime("2011-04-15"); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); @@ -168,7 +172,7 @@ public class TimeseriesQueryRunnerTest DateTime expectedLast = new DateTime("2011-04-15"); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); Result result = results.iterator().next(); @@ -215,7 +219,7 @@ public class TimeseriesQueryRunnerTest DateTime expectedLast = new DateTime("2011-04-15"); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); @@ -281,7 +285,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); @@ -328,7 +332,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); @@ -370,7 +374,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results1 = Sequences.toList( - runner.run(query1), + runner.run(query1, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults1, results1); @@ -407,7 +411,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results2 = Sequences.toList( - runner.run(query2), + runner.run(query2, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults2, results2); @@ -460,7 +464,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results1 = Sequences.toList( - runner.run(query1), + runner.run(query1, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults1, results1); @@ -528,7 +532,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results1 = Sequences.toList( - runner.run(query1), + runner.run(query1, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults1, results1); @@ -575,7 +579,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results1 = Sequences.toList( - runner.run(query1), + runner.run(query1, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults1, results1); @@ -615,9 +619,8 @@ public class TimeseriesQueryRunnerTest ) ) ); - Iterable> results1 = Sequences.toList( - runner.run(query1), + runner.run(query1, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults1, results1); @@ -655,7 +658,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results2 = Sequences.toList( - runner.run(query2), + runner.run(query2, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults2, results2); @@ -690,7 +693,7 @@ public class TimeseriesQueryRunnerTest List> expectedResults = Arrays.asList(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -740,7 +743,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -790,7 +793,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -840,7 +843,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -890,7 +893,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -940,7 +943,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -998,7 +1001,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1056,7 +1059,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1114,7 +1117,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1170,7 +1173,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1232,7 +1235,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1276,7 +1279,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1320,7 +1323,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1378,7 +1381,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1416,8 +1419,9 @@ public class TimeseriesQueryRunnerTest ) ) ); + Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1456,8 +1460,9 @@ public class TimeseriesQueryRunnerTest ) ) ); + Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1483,12 +1488,13 @@ public class TimeseriesQueryRunnerTest .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .build() + .build(), + CONTEXT ), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1515,12 +1521,13 @@ public class TimeseriesQueryRunnerTest .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .build() + .build(), + CONTEXT ), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1576,12 +1583,13 @@ public class TimeseriesQueryRunnerTest .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .build() + .build(), + CONTEXT ), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1639,12 +1647,13 @@ public class TimeseriesQueryRunnerTest .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .build() + .build(), + CONTEXT ), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index d155c35f91a..2e978dd408f 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -52,6 +52,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -166,8 +167,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -230,8 +231,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -295,8 +296,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -345,8 +346,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -394,8 +395,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -443,8 +444,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -485,8 +486,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -520,8 +521,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -569,8 +570,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -622,8 +623,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -664,8 +665,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -682,7 +683,7 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - + HashMap context = new HashMap(); TestHelper.assertExpectedResults( Lists.>newArrayList( new Result( @@ -690,7 +691,7 @@ public class TopNQueryRunnerTest new TopNResultValue(Lists.>newArrayList()) ) ), - runner.run(query) + runner.run(query, context) ); } @@ -721,7 +722,7 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - + HashMap context = new HashMap(); TestHelper.assertExpectedResults( Lists.>newArrayList( new Result( @@ -729,7 +730,7 @@ public class TopNQueryRunnerTest new TopNResultValue(Lists.>newArrayList()) ) ), - runner.run(query) + runner.run(query, context) ); } @@ -747,7 +748,7 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - + HashMap context = new HashMap(); TestHelper.assertExpectedResults( Sequences.toList( runner.run( @@ -761,9 +762,10 @@ public class TopNQueryRunnerTest .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .build() + .build(), + context ), Lists.>newArrayList() - ), runner.run(query) + ), runner.run(query, context) ); } @@ -781,7 +783,7 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - + HashMap context = new HashMap(); TestHelper.assertExpectedResults( Sequences.toList( runner.run( @@ -795,10 +797,11 @@ public class TopNQueryRunnerTest .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .build() + .build(), + context ), Lists.>newArrayList() ) - , runner.run(query) + , runner.run(query, context) ); } @@ -840,8 +843,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -889,8 +892,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -945,8 +948,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -993,8 +996,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1034,8 +1037,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1075,8 +1078,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1116,8 +1119,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1157,8 +1160,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1209,8 +1212,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1261,8 +1264,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1313,8 +1316,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1358,8 +1361,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -1404,8 +1407,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1449,8 +1452,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1498,8 +1501,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1583,8 +1586,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1666,6 +1669,7 @@ public class TopNQueryRunnerTest ) ) ); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } } diff --git a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java index d9975f39153..6e657bcbeb0 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java @@ -42,6 +42,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -173,8 +174,8 @@ public class TopNUnionQueryTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } diff --git a/processing/src/test/java/io/druid/segment/AppendTest.java b/processing/src/test/java/io/druid/segment/AppendTest.java index 1ddb8122f94..e95c64a415f 100644 --- a/processing/src/test/java/io/druid/segment/AppendTest.java +++ b/processing/src/test/java/io/druid/segment/AppendTest.java @@ -54,6 +54,7 @@ import io.druid.query.topn.TopNResultValue; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import java.util.Arrays; @@ -63,6 +64,7 @@ import java.util.Map; /** */ +@Ignore public class AppendTest { private static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ @@ -177,7 +179,8 @@ public class AppendTest .dataSource(dataSource) .build(); QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -201,7 +204,8 @@ public class AppendTest .dataSource(dataSource) .build(); QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment2); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -225,7 +229,8 @@ public class AppendTest TimeseriesQuery query = makeTimeseriesQuery(); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -249,7 +254,8 @@ public class AppendTest TimeseriesQuery query = makeTimeseriesQuery(); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment2); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -273,7 +279,8 @@ public class AppendTest TimeseriesQuery query = makeFilteredTimeseriesQuery(); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -297,7 +304,8 @@ public class AppendTest TimeseriesQuery query = makeFilteredTimeseriesQuery(); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment2); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -343,7 +351,8 @@ public class AppendTest TopNQuery query = makeTopNQuery(); QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -389,7 +398,8 @@ public class AppendTest TopNQuery query = makeTopNQuery(); QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment2); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -416,7 +426,8 @@ public class AppendTest TopNQuery query = makeFilteredTopNQuery(); QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -433,7 +444,8 @@ public class AppendTest TopNQuery query = makeFilteredTopNQuery(); QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment2); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -455,7 +467,8 @@ public class AppendTest SearchQuery query = makeSearchQuery(); QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -476,7 +489,8 @@ public class AppendTest SearchQuery query = makeSearchQuery(); QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment2); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -496,7 +510,8 @@ public class AppendTest SearchQuery query = makeFilteredSearchQuery(); QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -517,7 +532,8 @@ public class AppendTest SearchQuery query = makeFilteredSearchQuery(); QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment2); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -558,7 +574,8 @@ public class AppendTest .postAggregators(Arrays.asList(addRowsIndexConstant)) .build(); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment3); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } private TimeseriesQuery makeTimeseriesQuery() diff --git a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java index d4e835566a9..d653a367155 100644 --- a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java +++ b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java @@ -22,7 +22,9 @@ package io.druid.segment; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import io.druid.granularity.QueryGranularity; +import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.column.Column; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexAdapter; import org.joda.time.Interval; @@ -45,15 +47,19 @@ public class EmptyIndexTest } tmpDir.deleteOnExit(); - IncrementalIndex emptyIndex = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]); + IncrementalIndex emptyIndex = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0], TestQueryRunners.pool); IncrementalIndexAdapter emptyIndexAdapter = new IncrementalIndexAdapter(new Interval("2012-08-01/P3D"), emptyIndex); - IndexMerger.merge(Lists.newArrayList(emptyIndexAdapter), new AggregatorFactory[0], tmpDir); + IndexMaker.merge( + Lists.newArrayList(emptyIndexAdapter), + new AggregatorFactory[0], + tmpDir + ); QueryableIndex emptyQueryableIndex = IndexIO.loadIndex(tmpDir); - Assert.assertEquals("getAvailableDimensions", 0, Iterables.size(emptyQueryableIndex.getAvailableDimensions())); - Assert.assertEquals("getAvailableMetrics", 0, Iterables.size(emptyQueryableIndex.getColumnNames())); + Assert.assertEquals("getDimensionNames", 0, Iterables.size(emptyQueryableIndex.getAvailableDimensions())); + Assert.assertEquals("getMetricNames", 0, Iterables.size(emptyQueryableIndex.getColumnNames())); Assert.assertEquals("getDataInterval", new Interval("2012-08-01/P3D"), emptyQueryableIndex.getDataInterval()); - Assert.assertEquals("getReadOnlyTimestamps", 0, emptyQueryableIndex.getTimeColumn().getLength()); + Assert.assertEquals("getReadOnlyTimestamps", 0, emptyQueryableIndex.getColumn(Column.TIME_COLUMN_NAME).getLength()); } } diff --git a/processing/src/test/java/io/druid/segment/IndexMergerTest.java b/processing/src/test/java/io/druid/segment/IndexMakerTest.java similarity index 74% rename from processing/src/test/java/io/druid/segment/IndexMergerTest.java rename to processing/src/test/java/io/druid/segment/IndexMakerTest.java index a21eb92c718..4b39d515a6b 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMakerTest.java @@ -25,7 +25,9 @@ import com.google.common.collect.Lists; import com.google.common.io.Files; import io.druid.data.input.MapBasedInputRow; import io.druid.granularity.QueryGranularity; +import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.column.Column; import io.druid.segment.data.IncrementalIndexTest; import io.druid.segment.incremental.IncrementalIndex; import junit.framework.Assert; @@ -37,7 +39,7 @@ import java.util.Arrays; /** */ -public class IndexMergerTest +public class IndexMakerTest { @Test public void testPersistCaseInsensitive() throws Exception @@ -48,9 +50,9 @@ public class IndexMergerTest final File tempDir = Files.createTempDir(); try { - QueryableIndex index = IndexIO.loadIndex(IndexMerger.persist(toPersist, tempDir)); + QueryableIndex index = IndexIO.loadIndex(IndexMaker.persist(toPersist, tempDir)); - Assert.assertEquals(2, index.getTimeColumn().getLength()); + Assert.assertEquals(2, index.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); Assert.assertEquals(2, index.getColumnNames().size()); } @@ -65,7 +67,7 @@ public class IndexMergerTest final long timestamp = System.currentTimeMillis(); IncrementalIndex toPersist1 = IncrementalIndexTest.createCaseInsensitiveIndex(timestamp); - IncrementalIndex toPersist2 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}); + IncrementalIndex toPersist2 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}, TestQueryRunners.pool); toPersist2.add( new MapBasedInputRow( @@ -87,23 +89,27 @@ public class IndexMergerTest final File tempDir2 = Files.createTempDir(); final File mergedDir = Files.createTempDir(); try { - QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tempDir1)); + QueryableIndex index1 = IndexIO.loadIndex(IndexMaker.persist(toPersist1, tempDir1)); - Assert.assertEquals(2, index1.getTimeColumn().getLength()); + Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); Assert.assertEquals(2, index1.getColumnNames().size()); - QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist2, tempDir2)); + QueryableIndex index2 = IndexIO.loadIndex(IndexMaker.persist(toPersist2, tempDir2)); - Assert.assertEquals(2, index2.getTimeColumn().getLength()); + Assert.assertEquals(2, index2.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index2.getAvailableDimensions())); Assert.assertEquals(2, index2.getColumnNames().size()); QueryableIndex merged = IndexIO.loadIndex( - IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, mergedDir) + IndexMaker.mergeQueryableIndex( + Arrays.asList(index1, index2), + new AggregatorFactory[]{}, + mergedDir + ) ); - Assert.assertEquals(3, merged.getTimeColumn().getLength()); + Assert.assertEquals(3, merged.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); Assert.assertEquals(2, merged.getColumnNames().size()); } @@ -117,8 +123,8 @@ public class IndexMergerTest @Test public void testPersistEmptyColumn() throws Exception { - final IncrementalIndex toPersist1 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}); - final IncrementalIndex toPersist2 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}); + final IncrementalIndex toPersist1 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}, TestQueryRunners.pool); + final IncrementalIndex toPersist2 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}, TestQueryRunners.pool); final File tmpDir1 = Files.createTempDir(); final File tmpDir2 = Files.createTempDir(); final File tmpDir3 = Files.createTempDir(); @@ -140,21 +146,22 @@ public class IndexMergerTest ) ); - final QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir1)); - final QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir2)); + final QueryableIndex index1 = IndexIO.loadIndex(IndexMaker.persist(toPersist1, tmpDir1)); + final QueryableIndex index2 = IndexIO.loadIndex(IndexMaker.persist(toPersist1, tmpDir2)); final QueryableIndex merged = IndexIO.loadIndex( - IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3) + IndexMaker.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3) ); - Assert.assertEquals(1, index1.getTimeColumn().getLength()); + Assert.assertEquals(1, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(ImmutableList.of("dim2"), ImmutableList.copyOf(index1.getAvailableDimensions())); - Assert.assertEquals(1, index2.getTimeColumn().getLength()); + Assert.assertEquals(1, index2.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(ImmutableList.of("dim2"), ImmutableList.copyOf(index2.getAvailableDimensions())); - Assert.assertEquals(1, merged.getTimeColumn().getLength()); + Assert.assertEquals(1, merged.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(ImmutableList.of("dim2"), ImmutableList.copyOf(merged.getAvailableDimensions())); - } finally { + } + finally { FileUtils.deleteQuietly(tmpDir1); FileUtils.deleteQuietly(tmpDir2); FileUtils.deleteQuietly(tmpDir3); diff --git a/processing/src/test/java/io/druid/segment/SchemalessIndex.java b/processing/src/test/java/io/druid/segment/SchemalessIndex.java index 836127120f5..823fd83774a 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessIndex.java +++ b/processing/src/test/java/io/druid/segment/SchemalessIndex.java @@ -33,6 +33,7 @@ import com.metamx.common.logger.Logger; import io.druid.data.input.MapBasedInputRow; import io.druid.granularity.QueryGranularity; import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -133,7 +134,7 @@ public class SchemalessIndex final long timestamp = new DateTime(event.get(TIMESTAMP)).getMillis(); if (theIndex == null) { - theIndex = new IncrementalIndex(timestamp, QueryGranularity.MINUTE, METRIC_AGGS); + theIndex = new IncrementalIndex(timestamp, QueryGranularity.MINUTE, METRIC_AGGS, TestQueryRunners.pool); } final List dims = Lists.newArrayList(); @@ -178,11 +179,11 @@ public class SchemalessIndex mergedFile.mkdirs(); mergedFile.deleteOnExit(); - IndexMerger.persist(top, topFile); - IndexMerger.persist(bottom, bottomFile); + IndexMaker.persist(top, topFile); + IndexMaker.persist(bottom, bottomFile); mergedIndex = io.druid.segment.IndexIO.loadIndex( - IndexMerger.mergeQueryableIndex( + IndexMaker.mergeQueryableIndex( Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)), METRIC_AGGS, mergedFile ) ); @@ -224,7 +225,7 @@ public class SchemalessIndex mergedFile.deleteOnExit(); QueryableIndex index = IndexIO.loadIndex( - IndexMerger.mergeQueryableIndex( + IndexMaker.mergeQueryableIndex( Arrays.asList(rowPersistedIndexes.get(index1), rowPersistedIndexes.get(index2)), METRIC_AGGS, mergedFile ) ); @@ -261,7 +262,7 @@ public class SchemalessIndex } QueryableIndex index = IndexIO.loadIndex( - IndexMerger.mergeQueryableIndex(indexesToMerge, METRIC_AGGS, mergedFile) + IndexMaker.mergeQueryableIndex(indexesToMerge, METRIC_AGGS, mergedFile) ); return index; @@ -330,7 +331,7 @@ public class SchemalessIndex } final IncrementalIndex rowIndex = new IncrementalIndex( - timestamp, QueryGranularity.MINUTE, METRIC_AGGS + timestamp, QueryGranularity.MINUTE, METRIC_AGGS, TestQueryRunners.pool ); rowIndex.add( @@ -342,7 +343,7 @@ public class SchemalessIndex tmpFile.mkdirs(); tmpFile.deleteOnExit(); - IndexMerger.persist(rowIndex, tmpFile); + IndexMaker.persist(rowIndex, tmpFile); rowPersistedIndexes.add(IndexIO.loadIndex(tmpFile)); } } @@ -360,7 +361,7 @@ public class SchemalessIndex log.info("Realtime loading index file[%s]", filename); final IncrementalIndex retVal = new IncrementalIndex( - new DateTime("2011-01-12T00:00:00.000Z").getMillis(), QueryGranularity.MINUTE, aggs + new DateTime("2011-01-12T00:00:00.000Z").getMillis(), QueryGranularity.MINUTE, aggs, TestQueryRunners.pool ); try { @@ -402,7 +403,7 @@ public class SchemalessIndex theFile.mkdirs(); theFile.deleteOnExit(); filesToMap.add(theFile); - IndexMerger.persist(index, theFile); + IndexMaker.persist(index, theFile); } return filesToMap; @@ -462,7 +463,7 @@ public class SchemalessIndex ); } - return IndexIO.loadIndex(IndexMerger.append(adapters, mergedFile)); + return IndexIO.loadIndex(IndexMaker.append(adapters, mergedFile)); } catch (IOException e) { throw Throwables.propagate(e); @@ -481,7 +482,7 @@ public class SchemalessIndex List filesToMap = makeFilesToMap(tmpFile, files); return IndexIO.loadIndex( - IndexMerger.mergeQueryableIndex( + IndexMaker.mergeQueryableIndex( Lists.newArrayList( Iterables.transform( filesToMap, diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java index 90b138aca5a..29548d8b55e 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java @@ -1174,6 +1174,15 @@ public class SchemalessTestFull new DateTime("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( + ImmutableMap.builder() + .put("market", "") + .put("rows", 6L) + .put("index", 400.0D) + .put("addRowsIndexConstant", 407.0D) + .put("uniques", 0.0) + .put("maxIndex", 100.0) + .put("minIndex", 0.0) + .build(), ImmutableMap.builder() .put("market", "spot") .put("rows", 4L) @@ -1183,15 +1192,6 @@ public class SchemalessTestFull .put("maxIndex", 100.0) .put("minIndex", 100.0) .build(), - ImmutableMap.builder() - .put("market", "") - .put("rows", 3L) - .put("index", 200.0D) - .put("addRowsIndexConstant", 204.0D) - .put("uniques", 0.0) - .put("maxIndex", 100.0) - .put("minIndex", 0.0) - .build(), ImmutableMap.builder() .put("market", "total_market") .put("rows", 2L) @@ -1386,8 +1386,9 @@ public class SchemalessTestFull .build(); failMsg += " timeseries "; + HashMap context = new HashMap(); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); @@ -1419,8 +1420,9 @@ public class SchemalessTestFull .build(); failMsg += " filtered timeseries "; + HashMap context = new HashMap(); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); @@ -1451,8 +1453,9 @@ public class SchemalessTestFull .build(); failMsg += " topN "; + HashMap context = new HashMap(); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, context), Lists.>newArrayList() ); @@ -1484,8 +1487,9 @@ public class SchemalessTestFull .build(); failMsg += " filtered topN "; + HashMap context = new HashMap(); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); @@ -1501,8 +1505,9 @@ public class SchemalessTestFull .build(); failMsg += " search "; + HashMap context = new HashMap(); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); @@ -1519,8 +1524,9 @@ public class SchemalessTestFull .build(); failMsg += " filtered search "; + HashMap context = new HashMap(); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); @@ -1537,8 +1543,9 @@ public class SchemalessTestFull .build(); failMsg += " timeBoundary "; + HashMap context = new HashMap(); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java b/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java index 524aa5b8668..9544391c8f8 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java @@ -60,6 +60,7 @@ import org.junit.runners.Parameterized; import java.io.IOException; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.List; /** @@ -164,7 +165,8 @@ public class SchemalessTestSimple ) ); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -237,7 +239,8 @@ public class SchemalessTestSimple ); QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -265,7 +268,8 @@ public class SchemalessTestSimple ); QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -290,6 +294,7 @@ public class SchemalessTestSimple ); QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } } diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index bba10185dee..466eb1c8ec4 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -31,11 +31,14 @@ import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; import io.druid.granularity.QueryGranularity; +import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.segment.incremental.OffheapIncrementalIndex; import io.druid.segment.serde.ComplexMetrics; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -51,12 +54,6 @@ import java.util.concurrent.atomic.AtomicLong; */ public class TestIndex { - private static final Logger log = new Logger(TestIndex.class); - - private static IncrementalIndex realtimeIndex = null; - private static QueryableIndex mmappedIndex = null; - private static QueryableIndex mergedRealtime = null; - public static final String[] COLUMNS = new String[]{ "ts", "market", @@ -68,6 +65,7 @@ public class TestIndex }; public static final String[] DIMENSIONS = new String[]{"market", "quALIty", "plAcEmEnT", "pLacementish"}; public static final String[] METRICS = new String[]{"iNdEx"}; + private static final Logger log = new Logger(TestIndex.class); private static final Interval DATA_INTERVAL = new Interval("2011-01-12T00:00:00.000Z/2011-05-01T00:00:00.000Z"); private static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ new DoubleSumAggregatorFactory(METRICS[0], METRICS[0]), @@ -80,7 +78,11 @@ public class TestIndex } } - public static IncrementalIndex getIncrementalTestIndex() + private static IncrementalIndex realtimeIndex = null; + private static QueryableIndex mmappedIndex = null; + private static QueryableIndex mergedRealtime = null; + + public static IncrementalIndex getIncrementalTestIndex(boolean useOffheap) { synchronized (log) { if (realtimeIndex != null) { @@ -88,7 +90,7 @@ public class TestIndex } } - return realtimeIndex = makeRealtimeIndex("druid.sample.tsv"); + return realtimeIndex = makeRealtimeIndex("druid.sample.tsv", useOffheap); } public static QueryableIndex getMMappedTestIndex() @@ -99,7 +101,7 @@ public class TestIndex } } - IncrementalIndex incrementalIndex = getIncrementalTestIndex(); + IncrementalIndex incrementalIndex = getIncrementalTestIndex(false); mmappedIndex = persistRealtimeAndLoadMMapped(incrementalIndex); return mmappedIndex; @@ -113,8 +115,8 @@ public class TestIndex } try { - IncrementalIndex top = makeRealtimeIndex("druid.sample.tsv.top"); - IncrementalIndex bottom = makeRealtimeIndex("druid.sample.tsv.bottom"); + IncrementalIndex top = makeRealtimeIndex("druid.sample.tsv.top", false); + IncrementalIndex bottom = makeRealtimeIndex("druid.sample.tsv.bottom", false); File tmpFile = File.createTempFile("yay", "who"); tmpFile.delete(); @@ -130,11 +132,11 @@ public class TestIndex mergedFile.mkdirs(); mergedFile.deleteOnExit(); - IndexMerger.persist(top, DATA_INTERVAL, topFile); - IndexMerger.persist(bottom, DATA_INTERVAL, bottomFile); + IndexMaker.persist(top, DATA_INTERVAL, topFile); + IndexMaker.persist(bottom, DATA_INTERVAL, bottomFile); mergedRealtime = IndexIO.loadIndex( - IndexMerger.mergeQueryableIndex( + IndexMaker.mergeQueryableIndex( Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)), METRIC_AGGS, mergedFile @@ -149,14 +151,27 @@ public class TestIndex } } - private static IncrementalIndex makeRealtimeIndex(final String resourceFilename) + private static IncrementalIndex makeRealtimeIndex(final String resourceFilename, final boolean useOffheap) { final URL resource = TestIndex.class.getClassLoader().getResource(resourceFilename); log.info("Realtime loading index file[%s]", resource); - - final IncrementalIndex retVal = new IncrementalIndex( - new DateTime("2011-01-12T00:00:00.000Z").getMillis(), QueryGranularity.NONE, METRIC_AGGS - ); + final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() + .withMinTimestamp(new DateTime("2011-01-12T00:00:00.000Z").getMillis()) + .withQueryGranularity(QueryGranularity.NONE) + .withMetrics(METRIC_AGGS) + .build(); + final IncrementalIndex retVal; + if (useOffheap) { + retVal = new OffheapIncrementalIndex( + schema, + TestQueryRunners.pool + ); + } else { + retVal = new IncrementalIndex( + schema, + TestQueryRunners.pool + ); + } final AtomicLong startTime = new AtomicLong(); int lineCount; @@ -228,7 +243,7 @@ public class TestIndex someTmpFile.mkdirs(); someTmpFile.deleteOnExit(); - IndexMerger.persist(index, someTmpFile); + IndexMaker.persist(index, someTmpFile); return IndexIO.loadIndex(someTmpFile); } catch (IOException e) { diff --git a/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java index d9cf4c5b6ab..0982f76f8bf 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java @@ -25,6 +25,8 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -36,10 +38,14 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; -/** - */ -public class CompressedFloatsIndexedSupplierTest +@RunWith(Parameterized.class) +public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest { + public CompressedFloatsIndexedSupplierTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy) + { + super(compressionStrategy); + } + private IndexedFloats indexed; private CompressedFloatsIndexedSupplier supplier; private float[] vals; @@ -68,7 +74,8 @@ public class CompressedFloatsIndexedSupplierTest supplier = CompressedFloatsIndexedSupplier.fromFloatBuffer( FloatBuffer.wrap(vals), 5, - ByteOrder.nativeOrder() + ByteOrder.nativeOrder(), + compressionStrategy ); indexed = supplier.get(); @@ -82,7 +89,7 @@ public class CompressedFloatsIndexedSupplierTest ByteArrayOutputStream baos = new ByteArrayOutputStream(); final CompressedFloatsIndexedSupplier theSupplier = CompressedFloatsIndexedSupplier.fromFloatBuffer( - FloatBuffer.wrap(vals), 5, ByteOrder.nativeOrder() + FloatBuffer.wrap(vals), 5, ByteOrder.nativeOrder(), compressionStrategy ); theSupplier.writeToChannel(Channels.newChannel(baos)); diff --git a/processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java index 7b670110269..e61c01be8e5 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java @@ -23,6 +23,8 @@ import com.google.common.io.OutputSupplier; import io.druid.collections.ResourceHolder; import org.junit.Assert; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -31,21 +33,31 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.FloatBuffer; -/** - */ -public class CompressedFloatsSupplierSerializerTest +@RunWith(Parameterized.class) +public class CompressedFloatsSupplierSerializerTest extends CompressionStrategyTest { + public CompressedFloatsSupplierSerializerTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy) + { + super(compressionStrategy); + } + @Test public void testSanity() throws Exception { final ByteOrder order = ByteOrder.nativeOrder(); + final int sizePer = 999; CompressedFloatsSupplierSerializer serializer = new CompressedFloatsSupplierSerializer( - 999, + sizePer, new GenericIndexedWriter>( new IOPeonForTesting(), "test", - CompressedFloatBufferObjectStrategy.getBufferForOrder(order) - ) + CompressedFloatBufferObjectStrategy.getBufferForOrder( + order, + compressionStrategy, + sizePer + ) + ), + compressionStrategy ); serializer.open(); diff --git a/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java index 768100fd559..b251134ddc7 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java @@ -25,6 +25,8 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -36,10 +38,14 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; -/** - */ -public class CompressedLongsIndexedSupplierTest +@RunWith(Parameterized.class) +public class CompressedLongsIndexedSupplierTest extends CompressionStrategyTest { + public CompressedLongsIndexedSupplierTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy) + { + super(compressionStrategy); + } + private IndexedLongs indexed; private CompressedLongsIndexedSupplier supplier; private long[] vals; @@ -66,7 +72,8 @@ public class CompressedLongsIndexedSupplierTest supplier = CompressedLongsIndexedSupplier.fromLongBuffer( LongBuffer.wrap(vals), 5, - ByteOrder.nativeOrder() + ByteOrder.nativeOrder(), + compressionStrategy ); indexed = supplier.get(); @@ -78,7 +85,7 @@ public class CompressedLongsIndexedSupplierTest ByteArrayOutputStream baos = new ByteArrayOutputStream(); final CompressedLongsIndexedSupplier theSupplier = CompressedLongsIndexedSupplier.fromLongBuffer( - LongBuffer.wrap(vals), 5, ByteOrder.nativeOrder() + LongBuffer.wrap(vals), 5, ByteOrder.nativeOrder(), compressionStrategy ); theSupplier.writeToChannel(Channels.newChannel(baos)); diff --git a/processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java b/processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java index 029297e73e8..bb5d6ec444e 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java @@ -23,6 +23,8 @@ import com.google.common.io.OutputSupplier; import io.druid.collections.ResourceHolder; import org.junit.Assert; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -31,21 +33,27 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.LongBuffer; -/** - */ -public class CompressedLongsSupplierSerializerTest +@RunWith(Parameterized.class) +public class CompressedLongsSupplierSerializerTest extends CompressionStrategyTest { + public CompressedLongsSupplierSerializerTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy) + { + super(compressionStrategy); + } + @Test public void testSanity() throws Exception { final ByteOrder order = ByteOrder.nativeOrder(); + final int sizePer = 999; CompressedLongsSupplierSerializer serializer = new CompressedLongsSupplierSerializer( - 999, + sizePer, new GenericIndexedWriter>( new IOPeonForTesting(), "test", - CompressedLongBufferObjectStrategy.getBufferForOrder(order) - ) + CompressedLongBufferObjectStrategy.getBufferForOrder(order, compressionStrategy, sizePer) + ), + compressionStrategy ); serializer.open(); diff --git a/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java b/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java new file mode 100644 index 00000000000..845ed8369c7 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java @@ -0,0 +1,52 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment.data; + +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import org.junit.runners.Parameterized; + +import java.util.Arrays; + +public class CompressionStrategyTest +{ + @Parameterized.Parameters + public static Iterable compressionStrategies() + { + return Iterables.transform( + Arrays.asList(CompressedObjectStrategy.CompressionStrategy.values()), + new Function() + { + @Override + public Object[] apply(CompressedObjectStrategy.CompressionStrategy compressionStrategy) + { + return new Object[]{compressionStrategy}; + } + } + ); + } + + protected final CompressedObjectStrategy.CompressionStrategy compressionStrategy; + + public CompressionStrategyTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy) + { + this.compressionStrategy = compressionStrategy; + } +} diff --git a/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java b/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java index 35fb2b81c0e..7a131901501 100644 --- a/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.Row; import io.druid.granularity.QueryGranularity; +import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.incremental.IncrementalIndex; @@ -45,7 +46,10 @@ public class IncrementalIndexTest public static IncrementalIndex createCaseInsensitiveIndex(long timestamp) { - IncrementalIndex index = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}); + IncrementalIndex index = new IncrementalIndex( + 0L, QueryGranularity.NONE, new AggregatorFactory[]{}, + TestQueryRunners.pool + ); index.add( new MapBasedInputRow( @@ -105,7 +109,8 @@ public class IncrementalIndexTest final IncrementalIndex index = new IncrementalIndex( 0L, QueryGranularity.NONE, - new AggregatorFactory[]{new CountAggregatorFactory("count")} + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + TestQueryRunners.pool ); final int threadCount = 10; final int elementsPerThread = 200; diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java index c8155526a89..4e7ae7d3163 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.Lists; import com.metamx.collections.spatial.search.RadiusBound; import com.metamx.collections.spatial.search.RectangularBound; import io.druid.data.input.MapBasedInputRow; +import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.granularity.QueryGranularity; import io.druid.query.Druids; @@ -33,6 +34,7 @@ import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; +import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; @@ -44,7 +46,7 @@ import io.druid.query.timeseries.TimeseriesQueryRunnerFactory; import io.druid.query.timeseries.TimeseriesResultValue; import io.druid.segment.IncrementalIndexSegment; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMaker; import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; @@ -61,6 +63,7 @@ import java.io.File; import java.io.IOException; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Random; @@ -70,13 +73,17 @@ import java.util.Random; public class SpatialFilterBonusTest { private static Interval DATA_INTERVAL = new Interval("2013-01-01/2013-01-07"); - private static AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ new CountAggregatorFactory("rows"), new LongSumAggregatorFactory("val", "val") }; - private static List DIMS = Lists.newArrayList("dim", "dim.geo"); + private final Segment segment; + + public SpatialFilterBonusTest(Segment segment) + { + this.segment = segment; + } @Parameterized.Parameters public static Collection constructorFeeder() throws IOException @@ -105,14 +112,20 @@ public class SpatialFilterBonusTest new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) .withQueryGranularity(QueryGranularity.DAY) .withMetrics(METRIC_AGGS) - .withSpatialDimensions( - Arrays.asList( - new SpatialDimensionSchema( - "dim.geo", - Lists.newArrayList() + .withDimensionsSpec( + new DimensionsSpec( + null, + null, + Arrays.asList( + new SpatialDimensionSchema( + "dim.geo", + Lists.newArrayList() + ) ) ) - ).build() + ).build(), + TestQueryRunners.pool, + false ); theIndex.add( new MapBasedInputRow( @@ -219,7 +232,7 @@ public class SpatialFilterBonusTest tmpFile.mkdirs(); tmpFile.deleteOnExit(); - IndexMerger.persist(theIndex, tmpFile); + IndexMaker.persist(theIndex, tmpFile); return IndexIO.loadIndex(tmpFile); } @@ -230,40 +243,60 @@ public class SpatialFilterBonusTest new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) .withQueryGranularity(QueryGranularity.DAY) .withMetrics(METRIC_AGGS) - .withSpatialDimensions( - Arrays.asList( - new SpatialDimensionSchema( - "dim.geo", - Lists.newArrayList() + .withDimensionsSpec( + new DimensionsSpec( + null, + null, + Arrays.asList( + new SpatialDimensionSchema( + "dim.geo", + Lists.newArrayList() + ) ) ) - ).build() + + ).build(), + TestQueryRunners.pool, + false ); IncrementalIndex second = new IncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) .withQueryGranularity(QueryGranularity.DAY) .withMetrics(METRIC_AGGS) - .withSpatialDimensions( - Arrays.asList( - new SpatialDimensionSchema( - "dim.geo", - Lists.newArrayList() + .withDimensionsSpec( + new DimensionsSpec( + null, + null, + Arrays.asList( + new SpatialDimensionSchema( + "dim.geo", + Lists.newArrayList() + ) ) ) - ).build() + ).build(), + TestQueryRunners.pool, + false ); IncrementalIndex third = new IncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) .withQueryGranularity(QueryGranularity.DAY) .withMetrics(METRIC_AGGS) - .withSpatialDimensions( - Arrays.asList( - new SpatialDimensionSchema( - "dim.geo", - Lists.newArrayList() + .withDimensionsSpec( + new DimensionsSpec( + null, + null, + Arrays.asList( + new SpatialDimensionSchema( + "dim.geo", + Lists.newArrayList() + ) ) ) - ).build() + + ).build(), + TestQueryRunners.pool, + false ); @@ -379,12 +412,12 @@ public class SpatialFilterBonusTest mergedFile.mkdirs(); mergedFile.deleteOnExit(); - IndexMerger.persist(first, DATA_INTERVAL, firstFile); - IndexMerger.persist(second, DATA_INTERVAL, secondFile); - IndexMerger.persist(third, DATA_INTERVAL, thirdFile); + IndexMaker.persist(first, DATA_INTERVAL, firstFile); + IndexMaker.persist(second, DATA_INTERVAL, secondFile); + IndexMaker.persist(third, DATA_INTERVAL, thirdFile); QueryableIndex mergedRealtime = IndexIO.loadIndex( - IndexMerger.mergeQueryableIndex( + IndexMaker.mergeQueryableIndex( Arrays.asList(IndexIO.loadIndex(firstFile), IndexIO.loadIndex(secondFile), IndexIO.loadIndex(thirdFile)), METRIC_AGGS, mergedFile @@ -398,13 +431,6 @@ public class SpatialFilterBonusTest } } - private final Segment segment; - - public SpatialFilterBonusTest(Segment segment) - { - this.segment = segment; - } - @Test public void testSpatialQuery() { @@ -448,8 +474,8 @@ public class SpatialFilterBonusTest factory.createRunner(segment), factory.getToolchest() ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } catch (Exception e) { throw Throwables.propagate(e); @@ -535,8 +561,8 @@ public class SpatialFilterBonusTest factory.createRunner(segment), factory.getToolchest() ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } catch (Exception e) { throw Throwables.propagate(e); diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java index 75fc85f5587..f3397d3b4a5 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java @@ -22,9 +22,11 @@ package io.druid.segment.filter; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.metamx.collections.spatial.search.RadiusBound; import com.metamx.collections.spatial.search.RectangularBound; import io.druid.data.input.MapBasedInputRow; +import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.granularity.QueryGranularity; import io.druid.query.Druids; @@ -33,6 +35,7 @@ import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; +import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; @@ -105,18 +108,25 @@ public class SpatialFilterTest new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) .withQueryGranularity(QueryGranularity.DAY) .withMetrics(METRIC_AGGS) - .withSpatialDimensions( - Arrays.asList( - new SpatialDimensionSchema( - "dim.geo", - Arrays.asList("lat", "long") - ), - new SpatialDimensionSchema( - "spatialIsRad", - Arrays.asList("lat2", "long2") + .withDimensionsSpec( + new DimensionsSpec( + null, + null, + Arrays.asList( + new SpatialDimensionSchema( + "dim.geo", + Arrays.asList("lat", "long") + ), + new SpatialDimensionSchema( + "spatialIsRad", + Arrays.asList("lat2", "long2") + ) + ) ) - ).build() + ).build(), + TestQueryRunners.pool, + false ); theIndex.add( new MapBasedInputRow( @@ -261,52 +271,73 @@ public class SpatialFilterTest new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) .withQueryGranularity(QueryGranularity.DAY) .withMetrics(METRIC_AGGS) - .withSpatialDimensions( - Arrays.asList( - new SpatialDimensionSchema( - "dim.geo", - Arrays.asList("lat", "long") - ), - new SpatialDimensionSchema( - "spatialIsRad", - Arrays.asList("lat2", "long2") + .withDimensionsSpec( + new DimensionsSpec( + null, + null, + Arrays.asList( + new SpatialDimensionSchema( + "dim.geo", + Arrays.asList("lat", "long") + ), + new SpatialDimensionSchema( + "spatialIsRad", + Arrays.asList("lat2", "long2") + ) + ) ) - ).build() + ).build(), + TestQueryRunners.pool, + false ); IncrementalIndex second = new IncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) .withQueryGranularity(QueryGranularity.DAY) .withMetrics(METRIC_AGGS) - .withSpatialDimensions( - Arrays.asList( - new SpatialDimensionSchema( - "dim.geo", - Arrays.asList("lat", "long") - ), - new SpatialDimensionSchema( - "spatialIsRad", - Arrays.asList("lat2", "long2") + .withDimensionsSpec( + new DimensionsSpec( + null, + null, + Arrays.asList( + new SpatialDimensionSchema( + "dim.geo", + Arrays.asList("lat", "long") + ), + new SpatialDimensionSchema( + "spatialIsRad", + Arrays.asList("lat2", "long2") + ) + ) ) - ).build() + ).build(), + TestQueryRunners.pool, + false ); IncrementalIndex third = new IncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) .withQueryGranularity(QueryGranularity.DAY) .withMetrics(METRIC_AGGS) - .withSpatialDimensions( - Arrays.asList( - new SpatialDimensionSchema( - "dim.geo", - Arrays.asList("lat", "long") - ), - new SpatialDimensionSchema( - "spatialIsRad", - Arrays.asList("lat2", "long2") + .withDimensionsSpec( + new DimensionsSpec( + null, + null, + Arrays.asList( + new SpatialDimensionSchema( + "dim.geo", + Arrays.asList("lat", "long") + ), + new SpatialDimensionSchema( + "spatialIsRad", + Arrays.asList("lat2", "long2") + ) + ) ) - ).build() + ).build(), + TestQueryRunners.pool, + false ); @@ -519,7 +550,7 @@ public class SpatialFilterTest factory.getToolchest() ); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap())); } catch (Exception e) { throw Throwables.propagate(e); @@ -571,7 +602,7 @@ public class SpatialFilterTest factory.getToolchest() ); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap())); } catch (Exception e) { throw Throwables.propagate(e); @@ -658,7 +689,7 @@ public class SpatialFilterTest factory.getToolchest() ); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap())); } catch (Exception e) { throw Throwables.propagate(e); diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index 7f17d7fc2e9..bc15e5b930e 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -32,6 +32,7 @@ import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.granularity.QueryGranularity; import io.druid.query.Result; +import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.JavaScriptAggregatorFactory; @@ -63,7 +64,8 @@ public class IncrementalIndexStorageAdapterTest public void testSanity() throws Exception { IncrementalIndex index = new IncrementalIndex( - 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")} + 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, + TestQueryRunners.pool ); index.add( @@ -110,7 +112,7 @@ public class IncrementalIndexStorageAdapterTest public void testObjectColumnSelectorOnVaryingColumnSchema() throws Exception { IncrementalIndex index = new IncrementalIndex( - 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")} + 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, TestQueryRunners.pool ); index.add( @@ -196,7 +198,8 @@ public class IncrementalIndexStorageAdapterTest @Test public void testResetSanity() { IncrementalIndex index = new IncrementalIndex( - 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")} + 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, + TestQueryRunners.pool ); @@ -248,7 +251,8 @@ public class IncrementalIndexStorageAdapterTest public void testSingleValueTopN() { IncrementalIndex index = new IncrementalIndex( - 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")} + 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, + TestQueryRunners.pool ); DateTime t = DateTime.now(); @@ -303,7 +307,8 @@ public class IncrementalIndexStorageAdapterTest public void testFilterByNull() throws Exception { IncrementalIndex index = new IncrementalIndex( - 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")} + 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, + TestQueryRunners.pool ); index.add( diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 210771cd139..43cec97c472 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.161-SNAPSHOT + 0.7.0-SNAPSHOT @@ -46,4 +46,4 @@ test - \ No newline at end of file + diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 1b34ec00764..bdd0d117b2c 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.161-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index e264ce024c7..aa0b1508119 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -28,7 +29,7 @@ io.druid druid - 0.6.161-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index f189e060670..10af6f0d8e7 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -114,7 +114,7 @@ public class CachingClusteredClient implements QueryRunner } @Override - public Sequence run(final Query query) + public Sequence run(final Query query, final Map context) { final QueryToolChest> toolChest = warehouse.getToolChest(query); final CacheStrategy> strategy = toolChest.getCacheStrategy(query); @@ -328,7 +328,7 @@ public class CachingClusteredClient implements QueryRunner final QueryRunner clientQueryable = serverView.getQueryRunner(server); if (clientQueryable == null) { - log.makeAlert("WTF!? server[%s] doesn't have a client Queryable?", server).emit(); + log.error("WTF!? server[%s] doesn't have a client Queryable?", server); continue; } @@ -337,13 +337,13 @@ public class CachingClusteredClient implements QueryRunner List intervals = segmentSpec.getIntervals(); if (!server.isAssignable() || !populateCache || isBySegment) { - resultSeqToAdd = clientQueryable.run(query.withQuerySegmentSpec(segmentSpec)); + resultSeqToAdd = clientQueryable.run(query.withQuerySegmentSpec(segmentSpec), context); } else { // this could be more efficient, since we only need to reorder results // for batches of segments with the same segment start time. resultSeqToAdd = toolChest.mergeSequencesUnordered( Sequences.map( - clientQueryable.run(rewrittenQuery.withQuerySegmentSpec(segmentSpec)), + clientQueryable.run(rewrittenQuery.withQuerySegmentSpec(segmentSpec), context), new Function>() { private final Function cacheFn = strategy.prepareForCache(); diff --git a/server/src/main/java/io/druid/client/CachingQueryRunner.java b/server/src/main/java/io/druid/client/CachingQueryRunner.java index 41d3813ada7..f76dcbb9dd2 100644 --- a/server/src/main/java/io/druid/client/CachingQueryRunner.java +++ b/server/src/main/java/io/druid/client/CachingQueryRunner.java @@ -40,6 +40,7 @@ import io.druid.query.SegmentDescriptor; import java.io.IOException; import java.util.Iterator; import java.util.List; +import java.util.Map; public class CachingQueryRunner implements QueryRunner { @@ -72,7 +73,7 @@ public class CachingQueryRunner implements QueryRunner } @Override - public Sequence run(Query query) + public Sequence run(Query query, Map context) { final CacheStrategy strategy = toolChest.getCacheStrategy(query); @@ -142,7 +143,7 @@ public class CachingQueryRunner implements QueryRunner return Sequences.withEffect( Sequences.map( - base.run(query), + base.run(query, context), new Function() { @Override @@ -164,7 +165,7 @@ public class CachingQueryRunner implements QueryRunner MoreExecutors.sameThreadExecutor() ); } else { - return base.run(query); + return base.run(query, context); } } diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index c95ec12d348..c354bc18259 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -22,6 +22,7 @@ package io.druid.client; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonToken; import com.fasterxml.jackson.core.ObjectCodec; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.type.TypeFactory; @@ -111,7 +112,7 @@ public class DirectDruidClient implements QueryRunner } @Override - public Sequence run(final Query query) + public Sequence run(final Query query, final Map context) { QueryToolChest> toolChest = warehouse.getToolChest(query); boolean isBySegment = query.getContextBySegment(false); @@ -156,6 +157,19 @@ public class DirectDruidClient implements QueryRunner log.debug("Initial response from url[%s]", url); startTime = System.currentTimeMillis(); byteCount += response.getContent().readableBytes(); + + try { + final Map responseContext = objectMapper.readValue( + response.headers().get("X-Druid-Response-Context"), new TypeReference>() + { + } + ); + context.putAll(responseContext); + } + catch (IOException e) { + e.printStackTrace(); + } + return super.handleResponse(response); } @@ -190,40 +204,40 @@ public class DirectDruidClient implements QueryRunner openConnections.getAndIncrement(); Futures.addCallback( future, new FutureCallback() - { - @Override - public void onSuccess(InputStream result) - { - openConnections.getAndDecrement(); - } + { + @Override + public void onSuccess(InputStream result) + { + openConnections.getAndDecrement(); + } - @Override - public void onFailure(Throwable t) - { - openConnections.getAndDecrement(); - if (future.isCancelled()) { - // forward the cancellation to underlying queriable node - try { - StatusResponseHolder res = httpClient - .delete(new URL(cancelUrl)) - .setContent(objectMapper.writeValueAsBytes(query)) - .setHeader(HttpHeaders.Names.CONTENT_TYPE, isSmile ? "application/smile" : "application/json") - .go(new StatusResponseHandler(Charsets.UTF_8)) - .get(); - if (res.getStatus().getCode() >= 500) { - throw new RE( - "Error cancelling query[%s]: queriable node returned status[%d] [%s].", - res.getStatus().getCode(), - res.getStatus().getReasonPhrase() - ); + @Override + public void onFailure(Throwable t) + { + openConnections.getAndDecrement(); + if (future.isCancelled()) { + // forward the cancellation to underlying queriable node + try { + StatusResponseHolder res = httpClient + .delete(new URL(cancelUrl)) + .setContent(objectMapper.writeValueAsBytes(query)) + .setHeader(HttpHeaders.Names.CONTENT_TYPE, isSmile ? "application/smile" : "application/json") + .go(new StatusResponseHandler(Charsets.UTF_8)) + .get(); + if (res.getStatus().getCode() >= 500) { + throw new RE( + "Error cancelling query[%s]: queriable node returned status[%d] [%s].", + res.getStatus().getCode(), + res.getStatus().getReasonPhrase() + ); + } + } + catch (IOException | ExecutionException | InterruptedException e) { + Throwables.propagate(e); + } } } - catch (IOException | ExecutionException | InterruptedException e) { - Throwables.propagate(e); - } } - } - } ); } catch (IOException e) { @@ -321,8 +335,7 @@ public class DirectDruidClient implements QueryRunner if (nextToken == JsonToken.START_OBJECT) { QueryInterruptedException e = jp.getCodec().readValue(jp, QueryInterruptedException.class); throw e; - } - else if (nextToken != JsonToken.START_ARRAY) { + } else if (nextToken != JsonToken.START_ARRAY) { throw new IAE("Next token wasn't a START_ARRAY, was[%s] from url [%s]", jp.getCurrentToken(), url); } else { jp.nextToken(); @@ -330,7 +343,7 @@ public class DirectDruidClient implements QueryRunner } } catch (IOException | InterruptedException | ExecutionException e) { - throw new RE(e, "Failure getting results from[%s]", url); + throw new RE(e, "Failure getting results from[%s] because of [%s]", url, e.getMessage()); } catch (CancellationException e) { throw new QueryInterruptedException("Query cancelled"); diff --git a/server/src/main/java/io/druid/curator/CuratorConfig.java b/server/src/main/java/io/druid/curator/CuratorConfig.java index 7ac0247b342..50d23f64104 100644 --- a/server/src/main/java/io/druid/curator/CuratorConfig.java +++ b/server/src/main/java/io/druid/curator/CuratorConfig.java @@ -34,6 +34,6 @@ public abstract class CuratorConfig public abstract int getZkSessionTimeoutMs(); @Config("druid.curator.compress") - @Default("false") + @Default("true") public abstract boolean enableCompression(); } diff --git a/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java index d61265b501c..3c4bb2a2c6f 100644 --- a/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java +++ b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java @@ -104,8 +104,7 @@ public class CuratorInventoryManager } catch (Exception e) { synchronized (lock) { - try { - stop(); + try { stop(); } catch (IOException e1) { log.error(e1, "Exception when stopping InventoryManager that couldn't start."); diff --git a/server/src/main/java/io/druid/guice/DruidProcessingModule.java b/server/src/main/java/io/druid/guice/DruidProcessingModule.java index 86948316416..714b47220cc 100644 --- a/server/src/main/java/io/druid/guice/DruidProcessingModule.java +++ b/server/src/main/java/io/druid/guice/DruidProcessingModule.java @@ -19,7 +19,6 @@ package io.druid.guice; -import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; import com.google.inject.Module; @@ -34,13 +33,13 @@ import io.druid.collections.StupidPool; import io.druid.common.utils.VMUtils; import io.druid.guice.annotations.Global; import io.druid.guice.annotations.Processing; +import io.druid.offheap.OffheapBufferPool; import io.druid.query.DruidProcessingConfig; import io.druid.query.MetricsEmittingExecutorService; import io.druid.query.PrioritizedExecutorService; import java.nio.ByteBuffer; import java.util.concurrent.ExecutorService; -import java.util.concurrent.atomic.AtomicLong; /** */ @@ -99,31 +98,8 @@ public class DruidProcessingModule implements Module log.info(e.getMessage()); } - return new IntermediateProcessingBufferPool(config.intermediateComputeSizeBytes()); + return new OffheapBufferPool(config.intermediateComputeSizeBytes()); } - private static class IntermediateProcessingBufferPool extends StupidPool - { - private static final Logger log = new Logger(IntermediateProcessingBufferPool.class); - public IntermediateProcessingBufferPool(final int computationBufferSize) - { - super( - new Supplier() - { - final AtomicLong count = new AtomicLong(0); - - @Override - public ByteBuffer get() - { - log.info( - "Allocating new intermediate processing buffer[%,d] of size[%,d]", - count.getAndIncrement(), computationBufferSize - ); - return ByteBuffer.allocateDirect(computationBufferSize); - } - } - ); - } - } } diff --git a/server/src/main/java/io/druid/offheap/OffheapBufferPool.java b/server/src/main/java/io/druid/offheap/OffheapBufferPool.java new file mode 100644 index 00000000000..77c70f10583 --- /dev/null +++ b/server/src/main/java/io/druid/offheap/OffheapBufferPool.java @@ -0,0 +1,53 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.offheap; + +import com.google.common.base.Supplier; +import com.metamx.common.logger.Logger; +import io.druid.collections.StupidPool; + +import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicLong; + + +public class OffheapBufferPool extends StupidPool +{ + private static final Logger log = new Logger(OffheapBufferPool.class); + + public OffheapBufferPool(final int computationBufferSize) + { + super( + new Supplier() + { + final AtomicLong count = new AtomicLong(0); + + @Override + public ByteBuffer get() + { + log.info( + "Allocating new intermediate processing buffer[%,d] of size[%,d]", + count.getAndIncrement(), computationBufferSize + ); + return ByteBuffer.allocateDirect(computationBufferSize); + } + } + ); + } +} diff --git a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java index 24704e6e11e..fa0c37b2cfd 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -44,6 +44,9 @@ public class RealtimeTuningConfig implements TuningConfig private static final RejectionPolicyFactory defaultRejectionPolicyFactory = new ServerTimeRejectionPolicyFactory(); private static final int defaultMaxPendingPersists = 0; private static final ShardSpec defaultShardSpec = new NoneShardSpec(); + private static final boolean defaultPersistInHeap = false; + private static final boolean defaultIngestOffheap = false; + // Might make sense for this to be a builder public static RealtimeTuningConfig makeDefaultTuningConfig() @@ -56,7 +59,9 @@ public class RealtimeTuningConfig implements TuningConfig defaultVersioningPolicy, defaultRejectionPolicyFactory, defaultMaxPendingPersists, - defaultShardSpec + defaultShardSpec, + defaultPersistInHeap, + defaultIngestOffheap ); } @@ -68,6 +73,8 @@ public class RealtimeTuningConfig implements TuningConfig private final RejectionPolicyFactory rejectionPolicyFactory; private final int maxPendingPersists; private final ShardSpec shardSpec; + private final boolean persistInHeap; + private final boolean ingestOffheap; @JsonCreator public RealtimeTuningConfig( @@ -78,7 +85,9 @@ public class RealtimeTuningConfig implements TuningConfig @JsonProperty("versioningPolicy") VersioningPolicy versioningPolicy, @JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory, @JsonProperty("maxPendingPersists") Integer maxPendingPersists, - @JsonProperty("shardSpec") ShardSpec shardSpec + @JsonProperty("shardSpec") ShardSpec shardSpec, + @JsonProperty("persistInHeap") Boolean persistInHeap, + @JsonProperty("ingestOffheap") Boolean ingestOffheap ) { this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory; @@ -93,6 +102,8 @@ public class RealtimeTuningConfig implements TuningConfig : rejectionPolicyFactory; this.maxPendingPersists = maxPendingPersists == null ? defaultMaxPendingPersists : maxPendingPersists; this.shardSpec = shardSpec == null ? defaultShardSpec : shardSpec; + this.persistInHeap = persistInHeap == null ? defaultPersistInHeap : persistInHeap; + this.ingestOffheap = ingestOffheap == null ? defaultIngestOffheap : ingestOffheap; } @JsonProperty @@ -143,6 +154,17 @@ public class RealtimeTuningConfig implements TuningConfig return shardSpec; } + @JsonProperty + public boolean isPersistInHeap() + { + return persistInHeap; + } + + @JsonProperty + public boolean isIngestOffheap(){ + return ingestOffheap; + } + public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy) { return new RealtimeTuningConfig( @@ -153,7 +175,9 @@ public class RealtimeTuningConfig implements TuningConfig policy, rejectionPolicyFactory, maxPendingPersists, - shardSpec + shardSpec, + persistInHeap, + ingestOffheap ); } @@ -167,7 +191,9 @@ public class RealtimeTuningConfig implements TuningConfig versioningPolicy, rejectionPolicyFactory, maxPendingPersists, - shardSpec + shardSpec, + persistInHeap, + ingestOffheap ); } } diff --git a/server/src/main/java/io/druid/segment/realtime/FireDepartment.java b/server/src/main/java/io/druid/segment/realtime/FireDepartment.java index 580d4f6922c..3e2211f51fd 100644 --- a/server/src/main/java/io/druid/segment/realtime/FireDepartment.java +++ b/server/src/main/java/io/druid/segment/realtime/FireDepartment.java @@ -96,7 +96,9 @@ public class FireDepartment extends IngestionSpec apply(FireHydrant input) { - return factory.createRunner(input.getSegment()); + // Prevent the underlying segment from closing when its being iterated + final Closeable closeable = input.getSegment().increment(); + try { + return factory.createRunner(input.getSegment()); + } + finally { + try { + closeable.close(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } } } ) @@ -345,11 +360,20 @@ public class RealtimePlumber implements Plumber indexes.add(queryableIndex); } - final File mergedFile = IndexMerger.mergeQueryableIndex( - indexes, - schema.getAggregators(), - mergedTarget - ); + final File mergedFile; + if (config.isPersistInHeap()) { + mergedFile = IndexMaker.mergeQueryableIndex( + indexes, + schema.getAggregators(), + mergedTarget + ); + } else { + mergedFile = IndexMerger.mergeQueryableIndex( + indexes, + schema.getAggregators(), + mergedTarget + ); + } QueryableIndex index = IndexIO.loadIndex(mergedFile); @@ -729,10 +753,18 @@ public class RealtimePlumber implements Plumber try { int numRows = indexToPersist.getIndex().size(); - File persistedFile = IndexMerger.persist( - indexToPersist.getIndex(), - new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())) - ); + final File persistedFile; + if (config.isPersistInHeap()) { + persistedFile = IndexMaker.persist( + indexToPersist.getIndex(), + new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())) + ); + } else { + persistedFile = IndexMerger.persist( + indexToPersist.getIndex(), + new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())) + ); + } indexToPersist.swapSegment( new QueryableIndexSegment( @@ -740,7 +772,6 @@ public class RealtimePlumber implements Plumber IndexIO.loadIndex(persistedFile) ) ); - return numRows; } catch (IOException e) { diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java index cb95217a23a..d3e9a0b26a2 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java @@ -26,11 +26,12 @@ import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.metamx.common.IAE; import com.metamx.common.ISE; -import com.metamx.common.logger.Logger; import io.druid.data.input.InputRow; +import io.druid.offheap.OffheapBufferPool; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.segment.incremental.OffheapIncrementalIndex; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.realtime.FireHydrant; @@ -47,16 +48,14 @@ import java.util.concurrent.CopyOnWriteArrayList; */ public class Sink implements Iterable { - private static final Logger log = new Logger(Sink.class); - private volatile FireHydrant currHydrant; private final Object hydrantLock = new Object(); - private final Interval interval; private final DataSchema schema; private final RealtimeTuningConfig config; private final String version; private final CopyOnWriteArrayList hydrants = new CopyOnWriteArrayList(); + private volatile FireHydrant currHydrant; public Sink( Interval interval, @@ -177,14 +176,29 @@ public class Sink implements Iterable private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) { - IncrementalIndex newIndex = new IncrementalIndex( - new IncrementalIndexSchema.Builder() - .withMinTimestamp(minTimestamp) - .withQueryGranularity(schema.getGranularitySpec().getQueryGranularity()) - .withSpatialDimensions(schema.getParser()) - .withMetrics(schema.getAggregators()) - .build() - ); + int aggsSize = 0; + for (AggregatorFactory agg : schema.getAggregators()) { + aggsSize += agg.getMaxIntermediateSize(); + } + int bufferSize = aggsSize * config.getMaxRowsInMemory(); + final IncrementalIndexSchema indexSchema = new IncrementalIndexSchema.Builder() + .withMinTimestamp(minTimestamp) + .withQueryGranularity(schema.getGranularitySpec().getQueryGranularity()) + .withDimensionsSpec(schema.getParser()) + .withMetrics(schema.getAggregators()) + .build(); + final IncrementalIndex newIndex; + if (config.isIngestOffheap()) { + newIndex = new OffheapIncrementalIndex( + indexSchema, + new OffheapBufferPool(bufferSize) + ); + } else { + newIndex = new IncrementalIndex( + indexSchema, + new OffheapBufferPool(bufferSize) + ); + } final FireHydrant old; synchronized (hydrantLock) { diff --git a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java index 311f0162c5a..38ec31c1bc8 100644 --- a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java @@ -34,6 +34,8 @@ import io.druid.query.QueryRunner; import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChestWarehouse; +import io.druid.query.RetryQueryRunner; +import io.druid.query.RetryQueryRunnerConfig; import io.druid.query.SegmentDescriptor; import io.druid.query.UnionQueryRunner; import org.joda.time.Interval; @@ -48,6 +50,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker private final ServiceEmitter emitter; private final CachingClusteredClient baseClient; private final QueryToolChestWarehouse warehouse; + private final RetryQueryRunnerConfig retryConfig; private final ObjectMapper objectMapper; @Inject @@ -55,12 +58,14 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker ServiceEmitter emitter, CachingClusteredClient baseClient, QueryToolChestWarehouse warehouse, + RetryQueryRunnerConfig retryConfig, ObjectMapper objectMapper ) { this.emitter = emitter; this.baseClient = baseClient; this.warehouse = warehouse; + this.retryConfig = retryConfig; this.objectMapper = objectMapper; } @@ -82,7 +87,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker final FinalizeResultsQueryRunner baseRunner = new FinalizeResultsQueryRunner( toolChest.postMergeQueryDecoration( toolChest.mergeResults( - new UnionQueryRunner( + new UnionQueryRunner<>( new MetricsEmittingQueryRunner( emitter, new Function, ServiceMetricEvent.Builder>() @@ -93,7 +98,13 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker return toolChest.makeMetricBuilder(query); } }, - toolChest.preMergeQueryDecoration(baseClient) + toolChest.preMergeQueryDecoration( + new RetryQueryRunner( + baseClient, + retryConfig, + objectMapper + ) + ) ).withWaitMeasuredFromNow(), toolChest ) @@ -105,7 +116,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker final Map context = query.getContext(); PostProcessingOperator postProcessing = null; - if(context != null) { + if (context != null) { postProcessing = objectMapper.convertValue( context.get("postProcessing"), new TypeReference>() diff --git a/server/src/main/java/io/druid/server/DruidNode.java b/server/src/main/java/io/druid/server/DruidNode.java index 4c8528bd6ee..1928d6487d8 100644 --- a/server/src/main/java/io/druid/server/DruidNode.java +++ b/server/src/main/java/io/druid/server/DruidNode.java @@ -19,8 +19,10 @@ package io.druid.server; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.inject.name.Named; import io.druid.common.utils.SocketUtil; import javax.validation.constraints.Max; @@ -31,15 +33,17 @@ import javax.validation.constraints.NotNull; */ public class DruidNode { + public static final String DEFAULT_HOST = "localhost"; + private String hostNoPort; @JsonProperty("service") @NotNull - private String serviceName = null; + private String serviceName; @JsonProperty @NotNull - private String host = null; + private String host; @JsonProperty @Min(0) @Max(0xffff) @@ -47,16 +51,21 @@ public class DruidNode @JsonCreator public DruidNode( - @JsonProperty("service") String serviceName, + @JacksonInject @Named("serviceName") @JsonProperty("service") String serviceName, @JsonProperty("host") String host, - @JsonProperty("port") Integer port + @JacksonInject @Named("servicePort") @JsonProperty("port") Integer port ) + { + init(serviceName, host, port); + } + + private void init(String serviceName, String host, Integer port) { this.serviceName = serviceName; if (port == null) { if (host == null) { - setHostAndPort(null, -1, null); + setHostAndPort(DEFAULT_HOST, -1, DEFAULT_HOST); } else if (host.contains(":")) { final String[] hostParts = host.split(":"); @@ -74,7 +83,7 @@ public class DruidNode } else { if (host == null || host.contains(":")) { - setHostAndPort(host, port, host == null ? null : host.split(":")[0]); + setHostAndPort(host == null ? DEFAULT_HOST : host, port, host == null ? DEFAULT_HOST : host.split(":")[0]); } else { setHostAndPort(String.format("%s:%d", host, port), port, host); diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 0ce414f1b8e..b66b080f6b9 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.ObjectWriter; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.MapMaker; import com.google.common.io.ByteStreams; import com.google.inject.Inject; import com.metamx.common.guava.Sequence; @@ -39,6 +40,7 @@ import io.druid.query.Query; import io.druid.query.QueryInterruptedException; import io.druid.query.QueryMetricUtil; import io.druid.query.QuerySegmentWalker; +import io.druid.query.RetryQueryRunner; import io.druid.server.initialization.ServerConfig; import io.druid.server.log.RequestLogger; import org.joda.time.DateTime; @@ -57,6 +59,8 @@ import javax.ws.rs.core.Response; import javax.ws.rs.core.StreamingOutput; import java.io.IOException; import java.io.OutputStream; +import java.util.List; +import java.util.Map; import java.util.UUID; /** @@ -149,7 +153,8 @@ public class QueryResource log.debug("Got query [%s]", query); } - Sequence res = query.run(texasRanger); + final Map context = new MapMaker().makeMap(); + final Sequence res = query.run(texasRanger, context); final Sequence results; if (res == null) { results = Sequences.empty(); @@ -172,6 +177,7 @@ public class QueryResource try { long requestTime = System.currentTimeMillis() - start; + emitter.emit( QueryMetricUtil.makeRequestTimeMetric(jsonMapper, query, req.getRemoteAddr()) .build("request/time", requestTime) @@ -206,6 +212,7 @@ public class QueryResource isSmile ? APPLICATION_JSON : APPLICATION_SMILE ) .header("X-Druid-Query-Id", queryId) + .header("X-Druid-Response-Context", jsonMapper.writeValueAsString(context)) .build(); } catch (Exception e) { diff --git a/server/src/main/java/io/druid/server/bridge/BridgeQuerySegmentWalker.java b/server/src/main/java/io/druid/server/bridge/BridgeQuerySegmentWalker.java index d7e4674fab2..4f260002ccb 100644 --- a/server/src/main/java/io/druid/server/bridge/BridgeQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/server/bridge/BridgeQuerySegmentWalker.java @@ -40,6 +40,7 @@ import org.joda.time.Interval; import java.net.URL; import java.util.List; +import java.util.Map; /** */ @@ -86,7 +87,7 @@ public class BridgeQuerySegmentWalker implements QuerySegmentWalker return new QueryRunner() { @Override - public Sequence run(Query query) + public Sequence run(Query query, Map context) { try { Server instance = brokerSelector.pick(); diff --git a/server/src/main/java/io/druid/server/coordination/DataSegmentAnnouncerProvider.java b/server/src/main/java/io/druid/server/coordination/DataSegmentAnnouncerProvider.java index eeed92b045a..eb52deb13bd 100644 --- a/server/src/main/java/io/druid/server/coordination/DataSegmentAnnouncerProvider.java +++ b/server/src/main/java/io/druid/server/coordination/DataSegmentAnnouncerProvider.java @@ -25,7 +25,7 @@ import com.google.inject.Provider; /** */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LegacyDataSegmentAnnouncerProvider.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = BatchDataSegmentAnnouncerProvider.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "legacy", value = LegacyDataSegmentAnnouncerProvider.class), @JsonSubTypes.Type(name = "batch", value = BatchDataSegmentAnnouncerProvider.class) diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java index 7dbfe07ce15..0d8f4afb24d 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java @@ -454,7 +454,7 @@ public class DruidCoordinator private LeaderLatch createNewLeaderLatch() { final LeaderLatch newLeaderLatch = new LeaderLatch( - curator, ZKPaths.makePath(zkPaths.getCoordinatorPath(), COORDINATOR_OWNER_NODE), config.getHost() + curator, ZKPaths.makePath(zkPaths.getCoordinatorPath(), COORDINATOR_OWNER_NODE), self.getHost() ); newLeaderLatch.addListener( diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java index 571e70ee3bd..e58747dfb95 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java @@ -27,9 +27,6 @@ import org.skife.config.Default; */ public abstract class DruidCoordinatorConfig { - @Config("druid.host") - public abstract String getHost(); - @Config("druid.coordinator.startDelay") @Default("PT300s") public abstract Duration getCoordinatorStartDelay(); diff --git a/server/src/main/java/io/druid/server/initialization/EmitterModule.java b/server/src/main/java/io/druid/server/initialization/EmitterModule.java index c7b29d3af53..410bd70ac82 100644 --- a/server/src/main/java/io/druid/server/initialization/EmitterModule.java +++ b/server/src/main/java/io/druid/server/initialization/EmitterModule.java @@ -67,6 +67,7 @@ public class EmitterModule implements Module { String emitterType = props.getProperty(EMITTER_PROPERTY, ""); + binder.install(new NoopEmitterModule()); binder.install(new LogEmitterModule()); binder.install(new HttpEmitterModule()); @@ -104,7 +105,7 @@ public class EmitterModule implements Module emitter = findEmitter(emitterType, emitterBindings); if (emitter == null) { - emitter = findEmitter(LogEmitterModule.EMITTER_TYPE, emitterBindings); + emitter = findEmitter(NoopEmitterModule.EMITTER_TYPE, emitterBindings); } if (emitter == null) { @@ -115,7 +116,7 @@ public class EmitterModule implements Module knownTypes.add(((Named) annotation).value()); } } - throw new ISE("Uknown emitter type[%s]=[%s], known types[%s]", EMITTER_PROPERTY, emitterType, knownTypes); + throw new ISE("Unknown emitter type[%s]=[%s], known types[%s]", EMITTER_PROPERTY, emitterType, knownTypes); } } diff --git a/server/src/main/java/io/druid/server/initialization/NoopEmitterModule.java b/server/src/main/java/io/druid/server/initialization/NoopEmitterModule.java new file mode 100644 index 00000000000..f874d27021e --- /dev/null +++ b/server/src/main/java/io/druid/server/initialization/NoopEmitterModule.java @@ -0,0 +1,48 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.server.initialization; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.name.Named; +import com.metamx.emitter.core.Emitter; +import com.metamx.emitter.core.NoopEmitter; +import io.druid.guice.ManageLifecycle; + +/** + */ +public class NoopEmitterModule implements Module +{ + public static final String EMITTER_TYPE = "noop"; + + @Override + public void configure(Binder binder) + { + } + + @Provides + @ManageLifecycle + @Named(EMITTER_TYPE) + public Emitter makeEmitter() + { + return new NoopEmitter(); + } +} diff --git a/server/src/main/resources/static/css/bootstrap-theme.css b/server/src/main/resources/static/css/bootstrap-theme.css new file mode 100644 index 00000000000..a4069929bce --- /dev/null +++ b/server/src/main/resources/static/css/bootstrap-theme.css @@ -0,0 +1,347 @@ +/*! + * Bootstrap v3.1.1 (http://getbootstrap.com) + * Copyright 2011-2014 Twitter, Inc. + * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE) + */ + +.btn-default, +.btn-primary, +.btn-success, +.btn-info, +.btn-warning, +.btn-danger { + text-shadow: 0 -1px 0 rgba(0, 0, 0, .2); + -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, .15), 0 1px 1px rgba(0, 0, 0, .075); + box-shadow: inset 0 1px 0 rgba(255, 255, 255, .15), 0 1px 1px rgba(0, 0, 0, .075); +} +.btn-default:active, +.btn-primary:active, +.btn-success:active, +.btn-info:active, +.btn-warning:active, +.btn-danger:active, +.btn-default.active, +.btn-primary.active, +.btn-success.active, +.btn-info.active, +.btn-warning.active, +.btn-danger.active { + -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125); + box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125); +} +.btn:active, +.btn.active { + background-image: none; +} +.btn-default { + text-shadow: 0 1px 0 #fff; + background-image: -webkit-linear-gradient(top, #fff 0%, #e0e0e0 100%); + background-image: linear-gradient(to bottom, #fff 0%, #e0e0e0 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff', endColorstr='#ffe0e0e0', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + background-repeat: repeat-x; + border-color: #dbdbdb; + border-color: #ccc; +} +.btn-default:hover, +.btn-default:focus { + background-color: #e0e0e0; + background-position: 0 -15px; +} +.btn-default:active, +.btn-default.active { + background-color: #e0e0e0; + border-color: #dbdbdb; +} +.btn-primary { + background-image: -webkit-linear-gradient(top, #428bca 0%, #2d6ca2 100%); + background-image: linear-gradient(to bottom, #428bca 0%, #2d6ca2 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff428bca', endColorstr='#ff2d6ca2', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + background-repeat: repeat-x; + border-color: #2b669a; +} +.btn-primary:hover, +.btn-primary:focus { + background-color: #2d6ca2; + background-position: 0 -15px; +} +.btn-primary:active, +.btn-primary.active { + background-color: #2d6ca2; + border-color: #2b669a; +} +.btn-success { + background-image: -webkit-linear-gradient(top, #5cb85c 0%, #419641 100%); + background-image: linear-gradient(to bottom, #5cb85c 0%, #419641 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5cb85c', endColorstr='#ff419641', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + background-repeat: repeat-x; + border-color: #3e8f3e; +} +.btn-success:hover, +.btn-success:focus { + background-color: #419641; + background-position: 0 -15px; +} +.btn-success:active, +.btn-success.active { + background-color: #419641; + border-color: #3e8f3e; +} +.btn-info { + background-image: -webkit-linear-gradient(top, #5bc0de 0%, #2aabd2 100%); + background-image: linear-gradient(to bottom, #5bc0de 0%, #2aabd2 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de', endColorstr='#ff2aabd2', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + background-repeat: repeat-x; + border-color: #28a4c9; +} +.btn-info:hover, +.btn-info:focus { + background-color: #2aabd2; + background-position: 0 -15px; +} +.btn-info:active, +.btn-info.active { + background-color: #2aabd2; + border-color: #28a4c9; +} +.btn-warning { + background-image: -webkit-linear-gradient(top, #f0ad4e 0%, #eb9316 100%); + background-image: linear-gradient(to bottom, #f0ad4e 0%, #eb9316 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff0ad4e', endColorstr='#ffeb9316', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + background-repeat: repeat-x; + border-color: #e38d13; +} +.btn-warning:hover, +.btn-warning:focus { + background-color: #eb9316; + background-position: 0 -15px; +} +.btn-warning:active, +.btn-warning.active { + background-color: #eb9316; + border-color: #e38d13; +} +.btn-danger { + background-image: -webkit-linear-gradient(top, #d9534f 0%, #c12e2a 100%); + background-image: linear-gradient(to bottom, #d9534f 0%, #c12e2a 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9534f', endColorstr='#ffc12e2a', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + background-repeat: repeat-x; + border-color: #b92c28; +} +.btn-danger:hover, +.btn-danger:focus { + background-color: #c12e2a; + background-position: 0 -15px; +} +.btn-danger:active, +.btn-danger.active { + background-color: #c12e2a; + border-color: #b92c28; +} +.thumbnail, +.img-thumbnail { + -webkit-box-shadow: 0 1px 2px rgba(0, 0, 0, .075); + box-shadow: 0 1px 2px rgba(0, 0, 0, .075); +} +.dropdown-menu > li > a:hover, +.dropdown-menu > li > a:focus { + background-color: #e8e8e8; + background-image: -webkit-linear-gradient(top, #f5f5f5 0%, #e8e8e8 100%); + background-image: linear-gradient(to bottom, #f5f5f5 0%, #e8e8e8 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff5f5f5', endColorstr='#ffe8e8e8', GradientType=0); + background-repeat: repeat-x; +} +.dropdown-menu > .active > a, +.dropdown-menu > .active > a:hover, +.dropdown-menu > .active > a:focus { + background-color: #357ebd; + background-image: -webkit-linear-gradient(top, #428bca 0%, #357ebd 100%); + background-image: linear-gradient(to bottom, #428bca 0%, #357ebd 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff428bca', endColorstr='#ff357ebd', GradientType=0); + background-repeat: repeat-x; +} +.navbar-default { + background-image: -webkit-linear-gradient(top, #fff 0%, #f8f8f8 100%); + background-image: linear-gradient(to bottom, #fff 0%, #f8f8f8 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff', endColorstr='#fff8f8f8', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + background-repeat: repeat-x; + border-radius: 4px; + -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, .15), 0 1px 5px rgba(0, 0, 0, .075); + box-shadow: inset 0 1px 0 rgba(255, 255, 255, .15), 0 1px 5px rgba(0, 0, 0, .075); +} +.navbar-default .navbar-nav > .active > a { + background-image: -webkit-linear-gradient(top, #ebebeb 0%, #f3f3f3 100%); + background-image: linear-gradient(to bottom, #ebebeb 0%, #f3f3f3 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffebebeb', endColorstr='#fff3f3f3', GradientType=0); + background-repeat: repeat-x; + -webkit-box-shadow: inset 0 3px 9px rgba(0, 0, 0, .075); + box-shadow: inset 0 3px 9px rgba(0, 0, 0, .075); +} +.navbar-brand, +.navbar-nav > li > a { + text-shadow: 0 1px 0 rgba(255, 255, 255, .25); +} +.navbar-inverse { + background-image: -webkit-linear-gradient(top, #3c3c3c 0%, #222 100%); + background-image: linear-gradient(to bottom, #3c3c3c 0%, #222 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff3c3c3c', endColorstr='#ff222222', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + background-repeat: repeat-x; +} +.navbar-inverse .navbar-nav > .active > a { + background-image: -webkit-linear-gradient(top, #222 0%, #282828 100%); + background-image: linear-gradient(to bottom, #222 0%, #282828 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff222222', endColorstr='#ff282828', GradientType=0); + background-repeat: repeat-x; + -webkit-box-shadow: inset 0 3px 9px rgba(0, 0, 0, .25); + box-shadow: inset 0 3px 9px rgba(0, 0, 0, .25); +} +.navbar-inverse .navbar-brand, +.navbar-inverse .navbar-nav > li > a { + text-shadow: 0 -1px 0 rgba(0, 0, 0, .25); +} +.navbar-static-top, +.navbar-fixed-top, +.navbar-fixed-bottom { + border-radius: 0; +} +.alert { + text-shadow: 0 1px 0 rgba(255, 255, 255, .2); + -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, .25), 0 1px 2px rgba(0, 0, 0, .05); + box-shadow: inset 0 1px 0 rgba(255, 255, 255, .25), 0 1px 2px rgba(0, 0, 0, .05); +} +.alert-success { + background-image: -webkit-linear-gradient(top, #dff0d8 0%, #c8e5bc 100%); + background-image: linear-gradient(to bottom, #dff0d8 0%, #c8e5bc 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffdff0d8', endColorstr='#ffc8e5bc', GradientType=0); + background-repeat: repeat-x; + border-color: #b2dba1; +} +.alert-info { + background-image: -webkit-linear-gradient(top, #d9edf7 0%, #b9def0 100%); + background-image: linear-gradient(to bottom, #d9edf7 0%, #b9def0 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9edf7', endColorstr='#ffb9def0', GradientType=0); + background-repeat: repeat-x; + border-color: #9acfea; +} +.alert-warning { + background-image: -webkit-linear-gradient(top, #fcf8e3 0%, #f8efc0 100%); + background-image: linear-gradient(to bottom, #fcf8e3 0%, #f8efc0 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffcf8e3', endColorstr='#fff8efc0', GradientType=0); + background-repeat: repeat-x; + border-color: #f5e79e; +} +.alert-danger { + background-image: -webkit-linear-gradient(top, #f2dede 0%, #e7c3c3 100%); + background-image: linear-gradient(to bottom, #f2dede 0%, #e7c3c3 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff2dede', endColorstr='#ffe7c3c3', GradientType=0); + background-repeat: repeat-x; + border-color: #dca7a7; +} +.progress { + background-image: -webkit-linear-gradient(top, #ebebeb 0%, #f5f5f5 100%); + background-image: linear-gradient(to bottom, #ebebeb 0%, #f5f5f5 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffebebeb', endColorstr='#fff5f5f5', GradientType=0); + background-repeat: repeat-x; +} +.progress-bar { + background-image: -webkit-linear-gradient(top, #428bca 0%, #3071a9 100%); + background-image: linear-gradient(to bottom, #428bca 0%, #3071a9 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff428bca', endColorstr='#ff3071a9', GradientType=0); + background-repeat: repeat-x; +} +.progress-bar-success { + background-image: -webkit-linear-gradient(top, #5cb85c 0%, #449d44 100%); + background-image: linear-gradient(to bottom, #5cb85c 0%, #449d44 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5cb85c', endColorstr='#ff449d44', GradientType=0); + background-repeat: repeat-x; +} +.progress-bar-info { + background-image: -webkit-linear-gradient(top, #5bc0de 0%, #31b0d5 100%); + background-image: linear-gradient(to bottom, #5bc0de 0%, #31b0d5 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de', endColorstr='#ff31b0d5', GradientType=0); + background-repeat: repeat-x; +} +.progress-bar-warning { + background-image: -webkit-linear-gradient(top, #f0ad4e 0%, #ec971f 100%); + background-image: linear-gradient(to bottom, #f0ad4e 0%, #ec971f 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff0ad4e', endColorstr='#ffec971f', GradientType=0); + background-repeat: repeat-x; +} +.progress-bar-danger { + background-image: -webkit-linear-gradient(top, #d9534f 0%, #c9302c 100%); + background-image: linear-gradient(to bottom, #d9534f 0%, #c9302c 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9534f', endColorstr='#ffc9302c', GradientType=0); + background-repeat: repeat-x; +} +.list-group { + border-radius: 4px; + -webkit-box-shadow: 0 1px 2px rgba(0, 0, 0, .075); + box-shadow: 0 1px 2px rgba(0, 0, 0, .075); +} +.list-group-item.active, +.list-group-item.active:hover, +.list-group-item.active:focus { + text-shadow: 0 -1px 0 #3071a9; + background-image: -webkit-linear-gradient(top, #428bca 0%, #3278b3 100%); + background-image: linear-gradient(to bottom, #428bca 0%, #3278b3 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff428bca', endColorstr='#ff3278b3', GradientType=0); + background-repeat: repeat-x; + border-color: #3278b3; +} +.panel { + -webkit-box-shadow: 0 1px 2px rgba(0, 0, 0, .05); + box-shadow: 0 1px 2px rgba(0, 0, 0, .05); +} +.panel-default > .panel-heading { + background-image: -webkit-linear-gradient(top, #f5f5f5 0%, #e8e8e8 100%); + background-image: linear-gradient(to bottom, #f5f5f5 0%, #e8e8e8 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff5f5f5', endColorstr='#ffe8e8e8', GradientType=0); + background-repeat: repeat-x; +} +.panel-primary > .panel-heading { + background-image: -webkit-linear-gradient(top, #428bca 0%, #357ebd 100%); + background-image: linear-gradient(to bottom, #428bca 0%, #357ebd 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff428bca', endColorstr='#ff357ebd', GradientType=0); + background-repeat: repeat-x; +} +.panel-success > .panel-heading { + background-image: -webkit-linear-gradient(top, #dff0d8 0%, #d0e9c6 100%); + background-image: linear-gradient(to bottom, #dff0d8 0%, #d0e9c6 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffdff0d8', endColorstr='#ffd0e9c6', GradientType=0); + background-repeat: repeat-x; +} +.panel-info > .panel-heading { + background-image: -webkit-linear-gradient(top, #d9edf7 0%, #c4e3f3 100%); + background-image: linear-gradient(to bottom, #d9edf7 0%, #c4e3f3 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9edf7', endColorstr='#ffc4e3f3', GradientType=0); + background-repeat: repeat-x; +} +.panel-warning > .panel-heading { + background-image: -webkit-linear-gradient(top, #fcf8e3 0%, #faf2cc 100%); + background-image: linear-gradient(to bottom, #fcf8e3 0%, #faf2cc 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffcf8e3', endColorstr='#fffaf2cc', GradientType=0); + background-repeat: repeat-x; +} +.panel-danger > .panel-heading { + background-image: -webkit-linear-gradient(top, #f2dede 0%, #ebcccc 100%); + background-image: linear-gradient(to bottom, #f2dede 0%, #ebcccc 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff2dede', endColorstr='#ffebcccc', GradientType=0); + background-repeat: repeat-x; +} +.well { + background-image: -webkit-linear-gradient(top, #e8e8e8 0%, #f5f5f5 100%); + background-image: linear-gradient(to bottom, #e8e8e8 0%, #f5f5f5 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffe8e8e8', endColorstr='#fff5f5f5', GradientType=0); + background-repeat: repeat-x; + border-color: #dcdcdc; + -webkit-box-shadow: inset 0 1px 3px rgba(0, 0, 0, .05), 0 1px 0 rgba(255, 255, 255, .1); + box-shadow: inset 0 1px 3px rgba(0, 0, 0, .05), 0 1px 0 rgba(255, 255, 255, .1); +} +/*# sourceMappingURL=bootstrap-theme.css.map */ diff --git a/server/src/main/resources/static/css/bootstrap.css b/server/src/main/resources/static/css/bootstrap.css new file mode 100644 index 00000000000..7f36651961e --- /dev/null +++ b/server/src/main/resources/static/css/bootstrap.css @@ -0,0 +1,5785 @@ +/*! + * Bootstrap v3.1.1 (http://getbootstrap.com) + * Copyright 2011-2014 Twitter, Inc. + * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE) + */ + +/*! normalize.css v3.0.0 | MIT License | git.io/normalize */ +html { + font-family: sans-serif; + -webkit-text-size-adjust: 100%; + -ms-text-size-adjust: 100%; +} +body { + margin: 0; +} +article, +aside, +details, +figcaption, +figure, +footer, +header, +hgroup, +main, +nav, +section, +summary { + display: block; +} +audio, +canvas, +progress, +video { + display: inline-block; + vertical-align: baseline; +} +audio:not([controls]) { + display: none; + height: 0; +} +[hidden], +template { + display: none; +} +a { + background: transparent; +} +a:active, +a:hover { + outline: 0; +} +abbr[title] { + border-bottom: 1px dotted; +} +b, +strong { + font-weight: bold; +} +dfn { + font-style: italic; +} +h1 { + margin: .67em 0; + font-size: 2em; +} +mark { + color: #000; + background: #ff0; +} +small { + font-size: 80%; +} +sub, +sup { + position: relative; + font-size: 75%; + line-height: 0; + vertical-align: baseline; +} +sup { + top: -.5em; +} +sub { + bottom: -.25em; +} +img { + border: 0; +} +svg:not(:root) { + overflow: hidden; +} +figure { + margin: 1em 40px; +} +hr { + height: 0; + -moz-box-sizing: content-box; + box-sizing: content-box; +} +pre { + overflow: auto; +} +code, +kbd, +pre, +samp { + font-family: monospace, monospace; + font-size: 1em; +} +button, +input, +optgroup, +select, +textarea { + margin: 0; + font: inherit; + color: inherit; +} +button { + overflow: visible; +} +button, +select { + text-transform: none; +} +button, +html input[type="button"], +input[type="reset"], +input[type="submit"] { + -webkit-appearance: button; + cursor: pointer; +} +button[disabled], +html input[disabled] { + cursor: default; +} +button::-moz-focus-inner, +input::-moz-focus-inner { + padding: 0; + border: 0; +} +input { + line-height: normal; +} +input[type="checkbox"], +input[type="radio"] { + box-sizing: border-box; + padding: 0; +} +input[type="number"]::-webkit-inner-spin-button, +input[type="number"]::-webkit-outer-spin-button { + height: auto; +} +input[type="search"] { + -webkit-box-sizing: content-box; + -moz-box-sizing: content-box; + box-sizing: content-box; + -webkit-appearance: textfield; +} +input[type="search"]::-webkit-search-cancel-button, +input[type="search"]::-webkit-search-decoration { + -webkit-appearance: none; +} +fieldset { + padding: .35em .625em .75em; + margin: 0 2px; + border: 1px solid #c0c0c0; +} +legend { + padding: 0; + border: 0; +} +textarea { + overflow: auto; +} +optgroup { + font-weight: bold; +} +table { + border-spacing: 0; + border-collapse: collapse; +} +td, +th { + padding: 0; +} +@media print { + * { + color: #000 !important; + text-shadow: none !important; + background: transparent !important; + box-shadow: none !important; + } + a, + a:visited { + text-decoration: underline; + } + a[href]:after { + content: " (" attr(href) ")"; + } + abbr[title]:after { + content: " (" attr(title) ")"; + } + a[href^="javascript:"]:after, + a[href^="#"]:after { + content: ""; + } + pre, + blockquote { + border: 1px solid #999; + + page-break-inside: avoid; + } + thead { + display: table-header-group; + } + tr, + img { + page-break-inside: avoid; + } + img { + max-width: 100% !important; + } + p, + h2, + h3 { + orphans: 3; + widows: 3; + } + h2, + h3 { + page-break-after: avoid; + } + select { + background: #fff !important; + } + .navbar { + display: none; + } + .table td, + .table th { + background-color: #fff !important; + } + .btn > .caret, + .dropup > .btn > .caret { + border-top-color: #000 !important; + } + .label { + border: 1px solid #000; + } + .table { + border-collapse: collapse !important; + } + .table-bordered th, + .table-bordered td { + border: 1px solid #ddd !important; + } +} +* { + -webkit-box-sizing: border-box; + -moz-box-sizing: border-box; + box-sizing: border-box; +} +*:before, +*:after { + -webkit-box-sizing: border-box; + -moz-box-sizing: border-box; + box-sizing: border-box; +} +html { + font-size: 62.5%; + + -webkit-tap-highlight-color: rgba(0, 0, 0, 0); +} +body { + font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; + font-size: 14px; + line-height: 1.42857143; + color: #333; + background-color: #fff; +} +input, +button, +select, +textarea { + font-family: inherit; + font-size: inherit; + line-height: inherit; +} +a { + color: #428bca; + text-decoration: none; +} +a:hover, +a:focus { + color: #2a6496; + text-decoration: underline; +} +a:focus { + outline: thin dotted; + outline: 5px auto -webkit-focus-ring-color; + outline-offset: -2px; +} +figure { + margin: 0; +} +img { + vertical-align: middle; +} +.img-responsive, +.thumbnail > img, +.thumbnail a > img, +.carousel-inner > .item > img, +.carousel-inner > .item > a > img { + display: block; + max-width: 100%; + height: auto; +} +.img-rounded { + border-radius: 6px; +} +.img-thumbnail { + display: inline-block; + max-width: 100%; + height: auto; + padding: 4px; + line-height: 1.42857143; + background-color: #fff; + border: 1px solid #ddd; + border-radius: 4px; + -webkit-transition: all .2s ease-in-out; + transition: all .2s ease-in-out; +} +.img-circle { + border-radius: 50%; +} +hr { + margin-top: 20px; + margin-bottom: 20px; + border: 0; + border-top: 1px solid #eee; +} +.sr-only { + position: absolute; + width: 1px; + height: 1px; + padding: 0; + margin: -1px; + overflow: hidden; + clip: rect(0, 0, 0, 0); + border: 0; +} +h1, +h2, +h3, +h4, +h5, +h6, +.h1, +.h2, +.h3, +.h4, +.h5, +.h6 { + font-family: inherit; + font-weight: 500; + line-height: 1.1; + color: inherit; +} +h1 small, +h2 small, +h3 small, +h4 small, +h5 small, +h6 small, +.h1 small, +.h2 small, +.h3 small, +.h4 small, +.h5 small, +.h6 small, +h1 .small, +h2 .small, +h3 .small, +h4 .small, +h5 .small, +h6 .small, +.h1 .small, +.h2 .small, +.h3 .small, +.h4 .small, +.h5 .small, +.h6 .small { + font-weight: normal; + line-height: 1; + color: #999; +} +h1, +.h1, +h2, +.h2, +h3, +.h3 { + margin-top: 20px; + margin-bottom: 10px; +} +h1 small, +.h1 small, +h2 small, +.h2 small, +h3 small, +.h3 small, +h1 .small, +.h1 .small, +h2 .small, +.h2 .small, +h3 .small, +.h3 .small { + font-size: 65%; +} +h4, +.h4, +h5, +.h5, +h6, +.h6 { + margin-top: 10px; + margin-bottom: 10px; +} +h4 small, +.h4 small, +h5 small, +.h5 small, +h6 small, +.h6 small, +h4 .small, +.h4 .small, +h5 .small, +.h5 .small, +h6 .small, +.h6 .small { + font-size: 75%; +} +h1, +.h1 { + font-size: 36px; +} +h2, +.h2 { + font-size: 30px; +} +h3, +.h3 { + font-size: 24px; +} +h4, +.h4 { + font-size: 18px; +} +h5, +.h5 { + font-size: 14px; +} +h6, +.h6 { + font-size: 12px; +} +p { + margin: 0 0 10px; +} +.lead { + margin-bottom: 20px; + font-size: 16px; + font-weight: 200; + line-height: 1.4; +} +@media (min-width: 768px) { + .lead { + font-size: 21px; + } +} +small, +.small { + font-size: 85%; +} +cite { + font-style: normal; +} +.text-left { + text-align: left; +} +.text-right { + text-align: right; +} +.text-center { + text-align: center; +} +.text-justify { + text-align: justify; +} +.text-muted { + color: #999; +} +.text-primary { + color: #428bca; +} +a.text-primary:hover { + color: #3071a9; +} +.text-success { + color: #3c763d; +} +a.text-success:hover { + color: #2b542c; +} +.text-info { + color: #31708f; +} +a.text-info:hover { + color: #245269; +} +.text-warning { + color: #8a6d3b; +} +a.text-warning:hover { + color: #66512c; +} +.text-danger { + color: #a94442; +} +a.text-danger:hover { + color: #843534; +} +.bg-primary { + color: #fff; + background-color: #428bca; +} +a.bg-primary:hover { + background-color: #3071a9; +} +.bg-success { + background-color: #dff0d8; +} +a.bg-success:hover { + background-color: #c1e2b3; +} +.bg-info { + background-color: #d9edf7; +} +a.bg-info:hover { + background-color: #afd9ee; +} +.bg-warning { + background-color: #fcf8e3; +} +a.bg-warning:hover { + background-color: #f7ecb5; +} +.bg-danger { + background-color: #f2dede; +} +a.bg-danger:hover { + background-color: #e4b9b9; +} +.page-header { + padding-bottom: 9px; + margin: 40px 0 20px; + border-bottom: 1px solid #eee; +} +ul, +ol { + margin-top: 0; + margin-bottom: 10px; +} +ul ul, +ol ul, +ul ol, +ol ol { + margin-bottom: 0; +} +.list-unstyled { + padding-left: 0; + list-style: none; +} +.list-inline { + padding-left: 0; + margin-left: -5px; + list-style: none; +} +.list-inline > li { + display: inline-block; + padding-right: 5px; + padding-left: 5px; +} +dl { + margin-top: 0; + margin-bottom: 20px; +} +dt, +dd { + line-height: 1.42857143; +} +dt { + font-weight: bold; +} +dd { + margin-left: 0; +} +@media (min-width: 768px) { + .dl-horizontal dt { + float: left; + width: 160px; + overflow: hidden; + clear: left; + text-align: right; + text-overflow: ellipsis; + white-space: nowrap; + } + .dl-horizontal dd { + margin-left: 180px; + } +} +abbr[title], +abbr[data-original-title] { + cursor: help; + border-bottom: 1px dotted #999; +} +.initialism { + font-size: 90%; + text-transform: uppercase; +} +blockquote { + padding: 10px 20px; + margin: 0 0 20px; + font-size: 17.5px; + border-left: 5px solid #eee; +} +blockquote p:last-child, +blockquote ul:last-child, +blockquote ol:last-child { + margin-bottom: 0; +} +blockquote footer, +blockquote small, +blockquote .small { + display: block; + font-size: 80%; + line-height: 1.42857143; + color: #999; +} +blockquote footer:before, +blockquote small:before, +blockquote .small:before { + content: '\2014 \00A0'; +} +.blockquote-reverse, +blockquote.pull-right { + padding-right: 15px; + padding-left: 0; + text-align: right; + border-right: 5px solid #eee; + border-left: 0; +} +.blockquote-reverse footer:before, +blockquote.pull-right footer:before, +.blockquote-reverse small:before, +blockquote.pull-right small:before, +.blockquote-reverse .small:before, +blockquote.pull-right .small:before { + content: ''; +} +.blockquote-reverse footer:after, +blockquote.pull-right footer:after, +.blockquote-reverse small:after, +blockquote.pull-right small:after, +.blockquote-reverse .small:after, +blockquote.pull-right .small:after { + content: '\00A0 \2014'; +} +blockquote:before, +blockquote:after { + content: ""; +} +address { + margin-bottom: 20px; + font-style: normal; + line-height: 1.42857143; +} +code, +kbd, +pre, +samp { + font-family: Menlo, Monaco, Consolas, "Courier New", monospace; +} +code { + padding: 2px 4px; + font-size: 90%; + color: #c7254e; + white-space: nowrap; + background-color: #f9f2f4; + border-radius: 4px; +} +kbd { + padding: 2px 4px; + font-size: 90%; + color: #fff; + background-color: #333; + border-radius: 3px; + box-shadow: inset 0 -1px 0 rgba(0, 0, 0, .25); +} +pre { + display: block; + padding: 9.5px; + margin: 0 0 10px; + font-size: 13px; + line-height: 1.42857143; + color: #333; + word-break: break-all; + word-wrap: break-word; + background-color: #f5f5f5; + border: 1px solid #ccc; + border-radius: 4px; +} +pre code { + padding: 0; + font-size: inherit; + color: inherit; + white-space: pre-wrap; + background-color: transparent; + border-radius: 0; +} +.pre-scrollable { + max-height: 340px; + overflow-y: scroll; +} +.container { + padding-right: 15px; + padding-left: 15px; + margin-right: auto; + margin-left: auto; +} +@media (min-width: 768px) { + .container { + width: 750px; + } +} +@media (min-width: 992px) { + .container { + width: 970px; + } +} +@media (min-width: 1200px) { + .container { + width: 1170px; + } +} +.container-fluid { + padding-right: 15px; + padding-left: 15px; + margin-right: auto; + margin-left: auto; +} +.row { + margin-right: -15px; + margin-left: -15px; +} +.col-xs-1, .col-sm-1, .col-md-1, .col-lg-1, .col-xs-2, .col-sm-2, .col-md-2, .col-lg-2, .col-xs-3, .col-sm-3, .col-md-3, .col-lg-3, .col-xs-4, .col-sm-4, .col-md-4, .col-lg-4, .col-xs-5, .col-sm-5, .col-md-5, .col-lg-5, .col-xs-6, .col-sm-6, .col-md-6, .col-lg-6, .col-xs-7, .col-sm-7, .col-md-7, .col-lg-7, .col-xs-8, .col-sm-8, .col-md-8, .col-lg-8, .col-xs-9, .col-sm-9, .col-md-9, .col-lg-9, .col-xs-10, .col-sm-10, .col-md-10, .col-lg-10, .col-xs-11, .col-sm-11, .col-md-11, .col-lg-11, .col-xs-12, .col-sm-12, .col-md-12, .col-lg-12 { + position: relative; + min-height: 1px; + padding-right: 15px; + padding-left: 15px; +} +.col-xs-1, .col-xs-2, .col-xs-3, .col-xs-4, .col-xs-5, .col-xs-6, .col-xs-7, .col-xs-8, .col-xs-9, .col-xs-10, .col-xs-11, .col-xs-12 { + float: left; +} +.col-xs-12 { + width: 100%; +} +.col-xs-11 { + width: 91.66666667%; +} +.col-xs-10 { + width: 83.33333333%; +} +.col-xs-9 { + width: 75%; +} +.col-xs-8 { + width: 66.66666667%; +} +.col-xs-7 { + width: 58.33333333%; +} +.col-xs-6 { + width: 50%; +} +.col-xs-5 { + width: 41.66666667%; +} +.col-xs-4 { + width: 33.33333333%; +} +.col-xs-3 { + width: 25%; +} +.col-xs-2 { + width: 16.66666667%; +} +.col-xs-1 { + width: 8.33333333%; +} +.col-xs-pull-12 { + right: 100%; +} +.col-xs-pull-11 { + right: 91.66666667%; +} +.col-xs-pull-10 { + right: 83.33333333%; +} +.col-xs-pull-9 { + right: 75%; +} +.col-xs-pull-8 { + right: 66.66666667%; +} +.col-xs-pull-7 { + right: 58.33333333%; +} +.col-xs-pull-6 { + right: 50%; +} +.col-xs-pull-5 { + right: 41.66666667%; +} +.col-xs-pull-4 { + right: 33.33333333%; +} +.col-xs-pull-3 { + right: 25%; +} +.col-xs-pull-2 { + right: 16.66666667%; +} +.col-xs-pull-1 { + right: 8.33333333%; +} +.col-xs-pull-0 { + right: 0; +} +.col-xs-push-12 { + left: 100%; +} +.col-xs-push-11 { + left: 91.66666667%; +} +.col-xs-push-10 { + left: 83.33333333%; +} +.col-xs-push-9 { + left: 75%; +} +.col-xs-push-8 { + left: 66.66666667%; +} +.col-xs-push-7 { + left: 58.33333333%; +} +.col-xs-push-6 { + left: 50%; +} +.col-xs-push-5 { + left: 41.66666667%; +} +.col-xs-push-4 { + left: 33.33333333%; +} +.col-xs-push-3 { + left: 25%; +} +.col-xs-push-2 { + left: 16.66666667%; +} +.col-xs-push-1 { + left: 8.33333333%; +} +.col-xs-push-0 { + left: 0; +} +.col-xs-offset-12 { + margin-left: 100%; +} +.col-xs-offset-11 { + margin-left: 91.66666667%; +} +.col-xs-offset-10 { + margin-left: 83.33333333%; +} +.col-xs-offset-9 { + margin-left: 75%; +} +.col-xs-offset-8 { + margin-left: 66.66666667%; +} +.col-xs-offset-7 { + margin-left: 58.33333333%; +} +.col-xs-offset-6 { + margin-left: 50%; +} +.col-xs-offset-5 { + margin-left: 41.66666667%; +} +.col-xs-offset-4 { + margin-left: 33.33333333%; +} +.col-xs-offset-3 { + margin-left: 25%; +} +.col-xs-offset-2 { + margin-left: 16.66666667%; +} +.col-xs-offset-1 { + margin-left: 8.33333333%; +} +.col-xs-offset-0 { + margin-left: 0; +} +@media (min-width: 768px) { + .col-sm-1, .col-sm-2, .col-sm-3, .col-sm-4, .col-sm-5, .col-sm-6, .col-sm-7, .col-sm-8, .col-sm-9, .col-sm-10, .col-sm-11, .col-sm-12 { + float: left; + } + .col-sm-12 { + width: 100%; + } + .col-sm-11 { + width: 91.66666667%; + } + .col-sm-10 { + width: 83.33333333%; + } + .col-sm-9 { + width: 75%; + } + .col-sm-8 { + width: 66.66666667%; + } + .col-sm-7 { + width: 58.33333333%; + } + .col-sm-6 { + width: 50%; + } + .col-sm-5 { + width: 41.66666667%; + } + .col-sm-4 { + width: 33.33333333%; + } + .col-sm-3 { + width: 25%; + } + .col-sm-2 { + width: 16.66666667%; + } + .col-sm-1 { + width: 8.33333333%; + } + .col-sm-pull-12 { + right: 100%; + } + .col-sm-pull-11 { + right: 91.66666667%; + } + .col-sm-pull-10 { + right: 83.33333333%; + } + .col-sm-pull-9 { + right: 75%; + } + .col-sm-pull-8 { + right: 66.66666667%; + } + .col-sm-pull-7 { + right: 58.33333333%; + } + .col-sm-pull-6 { + right: 50%; + } + .col-sm-pull-5 { + right: 41.66666667%; + } + .col-sm-pull-4 { + right: 33.33333333%; + } + .col-sm-pull-3 { + right: 25%; + } + .col-sm-pull-2 { + right: 16.66666667%; + } + .col-sm-pull-1 { + right: 8.33333333%; + } + .col-sm-pull-0 { + right: 0; + } + .col-sm-push-12 { + left: 100%; + } + .col-sm-push-11 { + left: 91.66666667%; + } + .col-sm-push-10 { + left: 83.33333333%; + } + .col-sm-push-9 { + left: 75%; + } + .col-sm-push-8 { + left: 66.66666667%; + } + .col-sm-push-7 { + left: 58.33333333%; + } + .col-sm-push-6 { + left: 50%; + } + .col-sm-push-5 { + left: 41.66666667%; + } + .col-sm-push-4 { + left: 33.33333333%; + } + .col-sm-push-3 { + left: 25%; + } + .col-sm-push-2 { + left: 16.66666667%; + } + .col-sm-push-1 { + left: 8.33333333%; + } + .col-sm-push-0 { + left: 0; + } + .col-sm-offset-12 { + margin-left: 100%; + } + .col-sm-offset-11 { + margin-left: 91.66666667%; + } + .col-sm-offset-10 { + margin-left: 83.33333333%; + } + .col-sm-offset-9 { + margin-left: 75%; + } + .col-sm-offset-8 { + margin-left: 66.66666667%; + } + .col-sm-offset-7 { + margin-left: 58.33333333%; + } + .col-sm-offset-6 { + margin-left: 50%; + } + .col-sm-offset-5 { + margin-left: 41.66666667%; + } + .col-sm-offset-4 { + margin-left: 33.33333333%; + } + .col-sm-offset-3 { + margin-left: 25%; + } + .col-sm-offset-2 { + margin-left: 16.66666667%; + } + .col-sm-offset-1 { + margin-left: 8.33333333%; + } + .col-sm-offset-0 { + margin-left: 0; + } +} +@media (min-width: 992px) { + .col-md-1, .col-md-2, .col-md-3, .col-md-4, .col-md-5, .col-md-6, .col-md-7, .col-md-8, .col-md-9, .col-md-10, .col-md-11, .col-md-12 { + float: left; + } + .col-md-12 { + width: 100%; + } + .col-md-11 { + width: 91.66666667%; + } + .col-md-10 { + width: 83.33333333%; + } + .col-md-9 { + width: 75%; + } + .col-md-8 { + width: 66.66666667%; + } + .col-md-7 { + width: 58.33333333%; + } + .col-md-6 { + width: 50%; + } + .col-md-5 { + width: 41.66666667%; + } + .col-md-4 { + width: 33.33333333%; + } + .col-md-3 { + width: 25%; + } + .col-md-2 { + width: 16.66666667%; + } + .col-md-1 { + width: 8.33333333%; + } + .col-md-pull-12 { + right: 100%; + } + .col-md-pull-11 { + right: 91.66666667%; + } + .col-md-pull-10 { + right: 83.33333333%; + } + .col-md-pull-9 { + right: 75%; + } + .col-md-pull-8 { + right: 66.66666667%; + } + .col-md-pull-7 { + right: 58.33333333%; + } + .col-md-pull-6 { + right: 50%; + } + .col-md-pull-5 { + right: 41.66666667%; + } + .col-md-pull-4 { + right: 33.33333333%; + } + .col-md-pull-3 { + right: 25%; + } + .col-md-pull-2 { + right: 16.66666667%; + } + .col-md-pull-1 { + right: 8.33333333%; + } + .col-md-pull-0 { + right: 0; + } + .col-md-push-12 { + left: 100%; + } + .col-md-push-11 { + left: 91.66666667%; + } + .col-md-push-10 { + left: 83.33333333%; + } + .col-md-push-9 { + left: 75%; + } + .col-md-push-8 { + left: 66.66666667%; + } + .col-md-push-7 { + left: 58.33333333%; + } + .col-md-push-6 { + left: 50%; + } + .col-md-push-5 { + left: 41.66666667%; + } + .col-md-push-4 { + left: 33.33333333%; + } + .col-md-push-3 { + left: 25%; + } + .col-md-push-2 { + left: 16.66666667%; + } + .col-md-push-1 { + left: 8.33333333%; + } + .col-md-push-0 { + left: 0; + } + .col-md-offset-12 { + margin-left: 100%; + } + .col-md-offset-11 { + margin-left: 91.66666667%; + } + .col-md-offset-10 { + margin-left: 83.33333333%; + } + .col-md-offset-9 { + margin-left: 75%; + } + .col-md-offset-8 { + margin-left: 66.66666667%; + } + .col-md-offset-7 { + margin-left: 58.33333333%; + } + .col-md-offset-6 { + margin-left: 50%; + } + .col-md-offset-5 { + margin-left: 41.66666667%; + } + .col-md-offset-4 { + margin-left: 33.33333333%; + } + .col-md-offset-3 { + margin-left: 25%; + } + .col-md-offset-2 { + margin-left: 16.66666667%; + } + .col-md-offset-1 { + margin-left: 8.33333333%; + } + .col-md-offset-0 { + margin-left: 0; + } +} +@media (min-width: 1200px) { + .col-lg-1, .col-lg-2, .col-lg-3, .col-lg-4, .col-lg-5, .col-lg-6, .col-lg-7, .col-lg-8, .col-lg-9, .col-lg-10, .col-lg-11, .col-lg-12 { + float: left; + } + .col-lg-12 { + width: 100%; + } + .col-lg-11 { + width: 91.66666667%; + } + .col-lg-10 { + width: 83.33333333%; + } + .col-lg-9 { + width: 75%; + } + .col-lg-8 { + width: 66.66666667%; + } + .col-lg-7 { + width: 58.33333333%; + } + .col-lg-6 { + width: 50%; + } + .col-lg-5 { + width: 41.66666667%; + } + .col-lg-4 { + width: 33.33333333%; + } + .col-lg-3 { + width: 25%; + } + .col-lg-2 { + width: 16.66666667%; + } + .col-lg-1 { + width: 8.33333333%; + } + .col-lg-pull-12 { + right: 100%; + } + .col-lg-pull-11 { + right: 91.66666667%; + } + .col-lg-pull-10 { + right: 83.33333333%; + } + .col-lg-pull-9 { + right: 75%; + } + .col-lg-pull-8 { + right: 66.66666667%; + } + .col-lg-pull-7 { + right: 58.33333333%; + } + .col-lg-pull-6 { + right: 50%; + } + .col-lg-pull-5 { + right: 41.66666667%; + } + .col-lg-pull-4 { + right: 33.33333333%; + } + .col-lg-pull-3 { + right: 25%; + } + .col-lg-pull-2 { + right: 16.66666667%; + } + .col-lg-pull-1 { + right: 8.33333333%; + } + .col-lg-pull-0 { + right: 0; + } + .col-lg-push-12 { + left: 100%; + } + .col-lg-push-11 { + left: 91.66666667%; + } + .col-lg-push-10 { + left: 83.33333333%; + } + .col-lg-push-9 { + left: 75%; + } + .col-lg-push-8 { + left: 66.66666667%; + } + .col-lg-push-7 { + left: 58.33333333%; + } + .col-lg-push-6 { + left: 50%; + } + .col-lg-push-5 { + left: 41.66666667%; + } + .col-lg-push-4 { + left: 33.33333333%; + } + .col-lg-push-3 { + left: 25%; + } + .col-lg-push-2 { + left: 16.66666667%; + } + .col-lg-push-1 { + left: 8.33333333%; + } + .col-lg-push-0 { + left: 0; + } + .col-lg-offset-12 { + margin-left: 100%; + } + .col-lg-offset-11 { + margin-left: 91.66666667%; + } + .col-lg-offset-10 { + margin-left: 83.33333333%; + } + .col-lg-offset-9 { + margin-left: 75%; + } + .col-lg-offset-8 { + margin-left: 66.66666667%; + } + .col-lg-offset-7 { + margin-left: 58.33333333%; + } + .col-lg-offset-6 { + margin-left: 50%; + } + .col-lg-offset-5 { + margin-left: 41.66666667%; + } + .col-lg-offset-4 { + margin-left: 33.33333333%; + } + .col-lg-offset-3 { + margin-left: 25%; + } + .col-lg-offset-2 { + margin-left: 16.66666667%; + } + .col-lg-offset-1 { + margin-left: 8.33333333%; + } + .col-lg-offset-0 { + margin-left: 0; + } +} +table { + max-width: 100%; + background-color: transparent; +} +th { + text-align: left; +} +.table { + width: 100%; + margin-bottom: 20px; +} +.table > thead > tr > th, +.table > tbody > tr > th, +.table > tfoot > tr > th, +.table > thead > tr > td, +.table > tbody > tr > td, +.table > tfoot > tr > td { + padding: 8px; + line-height: 1.42857143; + vertical-align: top; + border-top: 1px solid #ddd; +} +.table > thead > tr > th { + vertical-align: bottom; + border-bottom: 2px solid #ddd; +} +.table > caption + thead > tr:first-child > th, +.table > colgroup + thead > tr:first-child > th, +.table > thead:first-child > tr:first-child > th, +.table > caption + thead > tr:first-child > td, +.table > colgroup + thead > tr:first-child > td, +.table > thead:first-child > tr:first-child > td { + border-top: 0; +} +.table > tbody + tbody { + border-top: 2px solid #ddd; +} +.table .table { + background-color: #fff; +} +.table-condensed > thead > tr > th, +.table-condensed > tbody > tr > th, +.table-condensed > tfoot > tr > th, +.table-condensed > thead > tr > td, +.table-condensed > tbody > tr > td, +.table-condensed > tfoot > tr > td { + padding: 5px; +} +.table-bordered { + border: 1px solid #ddd; +} +.table-bordered > thead > tr > th, +.table-bordered > tbody > tr > th, +.table-bordered > tfoot > tr > th, +.table-bordered > thead > tr > td, +.table-bordered > tbody > tr > td, +.table-bordered > tfoot > tr > td { + border: 1px solid #ddd; +} +.table-bordered > thead > tr > th, +.table-bordered > thead > tr > td { + border-bottom-width: 2px; +} +.table-striped > tbody > tr:nth-child(odd) > td, +.table-striped > tbody > tr:nth-child(odd) > th { + background-color: #f9f9f9; +} +.table-hover > tbody > tr:hover > td, +.table-hover > tbody > tr:hover > th { + background-color: #f5f5f5; +} +table col[class*="col-"] { + position: static; + display: table-column; + float: none; +} +table td[class*="col-"], +table th[class*="col-"] { + position: static; + display: table-cell; + float: none; +} +.table > thead > tr > td.active, +.table > tbody > tr > td.active, +.table > tfoot > tr > td.active, +.table > thead > tr > th.active, +.table > tbody > tr > th.active, +.table > tfoot > tr > th.active, +.table > thead > tr.active > td, +.table > tbody > tr.active > td, +.table > tfoot > tr.active > td, +.table > thead > tr.active > th, +.table > tbody > tr.active > th, +.table > tfoot > tr.active > th { + background-color: #f5f5f5; +} +.table-hover > tbody > tr > td.active:hover, +.table-hover > tbody > tr > th.active:hover, +.table-hover > tbody > tr.active:hover > td, +.table-hover > tbody > tr.active:hover > th { + background-color: #e8e8e8; +} +.table > thead > tr > td.success, +.table > tbody > tr > td.success, +.table > tfoot > tr > td.success, +.table > thead > tr > th.success, +.table > tbody > tr > th.success, +.table > tfoot > tr > th.success, +.table > thead > tr.success > td, +.table > tbody > tr.success > td, +.table > tfoot > tr.success > td, +.table > thead > tr.success > th, +.table > tbody > tr.success > th, +.table > tfoot > tr.success > th { + background-color: #dff0d8; +} +.table-hover > tbody > tr > td.success:hover, +.table-hover > tbody > tr > th.success:hover, +.table-hover > tbody > tr.success:hover > td, +.table-hover > tbody > tr.success:hover > th { + background-color: #d0e9c6; +} +.table > thead > tr > td.info, +.table > tbody > tr > td.info, +.table > tfoot > tr > td.info, +.table > thead > tr > th.info, +.table > tbody > tr > th.info, +.table > tfoot > tr > th.info, +.table > thead > tr.info > td, +.table > tbody > tr.info > td, +.table > tfoot > tr.info > td, +.table > thead > tr.info > th, +.table > tbody > tr.info > th, +.table > tfoot > tr.info > th { + background-color: #d9edf7; +} +.table-hover > tbody > tr > td.info:hover, +.table-hover > tbody > tr > th.info:hover, +.table-hover > tbody > tr.info:hover > td, +.table-hover > tbody > tr.info:hover > th { + background-color: #c4e3f3; +} +.table > thead > tr > td.warning, +.table > tbody > tr > td.warning, +.table > tfoot > tr > td.warning, +.table > thead > tr > th.warning, +.table > tbody > tr > th.warning, +.table > tfoot > tr > th.warning, +.table > thead > tr.warning > td, +.table > tbody > tr.warning > td, +.table > tfoot > tr.warning > td, +.table > thead > tr.warning > th, +.table > tbody > tr.warning > th, +.table > tfoot > tr.warning > th { + background-color: #fcf8e3; +} +.table-hover > tbody > tr > td.warning:hover, +.table-hover > tbody > tr > th.warning:hover, +.table-hover > tbody > tr.warning:hover > td, +.table-hover > tbody > tr.warning:hover > th { + background-color: #faf2cc; +} +.table > thead > tr > td.danger, +.table > tbody > tr > td.danger, +.table > tfoot > tr > td.danger, +.table > thead > tr > th.danger, +.table > tbody > tr > th.danger, +.table > tfoot > tr > th.danger, +.table > thead > tr.danger > td, +.table > tbody > tr.danger > td, +.table > tfoot > tr.danger > td, +.table > thead > tr.danger > th, +.table > tbody > tr.danger > th, +.table > tfoot > tr.danger > th { + background-color: #f2dede; +} +.table-hover > tbody > tr > td.danger:hover, +.table-hover > tbody > tr > th.danger:hover, +.table-hover > tbody > tr.danger:hover > td, +.table-hover > tbody > tr.danger:hover > th { + background-color: #ebcccc; +} +@media (max-width: 767px) { + .table-responsive { + width: 100%; + margin-bottom: 15px; + overflow-x: scroll; + overflow-y: hidden; + -webkit-overflow-scrolling: touch; + -ms-overflow-style: -ms-autohiding-scrollbar; + border: 1px solid #ddd; + } + .table-responsive > .table { + margin-bottom: 0; + } + .table-responsive > .table > thead > tr > th, + .table-responsive > .table > tbody > tr > th, + .table-responsive > .table > tfoot > tr > th, + .table-responsive > .table > thead > tr > td, + .table-responsive > .table > tbody > tr > td, + .table-responsive > .table > tfoot > tr > td { + white-space: nowrap; + } + .table-responsive > .table-bordered { + border: 0; + } + .table-responsive > .table-bordered > thead > tr > th:first-child, + .table-responsive > .table-bordered > tbody > tr > th:first-child, + .table-responsive > .table-bordered > tfoot > tr > th:first-child, + .table-responsive > .table-bordered > thead > tr > td:first-child, + .table-responsive > .table-bordered > tbody > tr > td:first-child, + .table-responsive > .table-bordered > tfoot > tr > td:first-child { + border-left: 0; + } + .table-responsive > .table-bordered > thead > tr > th:last-child, + .table-responsive > .table-bordered > tbody > tr > th:last-child, + .table-responsive > .table-bordered > tfoot > tr > th:last-child, + .table-responsive > .table-bordered > thead > tr > td:last-child, + .table-responsive > .table-bordered > tbody > tr > td:last-child, + .table-responsive > .table-bordered > tfoot > tr > td:last-child { + border-right: 0; + } + .table-responsive > .table-bordered > tbody > tr:last-child > th, + .table-responsive > .table-bordered > tfoot > tr:last-child > th, + .table-responsive > .table-bordered > tbody > tr:last-child > td, + .table-responsive > .table-bordered > tfoot > tr:last-child > td { + border-bottom: 0; + } +} +fieldset { + min-width: 0; + padding: 0; + margin: 0; + border: 0; +} +legend { + display: block; + width: 100%; + padding: 0; + margin-bottom: 20px; + font-size: 21px; + line-height: inherit; + color: #333; + border: 0; + border-bottom: 1px solid #e5e5e5; +} +label { + display: inline-block; + margin-bottom: 5px; + font-weight: bold; +} +input[type="search"] { + -webkit-box-sizing: border-box; + -moz-box-sizing: border-box; + box-sizing: border-box; +} +input[type="radio"], +input[type="checkbox"] { + margin: 4px 0 0; + margin-top: 1px \9; + /* IE8-9 */ + line-height: normal; +} +input[type="file"] { + display: block; +} +input[type="range"] { + display: block; + width: 100%; +} +select[multiple], +select[size] { + height: auto; +} +input[type="file"]:focus, +input[type="radio"]:focus, +input[type="checkbox"]:focus { + outline: thin dotted; + outline: 5px auto -webkit-focus-ring-color; + outline-offset: -2px; +} +output { + display: block; + padding-top: 7px; + font-size: 14px; + line-height: 1.42857143; + color: #555; +} +.form-control { + display: block; + width: 100%; + height: 34px; + padding: 6px 12px; + font-size: 14px; + line-height: 1.42857143; + color: #555; + background-color: #fff; + background-image: none; + border: 1px solid #ccc; + border-radius: 4px; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); + -webkit-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s; + transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s; +} +.form-control:focus { + border-color: #66afe9; + outline: 0; + -webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, .6); + box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, .6); +} +.form-control::-moz-placeholder { + color: #999; + opacity: 1; +} +.form-control:-ms-input-placeholder { + color: #999; +} +.form-control::-webkit-input-placeholder { + color: #999; +} +.form-control[disabled], +.form-control[readonly], +fieldset[disabled] .form-control { + cursor: not-allowed; + background-color: #eee; + opacity: 1; +} +textarea.form-control { + height: auto; +} +input[type="search"] { + -webkit-appearance: none; +} +input[type="date"] { + line-height: 34px; +} +.form-group { + margin-bottom: 15px; +} +.radio, +.checkbox { + display: block; + min-height: 20px; + padding-left: 20px; + margin-top: 10px; + margin-bottom: 10px; +} +.radio label, +.checkbox label { + display: inline; + font-weight: normal; + cursor: pointer; +} +.radio input[type="radio"], +.radio-inline input[type="radio"], +.checkbox input[type="checkbox"], +.checkbox-inline input[type="checkbox"] { + float: left; + margin-left: -20px; +} +.radio + .radio, +.checkbox + .checkbox { + margin-top: -5px; +} +.radio-inline, +.checkbox-inline { + display: inline-block; + padding-left: 20px; + margin-bottom: 0; + font-weight: normal; + vertical-align: middle; + cursor: pointer; +} +.radio-inline + .radio-inline, +.checkbox-inline + .checkbox-inline { + margin-top: 0; + margin-left: 10px; +} +input[type="radio"][disabled], +input[type="checkbox"][disabled], +.radio[disabled], +.radio-inline[disabled], +.checkbox[disabled], +.checkbox-inline[disabled], +fieldset[disabled] input[type="radio"], +fieldset[disabled] input[type="checkbox"], +fieldset[disabled] .radio, +fieldset[disabled] .radio-inline, +fieldset[disabled] .checkbox, +fieldset[disabled] .checkbox-inline { + cursor: not-allowed; +} +.input-sm { + height: 30px; + padding: 5px 10px; + font-size: 12px; + line-height: 1.5; + border-radius: 3px; +} +select.input-sm { + height: 30px; + line-height: 30px; +} +textarea.input-sm, +select[multiple].input-sm { + height: auto; +} +.input-lg { + height: 46px; + padding: 10px 16px; + font-size: 18px; + line-height: 1.33; + border-radius: 6px; +} +select.input-lg { + height: 46px; + line-height: 46px; +} +textarea.input-lg, +select[multiple].input-lg { + height: auto; +} +.has-feedback { + position: relative; +} +.has-feedback .form-control { + padding-right: 42.5px; +} +.has-feedback .form-control-feedback { + position: absolute; + top: 25px; + right: 0; + display: block; + width: 34px; + height: 34px; + line-height: 34px; + text-align: center; +} +.has-success .help-block, +.has-success .control-label, +.has-success .radio, +.has-success .checkbox, +.has-success .radio-inline, +.has-success .checkbox-inline { + color: #3c763d; +} +.has-success .form-control { + border-color: #3c763d; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); +} +.has-success .form-control:focus { + border-color: #2b542c; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #67b168; + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #67b168; +} +.has-success .input-group-addon { + color: #3c763d; + background-color: #dff0d8; + border-color: #3c763d; +} +.has-success .form-control-feedback { + color: #3c763d; +} +.has-warning .help-block, +.has-warning .control-label, +.has-warning .radio, +.has-warning .checkbox, +.has-warning .radio-inline, +.has-warning .checkbox-inline { + color: #8a6d3b; +} +.has-warning .form-control { + border-color: #8a6d3b; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); +} +.has-warning .form-control:focus { + border-color: #66512c; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #c0a16b; + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #c0a16b; +} +.has-warning .input-group-addon { + color: #8a6d3b; + background-color: #fcf8e3; + border-color: #8a6d3b; +} +.has-warning .form-control-feedback { + color: #8a6d3b; +} +.has-error .help-block, +.has-error .control-label, +.has-error .radio, +.has-error .checkbox, +.has-error .radio-inline, +.has-error .checkbox-inline { + color: #a94442; +} +.has-error .form-control { + border-color: #a94442; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); +} +.has-error .form-control:focus { + border-color: #843534; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #ce8483; + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #ce8483; +} +.has-error .input-group-addon { + color: #a94442; + background-color: #f2dede; + border-color: #a94442; +} +.has-error .form-control-feedback { + color: #a94442; +} +.form-control-static { + margin-bottom: 0; +} +.help-block { + display: block; + margin-top: 5px; + margin-bottom: 10px; + color: #737373; +} +@media (min-width: 768px) { + .form-inline .form-group { + display: inline-block; + margin-bottom: 0; + vertical-align: middle; + } + .form-inline .form-control { + display: inline-block; + width: auto; + vertical-align: middle; + } + .form-inline .input-group > .form-control { + width: 100%; + } + .form-inline .control-label { + margin-bottom: 0; + vertical-align: middle; + } + .form-inline .radio, + .form-inline .checkbox { + display: inline-block; + padding-left: 0; + margin-top: 0; + margin-bottom: 0; + vertical-align: middle; + } + .form-inline .radio input[type="radio"], + .form-inline .checkbox input[type="checkbox"] { + float: none; + margin-left: 0; + } + .form-inline .has-feedback .form-control-feedback { + top: 0; + } +} +.form-horizontal .control-label, +.form-horizontal .radio, +.form-horizontal .checkbox, +.form-horizontal .radio-inline, +.form-horizontal .checkbox-inline { + padding-top: 7px; + margin-top: 0; + margin-bottom: 0; +} +.form-horizontal .radio, +.form-horizontal .checkbox { + min-height: 27px; +} +.form-horizontal .form-group { + margin-right: -15px; + margin-left: -15px; +} +.form-horizontal .form-control-static { + padding-top: 7px; +} +@media (min-width: 768px) { + .form-horizontal .control-label { + text-align: right; + } +} +.form-horizontal .has-feedback .form-control-feedback { + top: 0; + right: 15px; +} +.btn { + display: inline-block; + padding: 6px 12px; + margin-bottom: 0; + font-size: 14px; + font-weight: normal; + line-height: 1.42857143; + text-align: center; + white-space: nowrap; + vertical-align: middle; + cursor: pointer; + -webkit-user-select: none; + -moz-user-select: none; + -ms-user-select: none; + user-select: none; + background-image: none; + border: 1px solid transparent; + border-radius: 4px; +} +.btn:focus, +.btn:active:focus, +.btn.active:focus { + outline: thin dotted; + outline: 5px auto -webkit-focus-ring-color; + outline-offset: -2px; +} +.btn:hover, +.btn:focus { + color: #333; + text-decoration: none; +} +.btn:active, +.btn.active { + background-image: none; + outline: 0; + -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125); + box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125); +} +.btn.disabled, +.btn[disabled], +fieldset[disabled] .btn { + pointer-events: none; + cursor: not-allowed; + filter: alpha(opacity=65); + -webkit-box-shadow: none; + box-shadow: none; + opacity: .65; +} +.btn-default { + color: #333; + background-color: #fff; + border-color: #ccc; +} +.btn-default:hover, +.btn-default:focus, +.btn-default:active, +.btn-default.active, +.open .dropdown-toggle.btn-default { + color: #333; + background-color: #ebebeb; + border-color: #adadad; +} +.btn-default:active, +.btn-default.active, +.open .dropdown-toggle.btn-default { + background-image: none; +} +.btn-default.disabled, +.btn-default[disabled], +fieldset[disabled] .btn-default, +.btn-default.disabled:hover, +.btn-default[disabled]:hover, +fieldset[disabled] .btn-default:hover, +.btn-default.disabled:focus, +.btn-default[disabled]:focus, +fieldset[disabled] .btn-default:focus, +.btn-default.disabled:active, +.btn-default[disabled]:active, +fieldset[disabled] .btn-default:active, +.btn-default.disabled.active, +.btn-default[disabled].active, +fieldset[disabled] .btn-default.active { + background-color: #fff; + border-color: #ccc; +} +.btn-default .badge { + color: #fff; + background-color: #333; +} +.btn-primary { + color: #fff; + background-color: #428bca; + border-color: #357ebd; +} +.btn-primary:hover, +.btn-primary:focus, +.btn-primary:active, +.btn-primary.active, +.open .dropdown-toggle.btn-primary { + color: #fff; + background-color: #3276b1; + border-color: #285e8e; +} +.btn-primary:active, +.btn-primary.active, +.open .dropdown-toggle.btn-primary { + background-image: none; +} +.btn-primary.disabled, +.btn-primary[disabled], +fieldset[disabled] .btn-primary, +.btn-primary.disabled:hover, +.btn-primary[disabled]:hover, +fieldset[disabled] .btn-primary:hover, +.btn-primary.disabled:focus, +.btn-primary[disabled]:focus, +fieldset[disabled] .btn-primary:focus, +.btn-primary.disabled:active, +.btn-primary[disabled]:active, +fieldset[disabled] .btn-primary:active, +.btn-primary.disabled.active, +.btn-primary[disabled].active, +fieldset[disabled] .btn-primary.active { + background-color: #428bca; + border-color: #357ebd; +} +.btn-primary .badge { + color: #428bca; + background-color: #fff; +} +.btn-success { + color: #fff; + background-color: #5cb85c; + border-color: #4cae4c; +} +.btn-success:hover, +.btn-success:focus, +.btn-success:active, +.btn-success.active, +.open .dropdown-toggle.btn-success { + color: #fff; + background-color: #47a447; + border-color: #398439; +} +.btn-success:active, +.btn-success.active, +.open .dropdown-toggle.btn-success { + background-image: none; +} +.btn-success.disabled, +.btn-success[disabled], +fieldset[disabled] .btn-success, +.btn-success.disabled:hover, +.btn-success[disabled]:hover, +fieldset[disabled] .btn-success:hover, +.btn-success.disabled:focus, +.btn-success[disabled]:focus, +fieldset[disabled] .btn-success:focus, +.btn-success.disabled:active, +.btn-success[disabled]:active, +fieldset[disabled] .btn-success:active, +.btn-success.disabled.active, +.btn-success[disabled].active, +fieldset[disabled] .btn-success.active { + background-color: #5cb85c; + border-color: #4cae4c; +} +.btn-success .badge { + color: #5cb85c; + background-color: #fff; +} +.btn-info { + color: #fff; + background-color: #5bc0de; + border-color: #46b8da; +} +.btn-info:hover, +.btn-info:focus, +.btn-info:active, +.btn-info.active, +.open .dropdown-toggle.btn-info { + color: #fff; + background-color: #39b3d7; + border-color: #269abc; +} +.btn-info:active, +.btn-info.active, +.open .dropdown-toggle.btn-info { + background-image: none; +} +.btn-info.disabled, +.btn-info[disabled], +fieldset[disabled] .btn-info, +.btn-info.disabled:hover, +.btn-info[disabled]:hover, +fieldset[disabled] .btn-info:hover, +.btn-info.disabled:focus, +.btn-info[disabled]:focus, +fieldset[disabled] .btn-info:focus, +.btn-info.disabled:active, +.btn-info[disabled]:active, +fieldset[disabled] .btn-info:active, +.btn-info.disabled.active, +.btn-info[disabled].active, +fieldset[disabled] .btn-info.active { + background-color: #5bc0de; + border-color: #46b8da; +} +.btn-info .badge { + color: #5bc0de; + background-color: #fff; +} +.btn-warning { + color: #fff; + background-color: #f0ad4e; + border-color: #eea236; +} +.btn-warning:hover, +.btn-warning:focus, +.btn-warning:active, +.btn-warning.active, +.open .dropdown-toggle.btn-warning { + color: #fff; + background-color: #ed9c28; + border-color: #d58512; +} +.btn-warning:active, +.btn-warning.active, +.open .dropdown-toggle.btn-warning { + background-image: none; +} +.btn-warning.disabled, +.btn-warning[disabled], +fieldset[disabled] .btn-warning, +.btn-warning.disabled:hover, +.btn-warning[disabled]:hover, +fieldset[disabled] .btn-warning:hover, +.btn-warning.disabled:focus, +.btn-warning[disabled]:focus, +fieldset[disabled] .btn-warning:focus, +.btn-warning.disabled:active, +.btn-warning[disabled]:active, +fieldset[disabled] .btn-warning:active, +.btn-warning.disabled.active, +.btn-warning[disabled].active, +fieldset[disabled] .btn-warning.active { + background-color: #f0ad4e; + border-color: #eea236; +} +.btn-warning .badge { + color: #f0ad4e; + background-color: #fff; +} +.btn-danger { + color: #fff; + background-color: #d9534f; + border-color: #d43f3a; +} +.btn-danger:hover, +.btn-danger:focus, +.btn-danger:active, +.btn-danger.active, +.open .dropdown-toggle.btn-danger { + color: #fff; + background-color: #d2322d; + border-color: #ac2925; +} +.btn-danger:active, +.btn-danger.active, +.open .dropdown-toggle.btn-danger { + background-image: none; +} +.btn-danger.disabled, +.btn-danger[disabled], +fieldset[disabled] .btn-danger, +.btn-danger.disabled:hover, +.btn-danger[disabled]:hover, +fieldset[disabled] .btn-danger:hover, +.btn-danger.disabled:focus, +.btn-danger[disabled]:focus, +fieldset[disabled] .btn-danger:focus, +.btn-danger.disabled:active, +.btn-danger[disabled]:active, +fieldset[disabled] .btn-danger:active, +.btn-danger.disabled.active, +.btn-danger[disabled].active, +fieldset[disabled] .btn-danger.active { + background-color: #d9534f; + border-color: #d43f3a; +} +.btn-danger .badge { + color: #d9534f; + background-color: #fff; +} +.btn-link { + font-weight: normal; + color: #428bca; + cursor: pointer; + border-radius: 0; +} +.btn-link, +.btn-link:active, +.btn-link[disabled], +fieldset[disabled] .btn-link { + background-color: transparent; + -webkit-box-shadow: none; + box-shadow: none; +} +.btn-link, +.btn-link:hover, +.btn-link:focus, +.btn-link:active { + border-color: transparent; +} +.btn-link:hover, +.btn-link:focus { + color: #2a6496; + text-decoration: underline; + background-color: transparent; +} +.btn-link[disabled]:hover, +fieldset[disabled] .btn-link:hover, +.btn-link[disabled]:focus, +fieldset[disabled] .btn-link:focus { + color: #999; + text-decoration: none; +} +.btn-lg, +.btn-group-lg > .btn { + padding: 10px 16px; + font-size: 18px; + line-height: 1.33; + border-radius: 6px; +} +.btn-sm, +.btn-group-sm > .btn { + padding: 5px 10px; + font-size: 12px; + line-height: 1.5; + border-radius: 3px; +} +.btn-xs, +.btn-group-xs > .btn { + padding: 1px 5px; + font-size: 12px; + line-height: 1.5; + border-radius: 3px; +} +.btn-block { + display: block; + width: 100%; + padding-right: 0; + padding-left: 0; +} +.btn-block + .btn-block { + margin-top: 5px; +} +input[type="submit"].btn-block, +input[type="reset"].btn-block, +input[type="button"].btn-block { + width: 100%; +} +.fade { + opacity: 0; + -webkit-transition: opacity .15s linear; + transition: opacity .15s linear; +} +.fade.in { + opacity: 1; +} +.collapse { + display: none; +} +.collapse.in { + display: block; +} +.collapsing { + position: relative; + height: 0; + overflow: hidden; + -webkit-transition: height .35s ease; + transition: height .35s ease; +} +@font-face { + font-family: 'Glyphicons Halflings'; + + src: url('../fonts/glyphicons-halflings-regular.eot'); + src: url('../fonts/glyphicons-halflings-regular.eot?#iefix') format('embedded-opentype'), url('../fonts/glyphicons-halflings-regular.woff') format('woff'), url('../fonts/glyphicons-halflings-regular.ttf') format('truetype'), url('../fonts/glyphicons-halflings-regular.svg#glyphicons_halflingsregular') format('svg'); +} +.glyphicon { + position: relative; + top: 1px; + display: inline-block; + font-family: 'Glyphicons Halflings'; + font-style: normal; + font-weight: normal; + line-height: 1; + + -webkit-font-smoothing: antialiased; + -moz-osx-font-smoothing: grayscale; +} +.glyphicon-asterisk:before { + content: "\2a"; +} +.glyphicon-plus:before { + content: "\2b"; +} +.glyphicon-euro:before { + content: "\20ac"; +} +.glyphicon-minus:before { + content: "\2212"; +} +.glyphicon-cloud:before { + content: "\2601"; +} +.glyphicon-envelope:before { + content: "\2709"; +} +.glyphicon-pencil:before { + content: "\270f"; +} +.glyphicon-glass:before { + content: "\e001"; +} +.glyphicon-music:before { + content: "\e002"; +} +.glyphicon-search:before { + content: "\e003"; +} +.glyphicon-heart:before { + content: "\e005"; +} +.glyphicon-star:before { + content: "\e006"; +} +.glyphicon-star-empty:before { + content: "\e007"; +} +.glyphicon-user:before { + content: "\e008"; +} +.glyphicon-film:before { + content: "\e009"; +} +.glyphicon-th-large:before { + content: "\e010"; +} +.glyphicon-th:before { + content: "\e011"; +} +.glyphicon-th-list:before { + content: "\e012"; +} +.glyphicon-ok:before { + content: "\e013"; +} +.glyphicon-remove:before { + content: "\e014"; +} +.glyphicon-zoom-in:before { + content: "\e015"; +} +.glyphicon-zoom-out:before { + content: "\e016"; +} +.glyphicon-off:before { + content: "\e017"; +} +.glyphicon-signal:before { + content: "\e018"; +} +.glyphicon-cog:before { + content: "\e019"; +} +.glyphicon-trash:before { + content: "\e020"; +} +.glyphicon-home:before { + content: "\e021"; +} +.glyphicon-file:before { + content: "\e022"; +} +.glyphicon-time:before { + content: "\e023"; +} +.glyphicon-road:before { + content: "\e024"; +} +.glyphicon-download-alt:before { + content: "\e025"; +} +.glyphicon-download:before { + content: "\e026"; +} +.glyphicon-upload:before { + content: "\e027"; +} +.glyphicon-inbox:before { + content: "\e028"; +} +.glyphicon-play-circle:before { + content: "\e029"; +} +.glyphicon-repeat:before { + content: "\e030"; +} +.glyphicon-refresh:before { + content: "\e031"; +} +.glyphicon-list-alt:before { + content: "\e032"; +} +.glyphicon-lock:before { + content: "\e033"; +} +.glyphicon-flag:before { + content: "\e034"; +} +.glyphicon-headphones:before { + content: "\e035"; +} +.glyphicon-volume-off:before { + content: "\e036"; +} +.glyphicon-volume-down:before { + content: "\e037"; +} +.glyphicon-volume-up:before { + content: "\e038"; +} +.glyphicon-qrcode:before { + content: "\e039"; +} +.glyphicon-barcode:before { + content: "\e040"; +} +.glyphicon-tag:before { + content: "\e041"; +} +.glyphicon-tags:before { + content: "\e042"; +} +.glyphicon-book:before { + content: "\e043"; +} +.glyphicon-bookmark:before { + content: "\e044"; +} +.glyphicon-print:before { + content: "\e045"; +} +.glyphicon-camera:before { + content: "\e046"; +} +.glyphicon-font:before { + content: "\e047"; +} +.glyphicon-bold:before { + content: "\e048"; +} +.glyphicon-italic:before { + content: "\e049"; +} +.glyphicon-text-height:before { + content: "\e050"; +} +.glyphicon-text-width:before { + content: "\e051"; +} +.glyphicon-align-left:before { + content: "\e052"; +} +.glyphicon-align-center:before { + content: "\e053"; +} +.glyphicon-align-right:before { + content: "\e054"; +} +.glyphicon-align-justify:before { + content: "\e055"; +} +.glyphicon-list:before { + content: "\e056"; +} +.glyphicon-indent-left:before { + content: "\e057"; +} +.glyphicon-indent-right:before { + content: "\e058"; +} +.glyphicon-facetime-video:before { + content: "\e059"; +} +.glyphicon-picture:before { + content: "\e060"; +} +.glyphicon-map-marker:before { + content: "\e062"; +} +.glyphicon-adjust:before { + content: "\e063"; +} +.glyphicon-tint:before { + content: "\e064"; +} +.glyphicon-edit:before { + content: "\e065"; +} +.glyphicon-share:before { + content: "\e066"; +} +.glyphicon-check:before { + content: "\e067"; +} +.glyphicon-move:before { + content: "\e068"; +} +.glyphicon-step-backward:before { + content: "\e069"; +} +.glyphicon-fast-backward:before { + content: "\e070"; +} +.glyphicon-backward:before { + content: "\e071"; +} +.glyphicon-play:before { + content: "\e072"; +} +.glyphicon-pause:before { + content: "\e073"; +} +.glyphicon-stop:before { + content: "\e074"; +} +.glyphicon-forward:before { + content: "\e075"; +} +.glyphicon-fast-forward:before { + content: "\e076"; +} +.glyphicon-step-forward:before { + content: "\e077"; +} +.glyphicon-eject:before { + content: "\e078"; +} +.glyphicon-chevron-left:before { + content: "\e079"; +} +.glyphicon-chevron-right:before { + content: "\e080"; +} +.glyphicon-plus-sign:before { + content: "\e081"; +} +.glyphicon-minus-sign:before { + content: "\e082"; +} +.glyphicon-remove-sign:before { + content: "\e083"; +} +.glyphicon-ok-sign:before { + content: "\e084"; +} +.glyphicon-question-sign:before { + content: "\e085"; +} +.glyphicon-info-sign:before { + content: "\e086"; +} +.glyphicon-screenshot:before { + content: "\e087"; +} +.glyphicon-remove-circle:before { + content: "\e088"; +} +.glyphicon-ok-circle:before { + content: "\e089"; +} +.glyphicon-ban-circle:before { + content: "\e090"; +} +.glyphicon-arrow-left:before { + content: "\e091"; +} +.glyphicon-arrow-right:before { + content: "\e092"; +} +.glyphicon-arrow-up:before { + content: "\e093"; +} +.glyphicon-arrow-down:before { + content: "\e094"; +} +.glyphicon-share-alt:before { + content: "\e095"; +} +.glyphicon-resize-full:before { + content: "\e096"; +} +.glyphicon-resize-small:before { + content: "\e097"; +} +.glyphicon-exclamation-sign:before { + content: "\e101"; +} +.glyphicon-gift:before { + content: "\e102"; +} +.glyphicon-leaf:before { + content: "\e103"; +} +.glyphicon-fire:before { + content: "\e104"; +} +.glyphicon-eye-open:before { + content: "\e105"; +} +.glyphicon-eye-close:before { + content: "\e106"; +} +.glyphicon-warning-sign:before { + content: "\e107"; +} +.glyphicon-plane:before { + content: "\e108"; +} +.glyphicon-calendar:before { + content: "\e109"; +} +.glyphicon-random:before { + content: "\e110"; +} +.glyphicon-comment:before { + content: "\e111"; +} +.glyphicon-magnet:before { + content: "\e112"; +} +.glyphicon-chevron-up:before { + content: "\e113"; +} +.glyphicon-chevron-down:before { + content: "\e114"; +} +.glyphicon-retweet:before { + content: "\e115"; +} +.glyphicon-shopping-cart:before { + content: "\e116"; +} +.glyphicon-folder-close:before { + content: "\e117"; +} +.glyphicon-folder-open:before { + content: "\e118"; +} +.glyphicon-resize-vertical:before { + content: "\e119"; +} +.glyphicon-resize-horizontal:before { + content: "\e120"; +} +.glyphicon-hdd:before { + content: "\e121"; +} +.glyphicon-bullhorn:before { + content: "\e122"; +} +.glyphicon-bell:before { + content: "\e123"; +} +.glyphicon-certificate:before { + content: "\e124"; +} +.glyphicon-thumbs-up:before { + content: "\e125"; +} +.glyphicon-thumbs-down:before { + content: "\e126"; +} +.glyphicon-hand-right:before { + content: "\e127"; +} +.glyphicon-hand-left:before { + content: "\e128"; +} +.glyphicon-hand-up:before { + content: "\e129"; +} +.glyphicon-hand-down:before { + content: "\e130"; +} +.glyphicon-circle-arrow-right:before { + content: "\e131"; +} +.glyphicon-circle-arrow-left:before { + content: "\e132"; +} +.glyphicon-circle-arrow-up:before { + content: "\e133"; +} +.glyphicon-circle-arrow-down:before { + content: "\e134"; +} +.glyphicon-globe:before { + content: "\e135"; +} +.glyphicon-wrench:before { + content: "\e136"; +} +.glyphicon-tasks:before { + content: "\e137"; +} +.glyphicon-filter:before { + content: "\e138"; +} +.glyphicon-briefcase:before { + content: "\e139"; +} +.glyphicon-fullscreen:before { + content: "\e140"; +} +.glyphicon-dashboard:before { + content: "\e141"; +} +.glyphicon-paperclip:before { + content: "\e142"; +} +.glyphicon-heart-empty:before { + content: "\e143"; +} +.glyphicon-link:before { + content: "\e144"; +} +.glyphicon-phone:before { + content: "\e145"; +} +.glyphicon-pushpin:before { + content: "\e146"; +} +.glyphicon-usd:before { + content: "\e148"; +} +.glyphicon-gbp:before { + content: "\e149"; +} +.glyphicon-sort:before { + content: "\e150"; +} +.glyphicon-sort-by-alphabet:before { + content: "\e151"; +} +.glyphicon-sort-by-alphabet-alt:before { + content: "\e152"; +} +.glyphicon-sort-by-order:before { + content: "\e153"; +} +.glyphicon-sort-by-order-alt:before { + content: "\e154"; +} +.glyphicon-sort-by-attributes:before { + content: "\e155"; +} +.glyphicon-sort-by-attributes-alt:before { + content: "\e156"; +} +.glyphicon-unchecked:before { + content: "\e157"; +} +.glyphicon-expand:before { + content: "\e158"; +} +.glyphicon-collapse-down:before { + content: "\e159"; +} +.glyphicon-collapse-up:before { + content: "\e160"; +} +.glyphicon-log-in:before { + content: "\e161"; +} +.glyphicon-flash:before { + content: "\e162"; +} +.glyphicon-log-out:before { + content: "\e163"; +} +.glyphicon-new-window:before { + content: "\e164"; +} +.glyphicon-record:before { + content: "\e165"; +} +.glyphicon-save:before { + content: "\e166"; +} +.glyphicon-open:before { + content: "\e167"; +} +.glyphicon-saved:before { + content: "\e168"; +} +.glyphicon-import:before { + content: "\e169"; +} +.glyphicon-export:before { + content: "\e170"; +} +.glyphicon-send:before { + content: "\e171"; +} +.glyphicon-floppy-disk:before { + content: "\e172"; +} +.glyphicon-floppy-saved:before { + content: "\e173"; +} +.glyphicon-floppy-remove:before { + content: "\e174"; +} +.glyphicon-floppy-save:before { + content: "\e175"; +} +.glyphicon-floppy-open:before { + content: "\e176"; +} +.glyphicon-credit-card:before { + content: "\e177"; +} +.glyphicon-transfer:before { + content: "\e178"; +} +.glyphicon-cutlery:before { + content: "\e179"; +} +.glyphicon-header:before { + content: "\e180"; +} +.glyphicon-compressed:before { + content: "\e181"; +} +.glyphicon-earphone:before { + content: "\e182"; +} +.glyphicon-phone-alt:before { + content: "\e183"; +} +.glyphicon-tower:before { + content: "\e184"; +} +.glyphicon-stats:before { + content: "\e185"; +} +.glyphicon-sd-video:before { + content: "\e186"; +} +.glyphicon-hd-video:before { + content: "\e187"; +} +.glyphicon-subtitles:before { + content: "\e188"; +} +.glyphicon-sound-stereo:before { + content: "\e189"; +} +.glyphicon-sound-dolby:before { + content: "\e190"; +} +.glyphicon-sound-5-1:before { + content: "\e191"; +} +.glyphicon-sound-6-1:before { + content: "\e192"; +} +.glyphicon-sound-7-1:before { + content: "\e193"; +} +.glyphicon-copyright-mark:before { + content: "\e194"; +} +.glyphicon-registration-mark:before { + content: "\e195"; +} +.glyphicon-cloud-download:before { + content: "\e197"; +} +.glyphicon-cloud-upload:before { + content: "\e198"; +} +.glyphicon-tree-conifer:before { + content: "\e199"; +} +.glyphicon-tree-deciduous:before { + content: "\e200"; +} +.caret { + display: inline-block; + width: 0; + height: 0; + margin-left: 2px; + vertical-align: middle; + border-top: 4px solid; + border-right: 4px solid transparent; + border-left: 4px solid transparent; +} +.dropdown { + position: relative; +} +.dropdown-toggle:focus { + outline: 0; +} +.dropdown-menu { + position: absolute; + top: 100%; + left: 0; + z-index: 1000; + display: none; + float: left; + min-width: 160px; + padding: 5px 0; + margin: 2px 0 0; + font-size: 14px; + list-style: none; + background-color: #fff; + background-clip: padding-box; + border: 1px solid #ccc; + border: 1px solid rgba(0, 0, 0, .15); + border-radius: 4px; + -webkit-box-shadow: 0 6px 12px rgba(0, 0, 0, .175); + box-shadow: 0 6px 12px rgba(0, 0, 0, .175); +} +.dropdown-menu.pull-right { + right: 0; + left: auto; +} +.dropdown-menu .divider { + height: 1px; + margin: 9px 0; + overflow: hidden; + background-color: #e5e5e5; +} +.dropdown-menu > li > a { + display: block; + padding: 3px 20px; + clear: both; + font-weight: normal; + line-height: 1.42857143; + color: #333; + white-space: nowrap; +} +.dropdown-menu > li > a:hover, +.dropdown-menu > li > a:focus { + color: #262626; + text-decoration: none; + background-color: #f5f5f5; +} +.dropdown-menu > .active > a, +.dropdown-menu > .active > a:hover, +.dropdown-menu > .active > a:focus { + color: #fff; + text-decoration: none; + background-color: #428bca; + outline: 0; +} +.dropdown-menu > .disabled > a, +.dropdown-menu > .disabled > a:hover, +.dropdown-menu > .disabled > a:focus { + color: #999; +} +.dropdown-menu > .disabled > a:hover, +.dropdown-menu > .disabled > a:focus { + text-decoration: none; + cursor: not-allowed; + background-color: transparent; + background-image: none; + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); +} +.open > .dropdown-menu { + display: block; +} +.open > a { + outline: 0; +} +.dropdown-menu-right { + right: 0; + left: auto; +} +.dropdown-menu-left { + right: auto; + left: 0; +} +.dropdown-header { + display: block; + padding: 3px 20px; + font-size: 12px; + line-height: 1.42857143; + color: #999; +} +.dropdown-backdrop { + position: fixed; + top: 0; + right: 0; + bottom: 0; + left: 0; + z-index: 990; +} +.pull-right > .dropdown-menu { + right: 0; + left: auto; +} +.dropup .caret, +.navbar-fixed-bottom .dropdown .caret { + content: ""; + border-top: 0; + border-bottom: 4px solid; +} +.dropup .dropdown-menu, +.navbar-fixed-bottom .dropdown .dropdown-menu { + top: auto; + bottom: 100%; + margin-bottom: 1px; +} +@media (min-width: 768px) { + .navbar-right .dropdown-menu { + right: 0; + left: auto; + } + .navbar-right .dropdown-menu-left { + right: auto; + left: 0; + } +} +.btn-group, +.btn-group-vertical { + position: relative; + display: inline-block; + vertical-align: middle; +} +.btn-group > .btn, +.btn-group-vertical > .btn { + position: relative; + float: left; +} +.btn-group > .btn:hover, +.btn-group-vertical > .btn:hover, +.btn-group > .btn:focus, +.btn-group-vertical > .btn:focus, +.btn-group > .btn:active, +.btn-group-vertical > .btn:active, +.btn-group > .btn.active, +.btn-group-vertical > .btn.active { + z-index: 2; +} +.btn-group > .btn:focus, +.btn-group-vertical > .btn:focus { + outline: none; +} +.btn-group .btn + .btn, +.btn-group .btn + .btn-group, +.btn-group .btn-group + .btn, +.btn-group .btn-group + .btn-group { + margin-left: -1px; +} +.btn-toolbar { + margin-left: -5px; +} +.btn-toolbar .btn-group, +.btn-toolbar .input-group { + float: left; +} +.btn-toolbar > .btn, +.btn-toolbar > .btn-group, +.btn-toolbar > .input-group { + margin-left: 5px; +} +.btn-group > .btn:not(:first-child):not(:last-child):not(.dropdown-toggle) { + border-radius: 0; +} +.btn-group > .btn:first-child { + margin-left: 0; +} +.btn-group > .btn:first-child:not(:last-child):not(.dropdown-toggle) { + border-top-right-radius: 0; + border-bottom-right-radius: 0; +} +.btn-group > .btn:last-child:not(:first-child), +.btn-group > .dropdown-toggle:not(:first-child) { + border-top-left-radius: 0; + border-bottom-left-radius: 0; +} +.btn-group > .btn-group { + float: left; +} +.btn-group > .btn-group:not(:first-child):not(:last-child) > .btn { + border-radius: 0; +} +.btn-group > .btn-group:first-child > .btn:last-child, +.btn-group > .btn-group:first-child > .dropdown-toggle { + border-top-right-radius: 0; + border-bottom-right-radius: 0; +} +.btn-group > .btn-group:last-child > .btn:first-child { + border-top-left-radius: 0; + border-bottom-left-radius: 0; +} +.btn-group .dropdown-toggle:active, +.btn-group.open .dropdown-toggle { + outline: 0; +} +.btn-group > .btn + .dropdown-toggle { + padding-right: 8px; + padding-left: 8px; +} +.btn-group > .btn-lg + .dropdown-toggle { + padding-right: 12px; + padding-left: 12px; +} +.btn-group.open .dropdown-toggle { + -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125); + box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125); +} +.btn-group.open .dropdown-toggle.btn-link { + -webkit-box-shadow: none; + box-shadow: none; +} +.btn .caret { + margin-left: 0; +} +.btn-lg .caret { + border-width: 5px 5px 0; + border-bottom-width: 0; +} +.dropup .btn-lg .caret { + border-width: 0 5px 5px; +} +.btn-group-vertical > .btn, +.btn-group-vertical > .btn-group, +.btn-group-vertical > .btn-group > .btn { + display: block; + float: none; + width: 100%; + max-width: 100%; +} +.btn-group-vertical > .btn-group > .btn { + float: none; +} +.btn-group-vertical > .btn + .btn, +.btn-group-vertical > .btn + .btn-group, +.btn-group-vertical > .btn-group + .btn, +.btn-group-vertical > .btn-group + .btn-group { + margin-top: -1px; + margin-left: 0; +} +.btn-group-vertical > .btn:not(:first-child):not(:last-child) { + border-radius: 0; +} +.btn-group-vertical > .btn:first-child:not(:last-child) { + border-top-right-radius: 4px; + border-bottom-right-radius: 0; + border-bottom-left-radius: 0; +} +.btn-group-vertical > .btn:last-child:not(:first-child) { + border-top-left-radius: 0; + border-top-right-radius: 0; + border-bottom-left-radius: 4px; +} +.btn-group-vertical > .btn-group:not(:first-child):not(:last-child) > .btn { + border-radius: 0; +} +.btn-group-vertical > .btn-group:first-child:not(:last-child) > .btn:last-child, +.btn-group-vertical > .btn-group:first-child:not(:last-child) > .dropdown-toggle { + border-bottom-right-radius: 0; + border-bottom-left-radius: 0; +} +.btn-group-vertical > .btn-group:last-child:not(:first-child) > .btn:first-child { + border-top-left-radius: 0; + border-top-right-radius: 0; +} +.btn-group-justified { + display: table; + width: 100%; + table-layout: fixed; + border-collapse: separate; +} +.btn-group-justified > .btn, +.btn-group-justified > .btn-group { + display: table-cell; + float: none; + width: 1%; +} +.btn-group-justified > .btn-group .btn { + width: 100%; +} +[data-toggle="buttons"] > .btn > input[type="radio"], +[data-toggle="buttons"] > .btn > input[type="checkbox"] { + display: none; +} +.input-group { + position: relative; + display: table; + border-collapse: separate; +} +.input-group[class*="col-"] { + float: none; + padding-right: 0; + padding-left: 0; +} +.input-group .form-control { + position: relative; + z-index: 2; + float: left; + width: 100%; + margin-bottom: 0; +} +.input-group-lg > .form-control, +.input-group-lg > .input-group-addon, +.input-group-lg > .input-group-btn > .btn { + height: 46px; + padding: 10px 16px; + font-size: 18px; + line-height: 1.33; + border-radius: 6px; +} +select.input-group-lg > .form-control, +select.input-group-lg > .input-group-addon, +select.input-group-lg > .input-group-btn > .btn { + height: 46px; + line-height: 46px; +} +textarea.input-group-lg > .form-control, +textarea.input-group-lg > .input-group-addon, +textarea.input-group-lg > .input-group-btn > .btn, +select[multiple].input-group-lg > .form-control, +select[multiple].input-group-lg > .input-group-addon, +select[multiple].input-group-lg > .input-group-btn > .btn { + height: auto; +} +.input-group-sm > .form-control, +.input-group-sm > .input-group-addon, +.input-group-sm > .input-group-btn > .btn { + height: 30px; + padding: 5px 10px; + font-size: 12px; + line-height: 1.5; + border-radius: 3px; +} +select.input-group-sm > .form-control, +select.input-group-sm > .input-group-addon, +select.input-group-sm > .input-group-btn > .btn { + height: 30px; + line-height: 30px; +} +textarea.input-group-sm > .form-control, +textarea.input-group-sm > .input-group-addon, +textarea.input-group-sm > .input-group-btn > .btn, +select[multiple].input-group-sm > .form-control, +select[multiple].input-group-sm > .input-group-addon, +select[multiple].input-group-sm > .input-group-btn > .btn { + height: auto; +} +.input-group-addon, +.input-group-btn, +.input-group .form-control { + display: table-cell; +} +.input-group-addon:not(:first-child):not(:last-child), +.input-group-btn:not(:first-child):not(:last-child), +.input-group .form-control:not(:first-child):not(:last-child) { + border-radius: 0; +} +.input-group-addon, +.input-group-btn { + width: 1%; + white-space: nowrap; + vertical-align: middle; +} +.input-group-addon { + padding: 6px 12px; + font-size: 14px; + font-weight: normal; + line-height: 1; + color: #555; + text-align: center; + background-color: #eee; + border: 1px solid #ccc; + border-radius: 4px; +} +.input-group-addon.input-sm { + padding: 5px 10px; + font-size: 12px; + border-radius: 3px; +} +.input-group-addon.input-lg { + padding: 10px 16px; + font-size: 18px; + border-radius: 6px; +} +.input-group-addon input[type="radio"], +.input-group-addon input[type="checkbox"] { + margin-top: 0; +} +.input-group .form-control:first-child, +.input-group-addon:first-child, +.input-group-btn:first-child > .btn, +.input-group-btn:first-child > .btn-group > .btn, +.input-group-btn:first-child > .dropdown-toggle, +.input-group-btn:last-child > .btn:not(:last-child):not(.dropdown-toggle), +.input-group-btn:last-child > .btn-group:not(:last-child) > .btn { + border-top-right-radius: 0; + border-bottom-right-radius: 0; +} +.input-group-addon:first-child { + border-right: 0; +} +.input-group .form-control:last-child, +.input-group-addon:last-child, +.input-group-btn:last-child > .btn, +.input-group-btn:last-child > .btn-group > .btn, +.input-group-btn:last-child > .dropdown-toggle, +.input-group-btn:first-child > .btn:not(:first-child), +.input-group-btn:first-child > .btn-group:not(:first-child) > .btn { + border-top-left-radius: 0; + border-bottom-left-radius: 0; +} +.input-group-addon:last-child { + border-left: 0; +} +.input-group-btn { + position: relative; + font-size: 0; + white-space: nowrap; +} +.input-group-btn > .btn { + position: relative; +} +.input-group-btn > .btn + .btn { + margin-left: -1px; +} +.input-group-btn > .btn:hover, +.input-group-btn > .btn:focus, +.input-group-btn > .btn:active { + z-index: 2; +} +.input-group-btn:first-child > .btn, +.input-group-btn:first-child > .btn-group { + margin-right: -1px; +} +.input-group-btn:last-child > .btn, +.input-group-btn:last-child > .btn-group { + margin-left: -1px; +} +.nav { + padding-left: 0; + margin-bottom: 0; + list-style: none; +} +.nav > li { + position: relative; + display: block; +} +.nav > li > a { + position: relative; + display: block; + padding: 10px 15px; +} +.nav > li > a:hover, +.nav > li > a:focus { + text-decoration: none; + background-color: #eee; +} +.nav > li.disabled > a { + color: #999; +} +.nav > li.disabled > a:hover, +.nav > li.disabled > a:focus { + color: #999; + text-decoration: none; + cursor: not-allowed; + background-color: transparent; +} +.nav .open > a, +.nav .open > a:hover, +.nav .open > a:focus { + background-color: #eee; + border-color: #428bca; +} +.nav .nav-divider { + height: 1px; + margin: 9px 0; + overflow: hidden; + background-color: #e5e5e5; +} +.nav > li > a > img { + max-width: none; +} +.nav-tabs { + border-bottom: 1px solid #ddd; +} +.nav-tabs > li { + float: left; + margin-bottom: -1px; +} +.nav-tabs > li > a { + margin-right: 2px; + line-height: 1.42857143; + border: 1px solid transparent; + border-radius: 4px 4px 0 0; +} +.nav-tabs > li > a:hover { + border-color: #eee #eee #ddd; +} +.nav-tabs > li.active > a, +.nav-tabs > li.active > a:hover, +.nav-tabs > li.active > a:focus { + color: #555; + cursor: default; + background-color: #fff; + border: 1px solid #ddd; + border-bottom-color: transparent; +} +.nav-tabs.nav-justified { + width: 100%; + border-bottom: 0; +} +.nav-tabs.nav-justified > li { + float: none; +} +.nav-tabs.nav-justified > li > a { + margin-bottom: 5px; + text-align: center; +} +.nav-tabs.nav-justified > .dropdown .dropdown-menu { + top: auto; + left: auto; +} +@media (min-width: 768px) { + .nav-tabs.nav-justified > li { + display: table-cell; + width: 1%; + } + .nav-tabs.nav-justified > li > a { + margin-bottom: 0; + } +} +.nav-tabs.nav-justified > li > a { + margin-right: 0; + border-radius: 4px; +} +.nav-tabs.nav-justified > .active > a, +.nav-tabs.nav-justified > .active > a:hover, +.nav-tabs.nav-justified > .active > a:focus { + border: 1px solid #ddd; +} +@media (min-width: 768px) { + .nav-tabs.nav-justified > li > a { + border-bottom: 1px solid #ddd; + border-radius: 4px 4px 0 0; + } + .nav-tabs.nav-justified > .active > a, + .nav-tabs.nav-justified > .active > a:hover, + .nav-tabs.nav-justified > .active > a:focus { + border-bottom-color: #fff; + } +} +.nav-pills > li { + float: left; +} +.nav-pills > li > a { + border-radius: 4px; +} +.nav-pills > li + li { + margin-left: 2px; +} +.nav-pills > li.active > a, +.nav-pills > li.active > a:hover, +.nav-pills > li.active > a:focus { + color: #fff; + background-color: #428bca; +} +.nav-stacked > li { + float: none; +} +.nav-stacked > li + li { + margin-top: 2px; + margin-left: 0; +} +.nav-justified { + width: 100%; +} +.nav-justified > li { + float: none; +} +.nav-justified > li > a { + margin-bottom: 5px; + text-align: center; +} +.nav-justified > .dropdown .dropdown-menu { + top: auto; + left: auto; +} +@media (min-width: 768px) { + .nav-justified > li { + display: table-cell; + width: 1%; + } + .nav-justified > li > a { + margin-bottom: 0; + } +} +.nav-tabs-justified { + border-bottom: 0; +} +.nav-tabs-justified > li > a { + margin-right: 0; + border-radius: 4px; +} +.nav-tabs-justified > .active > a, +.nav-tabs-justified > .active > a:hover, +.nav-tabs-justified > .active > a:focus { + border: 1px solid #ddd; +} +@media (min-width: 768px) { + .nav-tabs-justified > li > a { + border-bottom: 1px solid #ddd; + border-radius: 4px 4px 0 0; + } + .nav-tabs-justified > .active > a, + .nav-tabs-justified > .active > a:hover, + .nav-tabs-justified > .active > a:focus { + border-bottom-color: #fff; + } +} +.tab-content > .tab-pane { + display: none; +} +.tab-content > .active { + display: block; +} +.nav-tabs .dropdown-menu { + margin-top: -1px; + border-top-left-radius: 0; + border-top-right-radius: 0; +} +.navbar { + position: relative; + min-height: 50px; + margin-bottom: 20px; + border: 1px solid transparent; +} +@media (min-width: 768px) { + .navbar { + border-radius: 4px; + } +} +@media (min-width: 768px) { + .navbar-header { + float: left; + } +} +.navbar-collapse { + max-height: 340px; + padding-right: 15px; + padding-left: 15px; + overflow-x: visible; + -webkit-overflow-scrolling: touch; + border-top: 1px solid transparent; + box-shadow: inset 0 1px 0 rgba(255, 255, 255, .1); +} +.navbar-collapse.in { + overflow-y: auto; +} +@media (min-width: 768px) { + .navbar-collapse { + width: auto; + border-top: 0; + box-shadow: none; + } + .navbar-collapse.collapse { + display: block !important; + height: auto !important; + padding-bottom: 0; + overflow: visible !important; + } + .navbar-collapse.in { + overflow-y: visible; + } + .navbar-fixed-top .navbar-collapse, + .navbar-static-top .navbar-collapse, + .navbar-fixed-bottom .navbar-collapse { + padding-right: 0; + padding-left: 0; + } +} +.container > .navbar-header, +.container-fluid > .navbar-header, +.container > .navbar-collapse, +.container-fluid > .navbar-collapse { + margin-right: -15px; + margin-left: -15px; +} +@media (min-width: 768px) { + .container > .navbar-header, + .container-fluid > .navbar-header, + .container > .navbar-collapse, + .container-fluid > .navbar-collapse { + margin-right: 0; + margin-left: 0; + } +} +.navbar-static-top { + z-index: 1000; + border-width: 0 0 1px; +} +@media (min-width: 768px) { + .navbar-static-top { + border-radius: 0; + } +} +.navbar-fixed-top, +.navbar-fixed-bottom { + position: fixed; + right: 0; + left: 0; + z-index: 1030; +} +@media (min-width: 768px) { + .navbar-fixed-top, + .navbar-fixed-bottom { + border-radius: 0; + } +} +.navbar-fixed-top { + top: 0; + border-width: 0 0 1px; +} +.navbar-fixed-bottom { + bottom: 0; + margin-bottom: 0; + border-width: 1px 0 0; +} +.navbar-brand { + float: left; + height: 50px; + padding: 15px 15px; + font-size: 18px; + line-height: 20px; +} +.navbar-brand:hover, +.navbar-brand:focus { + text-decoration: none; +} +@media (min-width: 768px) { + .navbar > .container .navbar-brand, + .navbar > .container-fluid .navbar-brand { + margin-left: -15px; + } +} +.navbar-toggle { + position: relative; + float: right; + padding: 9px 10px; + margin-top: 8px; + margin-right: 15px; + margin-bottom: 8px; + background-color: transparent; + background-image: none; + border: 1px solid transparent; + border-radius: 4px; +} +.navbar-toggle:focus { + outline: none; +} +.navbar-toggle .icon-bar { + display: block; + width: 22px; + height: 2px; + border-radius: 1px; +} +.navbar-toggle .icon-bar + .icon-bar { + margin-top: 4px; +} +@media (min-width: 768px) { + .navbar-toggle { + display: none; + } +} +.navbar-nav { + margin: 7.5px -15px; +} +.navbar-nav > li > a { + padding-top: 10px; + padding-bottom: 10px; + line-height: 20px; +} +@media (max-width: 767px) { + .navbar-nav .open .dropdown-menu { + position: static; + float: none; + width: auto; + margin-top: 0; + background-color: transparent; + border: 0; + box-shadow: none; + } + .navbar-nav .open .dropdown-menu > li > a, + .navbar-nav .open .dropdown-menu .dropdown-header { + padding: 5px 15px 5px 25px; + } + .navbar-nav .open .dropdown-menu > li > a { + line-height: 20px; + } + .navbar-nav .open .dropdown-menu > li > a:hover, + .navbar-nav .open .dropdown-menu > li > a:focus { + background-image: none; + } +} +@media (min-width: 768px) { + .navbar-nav { + float: left; + margin: 0; + } + .navbar-nav > li { + float: left; + } + .navbar-nav > li > a { + padding-top: 15px; + padding-bottom: 15px; + } + .navbar-nav.navbar-right:last-child { + margin-right: -15px; + } +} +@media (min-width: 768px) { + .navbar-left { + float: left !important; + } + .navbar-right { + float: right !important; + } +} +.navbar-form { + padding: 10px 15px; + margin-top: 8px; + margin-right: -15px; + margin-bottom: 8px; + margin-left: -15px; + border-top: 1px solid transparent; + border-bottom: 1px solid transparent; + -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, .1), 0 1px 0 rgba(255, 255, 255, .1); + box-shadow: inset 0 1px 0 rgba(255, 255, 255, .1), 0 1px 0 rgba(255, 255, 255, .1); +} +@media (min-width: 768px) { + .navbar-form .form-group { + display: inline-block; + margin-bottom: 0; + vertical-align: middle; + } + .navbar-form .form-control { + display: inline-block; + width: auto; + vertical-align: middle; + } + .navbar-form .input-group > .form-control { + width: 100%; + } + .navbar-form .control-label { + margin-bottom: 0; + vertical-align: middle; + } + .navbar-form .radio, + .navbar-form .checkbox { + display: inline-block; + padding-left: 0; + margin-top: 0; + margin-bottom: 0; + vertical-align: middle; + } + .navbar-form .radio input[type="radio"], + .navbar-form .checkbox input[type="checkbox"] { + float: none; + margin-left: 0; + } + .navbar-form .has-feedback .form-control-feedback { + top: 0; + } +} +@media (max-width: 767px) { + .navbar-form .form-group { + margin-bottom: 5px; + } +} +@media (min-width: 768px) { + .navbar-form { + width: auto; + padding-top: 0; + padding-bottom: 0; + margin-right: 0; + margin-left: 0; + border: 0; + -webkit-box-shadow: none; + box-shadow: none; + } + .navbar-form.navbar-right:last-child { + margin-right: -15px; + } +} +.navbar-nav > li > .dropdown-menu { + margin-top: 0; + border-top-left-radius: 0; + border-top-right-radius: 0; +} +.navbar-fixed-bottom .navbar-nav > li > .dropdown-menu { + border-bottom-right-radius: 0; + border-bottom-left-radius: 0; +} +.navbar-btn { + margin-top: 8px; + margin-bottom: 8px; +} +.navbar-btn.btn-sm { + margin-top: 10px; + margin-bottom: 10px; +} +.navbar-btn.btn-xs { + margin-top: 14px; + margin-bottom: 14px; +} +.navbar-text { + margin-top: 15px; + margin-bottom: 15px; +} +@media (min-width: 768px) { + .navbar-text { + float: left; + margin-right: 15px; + margin-left: 15px; + } + .navbar-text.navbar-right:last-child { + margin-right: 0; + } +} +.navbar-default { + background-color: #f8f8f8; + border-color: #e7e7e7; +} +.navbar-default .navbar-brand { + color: #777; +} +.navbar-default .navbar-brand:hover, +.navbar-default .navbar-brand:focus { + color: #5e5e5e; + background-color: transparent; +} +.navbar-default .navbar-text { + color: #777; +} +.navbar-default .navbar-nav > li > a { + color: #777; +} +.navbar-default .navbar-nav > li > a:hover, +.navbar-default .navbar-nav > li > a:focus { + color: #333; + background-color: transparent; +} +.navbar-default .navbar-nav > .active > a, +.navbar-default .navbar-nav > .active > a:hover, +.navbar-default .navbar-nav > .active > a:focus { + color: #555; + background-color: #e7e7e7; +} +.navbar-default .navbar-nav > .disabled > a, +.navbar-default .navbar-nav > .disabled > a:hover, +.navbar-default .navbar-nav > .disabled > a:focus { + color: #ccc; + background-color: transparent; +} +.navbar-default .navbar-toggle { + border-color: #ddd; +} +.navbar-default .navbar-toggle:hover, +.navbar-default .navbar-toggle:focus { + background-color: #ddd; +} +.navbar-default .navbar-toggle .icon-bar { + background-color: #888; +} +.navbar-default .navbar-collapse, +.navbar-default .navbar-form { + border-color: #e7e7e7; +} +.navbar-default .navbar-nav > .open > a, +.navbar-default .navbar-nav > .open > a:hover, +.navbar-default .navbar-nav > .open > a:focus { + color: #555; + background-color: #e7e7e7; +} +@media (max-width: 767px) { + .navbar-default .navbar-nav .open .dropdown-menu > li > a { + color: #777; + } + .navbar-default .navbar-nav .open .dropdown-menu > li > a:hover, + .navbar-default .navbar-nav .open .dropdown-menu > li > a:focus { + color: #333; + background-color: transparent; + } + .navbar-default .navbar-nav .open .dropdown-menu > .active > a, + .navbar-default .navbar-nav .open .dropdown-menu > .active > a:hover, + .navbar-default .navbar-nav .open .dropdown-menu > .active > a:focus { + color: #555; + background-color: #e7e7e7; + } + .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a, + .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a:hover, + .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a:focus { + color: #ccc; + background-color: transparent; + } +} +.navbar-default .navbar-link { + color: #777; +} +.navbar-default .navbar-link:hover { + color: #333; +} +.navbar-inverse { + background-color: #222; + border-color: #080808; +} +.navbar-inverse .navbar-brand { + color: #999; +} +.navbar-inverse .navbar-brand:hover, +.navbar-inverse .navbar-brand:focus { + color: #fff; + background-color: transparent; +} +.navbar-inverse .navbar-text { + color: #999; +} +.navbar-inverse .navbar-nav > li > a { + color: #999; +} +.navbar-inverse .navbar-nav > li > a:hover, +.navbar-inverse .navbar-nav > li > a:focus { + color: #fff; + background-color: transparent; +} +.navbar-inverse .navbar-nav > .active > a, +.navbar-inverse .navbar-nav > .active > a:hover, +.navbar-inverse .navbar-nav > .active > a:focus { + color: #fff; + background-color: #080808; +} +.navbar-inverse .navbar-nav > .disabled > a, +.navbar-inverse .navbar-nav > .disabled > a:hover, +.navbar-inverse .navbar-nav > .disabled > a:focus { + color: #444; + background-color: transparent; +} +.navbar-inverse .navbar-toggle { + border-color: #333; +} +.navbar-inverse .navbar-toggle:hover, +.navbar-inverse .navbar-toggle:focus { + background-color: #333; +} +.navbar-inverse .navbar-toggle .icon-bar { + background-color: #fff; +} +.navbar-inverse .navbar-collapse, +.navbar-inverse .navbar-form { + border-color: #101010; +} +.navbar-inverse .navbar-nav > .open > a, +.navbar-inverse .navbar-nav > .open > a:hover, +.navbar-inverse .navbar-nav > .open > a:focus { + color: #fff; + background-color: #080808; +} +@media (max-width: 767px) { + .navbar-inverse .navbar-nav .open .dropdown-menu > .dropdown-header { + border-color: #080808; + } + .navbar-inverse .navbar-nav .open .dropdown-menu .divider { + background-color: #080808; + } + .navbar-inverse .navbar-nav .open .dropdown-menu > li > a { + color: #999; + } + .navbar-inverse .navbar-nav .open .dropdown-menu > li > a:hover, + .navbar-inverse .navbar-nav .open .dropdown-menu > li > a:focus { + color: #fff; + background-color: transparent; + } + .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a, + .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a:hover, + .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a:focus { + color: #fff; + background-color: #080808; + } + .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a, + .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a:hover, + .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a:focus { + color: #444; + background-color: transparent; + } +} +.navbar-inverse .navbar-link { + color: #999; +} +.navbar-inverse .navbar-link:hover { + color: #fff; +} +.breadcrumb { + padding: 8px 15px; + margin-bottom: 20px; + list-style: none; + background-color: #f5f5f5; + border-radius: 4px; +} +.breadcrumb > li { + display: inline-block; +} +.breadcrumb > li + li:before { + padding: 0 5px; + color: #ccc; + content: "/\00a0"; +} +.breadcrumb > .active { + color: #999; +} +.pagination { + display: inline-block; + padding-left: 0; + margin: 20px 0; + border-radius: 4px; +} +.pagination > li { + display: inline; +} +.pagination > li > a, +.pagination > li > span { + position: relative; + float: left; + padding: 6px 12px; + margin-left: -1px; + line-height: 1.42857143; + color: #428bca; + text-decoration: none; + background-color: #fff; + border: 1px solid #ddd; +} +.pagination > li:first-child > a, +.pagination > li:first-child > span { + margin-left: 0; + border-top-left-radius: 4px; + border-bottom-left-radius: 4px; +} +.pagination > li:last-child > a, +.pagination > li:last-child > span { + border-top-right-radius: 4px; + border-bottom-right-radius: 4px; +} +.pagination > li > a:hover, +.pagination > li > span:hover, +.pagination > li > a:focus, +.pagination > li > span:focus { + color: #2a6496; + background-color: #eee; + border-color: #ddd; +} +.pagination > .active > a, +.pagination > .active > span, +.pagination > .active > a:hover, +.pagination > .active > span:hover, +.pagination > .active > a:focus, +.pagination > .active > span:focus { + z-index: 2; + color: #fff; + cursor: default; + background-color: #428bca; + border-color: #428bca; +} +.pagination > .disabled > span, +.pagination > .disabled > span:hover, +.pagination > .disabled > span:focus, +.pagination > .disabled > a, +.pagination > .disabled > a:hover, +.pagination > .disabled > a:focus { + color: #999; + cursor: not-allowed; + background-color: #fff; + border-color: #ddd; +} +.pagination-lg > li > a, +.pagination-lg > li > span { + padding: 10px 16px; + font-size: 18px; +} +.pagination-lg > li:first-child > a, +.pagination-lg > li:first-child > span { + border-top-left-radius: 6px; + border-bottom-left-radius: 6px; +} +.pagination-lg > li:last-child > a, +.pagination-lg > li:last-child > span { + border-top-right-radius: 6px; + border-bottom-right-radius: 6px; +} +.pagination-sm > li > a, +.pagination-sm > li > span { + padding: 5px 10px; + font-size: 12px; +} +.pagination-sm > li:first-child > a, +.pagination-sm > li:first-child > span { + border-top-left-radius: 3px; + border-bottom-left-radius: 3px; +} +.pagination-sm > li:last-child > a, +.pagination-sm > li:last-child > span { + border-top-right-radius: 3px; + border-bottom-right-radius: 3px; +} +.pager { + padding-left: 0; + margin: 20px 0; + text-align: center; + list-style: none; +} +.pager li { + display: inline; +} +.pager li > a, +.pager li > span { + display: inline-block; + padding: 5px 14px; + background-color: #fff; + border: 1px solid #ddd; + border-radius: 15px; +} +.pager li > a:hover, +.pager li > a:focus { + text-decoration: none; + background-color: #eee; +} +.pager .next > a, +.pager .next > span { + float: right; +} +.pager .previous > a, +.pager .previous > span { + float: left; +} +.pager .disabled > a, +.pager .disabled > a:hover, +.pager .disabled > a:focus, +.pager .disabled > span { + color: #999; + cursor: not-allowed; + background-color: #fff; +} +.label { + display: inline; + padding: .2em .6em .3em; + font-size: 75%; + font-weight: bold; + line-height: 1; + color: #fff; + text-align: center; + white-space: nowrap; + vertical-align: baseline; + border-radius: .25em; +} +.label[href]:hover, +.label[href]:focus { + color: #fff; + text-decoration: none; + cursor: pointer; +} +.label:empty { + display: none; +} +.btn .label { + position: relative; + top: -1px; +} +.label-default { + background-color: #999; +} +.label-default[href]:hover, +.label-default[href]:focus { + background-color: #808080; +} +.label-primary { + background-color: #428bca; +} +.label-primary[href]:hover, +.label-primary[href]:focus { + background-color: #3071a9; +} +.label-success { + background-color: #5cb85c; +} +.label-success[href]:hover, +.label-success[href]:focus { + background-color: #449d44; +} +.label-info { + background-color: #5bc0de; +} +.label-info[href]:hover, +.label-info[href]:focus { + background-color: #31b0d5; +} +.label-warning { + background-color: #f0ad4e; +} +.label-warning[href]:hover, +.label-warning[href]:focus { + background-color: #ec971f; +} +.label-danger { + background-color: #d9534f; +} +.label-danger[href]:hover, +.label-danger[href]:focus { + background-color: #c9302c; +} +.badge { + display: inline-block; + min-width: 10px; + padding: 3px 7px; + font-size: 12px; + font-weight: bold; + line-height: 1; + color: #fff; + text-align: center; + white-space: nowrap; + vertical-align: baseline; + background-color: #999; + border-radius: 10px; +} +.badge:empty { + display: none; +} +.btn .badge { + position: relative; + top: -1px; +} +.btn-xs .badge { + top: 0; + padding: 1px 5px; +} +a.badge:hover, +a.badge:focus { + color: #fff; + text-decoration: none; + cursor: pointer; +} +a.list-group-item.active > .badge, +.nav-pills > .active > a > .badge { + color: #428bca; + background-color: #fff; +} +.nav-pills > li > a > .badge { + margin-left: 3px; +} +.jumbotron { + padding: 30px; + margin-bottom: 30px; + color: inherit; + background-color: #eee; +} +.jumbotron h1, +.jumbotron .h1 { + color: inherit; +} +.jumbotron p { + margin-bottom: 15px; + font-size: 21px; + font-weight: 200; +} +.container .jumbotron { + border-radius: 6px; +} +.jumbotron .container { + max-width: 100%; +} +@media screen and (min-width: 768px) { + .jumbotron { + padding-top: 48px; + padding-bottom: 48px; + } + .container .jumbotron { + padding-right: 60px; + padding-left: 60px; + } + .jumbotron h1, + .jumbotron .h1 { + font-size: 63px; + } +} +.thumbnail { + display: block; + padding: 4px; + margin-bottom: 20px; + line-height: 1.42857143; + background-color: #fff; + border: 1px solid #ddd; + border-radius: 4px; + -webkit-transition: all .2s ease-in-out; + transition: all .2s ease-in-out; +} +.thumbnail > img, +.thumbnail a > img { + margin-right: auto; + margin-left: auto; +} +a.thumbnail:hover, +a.thumbnail:focus, +a.thumbnail.active { + border-color: #428bca; +} +.thumbnail .caption { + padding: 9px; + color: #333; +} +.alert { + padding: 15px; + margin-bottom: 20px; + border: 1px solid transparent; + border-radius: 4px; +} +.alert h4 { + margin-top: 0; + color: inherit; +} +.alert .alert-link { + font-weight: bold; +} +.alert > p, +.alert > ul { + margin-bottom: 0; +} +.alert > p + p { + margin-top: 5px; +} +.alert-dismissable { + padding-right: 35px; +} +.alert-dismissable .close { + position: relative; + top: -2px; + right: -21px; + color: inherit; +} +.alert-success { + color: #3c763d; + background-color: #dff0d8; + border-color: #d6e9c6; +} +.alert-success hr { + border-top-color: #c9e2b3; +} +.alert-success .alert-link { + color: #2b542c; +} +.alert-info { + color: #31708f; + background-color: #d9edf7; + border-color: #bce8f1; +} +.alert-info hr { + border-top-color: #a6e1ec; +} +.alert-info .alert-link { + color: #245269; +} +.alert-warning { + color: #8a6d3b; + background-color: #fcf8e3; + border-color: #faebcc; +} +.alert-warning hr { + border-top-color: #f7e1b5; +} +.alert-warning .alert-link { + color: #66512c; +} +.alert-danger { + color: #a94442; + background-color: #f2dede; + border-color: #ebccd1; +} +.alert-danger hr { + border-top-color: #e4b9c0; +} +.alert-danger .alert-link { + color: #843534; +} +@-webkit-keyframes progress-bar-stripes { + from { + background-position: 40px 0; + } + to { + background-position: 0 0; + } +} +@keyframes progress-bar-stripes { + from { + background-position: 40px 0; + } + to { + background-position: 0 0; + } +} +.progress { + height: 20px; + margin-bottom: 20px; + overflow: hidden; + background-color: #f5f5f5; + border-radius: 4px; + -webkit-box-shadow: inset 0 1px 2px rgba(0, 0, 0, .1); + box-shadow: inset 0 1px 2px rgba(0, 0, 0, .1); +} +.progress-bar { + float: left; + width: 0; + height: 100%; + font-size: 12px; + line-height: 20px; + color: #fff; + text-align: center; + background-color: #428bca; + -webkit-box-shadow: inset 0 -1px 0 rgba(0, 0, 0, .15); + box-shadow: inset 0 -1px 0 rgba(0, 0, 0, .15); + -webkit-transition: width .6s ease; + transition: width .6s ease; +} +.progress-striped .progress-bar { + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-size: 40px 40px; +} +.progress.active .progress-bar { + -webkit-animation: progress-bar-stripes 2s linear infinite; + animation: progress-bar-stripes 2s linear infinite; +} +.progress-bar-success { + background-color: #5cb85c; +} +.progress-striped .progress-bar-success { + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); +} +.progress-bar-info { + background-color: #5bc0de; +} +.progress-striped .progress-bar-info { + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); +} +.progress-bar-warning { + background-color: #f0ad4e; +} +.progress-striped .progress-bar-warning { + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); +} +.progress-bar-danger { + background-color: #d9534f; +} +.progress-striped .progress-bar-danger { + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); +} +.media, +.media-body { + overflow: hidden; + zoom: 1; +} +.media, +.media .media { + margin-top: 15px; +} +.media:first-child { + margin-top: 0; +} +.media-object { + display: block; +} +.media-heading { + margin: 0 0 5px; +} +.media > .pull-left { + margin-right: 10px; +} +.media > .pull-right { + margin-left: 10px; +} +.media-list { + padding-left: 0; + list-style: none; +} +.list-group { + padding-left: 0; + margin-bottom: 20px; +} +.list-group-item { + position: relative; + display: block; + padding: 10px 15px; + margin-bottom: -1px; + background-color: #fff; + border: 1px solid #ddd; +} +.list-group-item:first-child { + border-top-left-radius: 4px; + border-top-right-radius: 4px; +} +.list-group-item:last-child { + margin-bottom: 0; + border-bottom-right-radius: 4px; + border-bottom-left-radius: 4px; +} +.list-group-item > .badge { + float: right; +} +.list-group-item > .badge + .badge { + margin-right: 5px; +} +a.list-group-item { + color: #555; +} +a.list-group-item .list-group-item-heading { + color: #333; +} +a.list-group-item:hover, +a.list-group-item:focus { + text-decoration: none; + background-color: #f5f5f5; +} +a.list-group-item.active, +a.list-group-item.active:hover, +a.list-group-item.active:focus { + z-index: 2; + color: #fff; + background-color: #428bca; + border-color: #428bca; +} +a.list-group-item.active .list-group-item-heading, +a.list-group-item.active:hover .list-group-item-heading, +a.list-group-item.active:focus .list-group-item-heading { + color: inherit; +} +a.list-group-item.active .list-group-item-text, +a.list-group-item.active:hover .list-group-item-text, +a.list-group-item.active:focus .list-group-item-text { + color: #e1edf7; +} +.list-group-item-success { + color: #3c763d; + background-color: #dff0d8; +} +a.list-group-item-success { + color: #3c763d; +} +a.list-group-item-success .list-group-item-heading { + color: inherit; +} +a.list-group-item-success:hover, +a.list-group-item-success:focus { + color: #3c763d; + background-color: #d0e9c6; +} +a.list-group-item-success.active, +a.list-group-item-success.active:hover, +a.list-group-item-success.active:focus { + color: #fff; + background-color: #3c763d; + border-color: #3c763d; +} +.list-group-item-info { + color: #31708f; + background-color: #d9edf7; +} +a.list-group-item-info { + color: #31708f; +} +a.list-group-item-info .list-group-item-heading { + color: inherit; +} +a.list-group-item-info:hover, +a.list-group-item-info:focus { + color: #31708f; + background-color: #c4e3f3; +} +a.list-group-item-info.active, +a.list-group-item-info.active:hover, +a.list-group-item-info.active:focus { + color: #fff; + background-color: #31708f; + border-color: #31708f; +} +.list-group-item-warning { + color: #8a6d3b; + background-color: #fcf8e3; +} +a.list-group-item-warning { + color: #8a6d3b; +} +a.list-group-item-warning .list-group-item-heading { + color: inherit; +} +a.list-group-item-warning:hover, +a.list-group-item-warning:focus { + color: #8a6d3b; + background-color: #faf2cc; +} +a.list-group-item-warning.active, +a.list-group-item-warning.active:hover, +a.list-group-item-warning.active:focus { + color: #fff; + background-color: #8a6d3b; + border-color: #8a6d3b; +} +.list-group-item-danger { + color: #a94442; + background-color: #f2dede; +} +a.list-group-item-danger { + color: #a94442; +} +a.list-group-item-danger .list-group-item-heading { + color: inherit; +} +a.list-group-item-danger:hover, +a.list-group-item-danger:focus { + color: #a94442; + background-color: #ebcccc; +} +a.list-group-item-danger.active, +a.list-group-item-danger.active:hover, +a.list-group-item-danger.active:focus { + color: #fff; + background-color: #a94442; + border-color: #a94442; +} +.list-group-item-heading { + margin-top: 0; + margin-bottom: 5px; +} +.list-group-item-text { + margin-bottom: 0; + line-height: 1.3; +} +.panel { + margin-bottom: 20px; + background-color: #fff; + border: 1px solid transparent; + border-radius: 4px; + -webkit-box-shadow: 0 1px 1px rgba(0, 0, 0, .05); + box-shadow: 0 1px 1px rgba(0, 0, 0, .05); +} +.panel-body { + padding: 15px; +} +.panel-heading { + padding: 10px 15px; + border-bottom: 1px solid transparent; + border-top-left-radius: 3px; + border-top-right-radius: 3px; +} +.panel-heading > .dropdown .dropdown-toggle { + color: inherit; +} +.panel-title { + margin-top: 0; + margin-bottom: 0; + font-size: 16px; + color: inherit; +} +.panel-title > a { + color: inherit; +} +.panel-footer { + padding: 10px 15px; + background-color: #f5f5f5; + border-top: 1px solid #ddd; + border-bottom-right-radius: 3px; + border-bottom-left-radius: 3px; +} +.panel > .list-group { + margin-bottom: 0; +} +.panel > .list-group .list-group-item { + border-width: 1px 0; + border-radius: 0; +} +.panel > .list-group:first-child .list-group-item:first-child { + border-top: 0; + border-top-left-radius: 3px; + border-top-right-radius: 3px; +} +.panel > .list-group:last-child .list-group-item:last-child { + border-bottom: 0; + border-bottom-right-radius: 3px; + border-bottom-left-radius: 3px; +} +.panel-heading + .list-group .list-group-item:first-child { + border-top-width: 0; +} +.panel > .table, +.panel > .table-responsive > .table { + margin-bottom: 0; +} +.panel > .table:first-child, +.panel > .table-responsive:first-child > .table:first-child { + border-top-left-radius: 3px; + border-top-right-radius: 3px; +} +.panel > .table:first-child > thead:first-child > tr:first-child td:first-child, +.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child td:first-child, +.panel > .table:first-child > tbody:first-child > tr:first-child td:first-child, +.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child td:first-child, +.panel > .table:first-child > thead:first-child > tr:first-child th:first-child, +.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child th:first-child, +.panel > .table:first-child > tbody:first-child > tr:first-child th:first-child, +.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child th:first-child { + border-top-left-radius: 3px; +} +.panel > .table:first-child > thead:first-child > tr:first-child td:last-child, +.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child td:last-child, +.panel > .table:first-child > tbody:first-child > tr:first-child td:last-child, +.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child td:last-child, +.panel > .table:first-child > thead:first-child > tr:first-child th:last-child, +.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child th:last-child, +.panel > .table:first-child > tbody:first-child > tr:first-child th:last-child, +.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child th:last-child { + border-top-right-radius: 3px; +} +.panel > .table:last-child, +.panel > .table-responsive:last-child > .table:last-child { + border-bottom-right-radius: 3px; + border-bottom-left-radius: 3px; +} +.panel > .table:last-child > tbody:last-child > tr:last-child td:first-child, +.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child td:first-child, +.panel > .table:last-child > tfoot:last-child > tr:last-child td:first-child, +.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child td:first-child, +.panel > .table:last-child > tbody:last-child > tr:last-child th:first-child, +.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child th:first-child, +.panel > .table:last-child > tfoot:last-child > tr:last-child th:first-child, +.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child th:first-child { + border-bottom-left-radius: 3px; +} +.panel > .table:last-child > tbody:last-child > tr:last-child td:last-child, +.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child td:last-child, +.panel > .table:last-child > tfoot:last-child > tr:last-child td:last-child, +.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child td:last-child, +.panel > .table:last-child > tbody:last-child > tr:last-child th:last-child, +.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child th:last-child, +.panel > .table:last-child > tfoot:last-child > tr:last-child th:last-child, +.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child th:last-child { + border-bottom-right-radius: 3px; +} +.panel > .panel-body + .table, +.panel > .panel-body + .table-responsive { + border-top: 1px solid #ddd; +} +.panel > .table > tbody:first-child > tr:first-child th, +.panel > .table > tbody:first-child > tr:first-child td { + border-top: 0; +} +.panel > .table-bordered, +.panel > .table-responsive > .table-bordered { + border: 0; +} +.panel > .table-bordered > thead > tr > th:first-child, +.panel > .table-responsive > .table-bordered > thead > tr > th:first-child, +.panel > .table-bordered > tbody > tr > th:first-child, +.panel > .table-responsive > .table-bordered > tbody > tr > th:first-child, +.panel > .table-bordered > tfoot > tr > th:first-child, +.panel > .table-responsive > .table-bordered > tfoot > tr > th:first-child, +.panel > .table-bordered > thead > tr > td:first-child, +.panel > .table-responsive > .table-bordered > thead > tr > td:first-child, +.panel > .table-bordered > tbody > tr > td:first-child, +.panel > .table-responsive > .table-bordered > tbody > tr > td:first-child, +.panel > .table-bordered > tfoot > tr > td:first-child, +.panel > .table-responsive > .table-bordered > tfoot > tr > td:first-child { + border-left: 0; +} +.panel > .table-bordered > thead > tr > th:last-child, +.panel > .table-responsive > .table-bordered > thead > tr > th:last-child, +.panel > .table-bordered > tbody > tr > th:last-child, +.panel > .table-responsive > .table-bordered > tbody > tr > th:last-child, +.panel > .table-bordered > tfoot > tr > th:last-child, +.panel > .table-responsive > .table-bordered > tfoot > tr > th:last-child, +.panel > .table-bordered > thead > tr > td:last-child, +.panel > .table-responsive > .table-bordered > thead > tr > td:last-child, +.panel > .table-bordered > tbody > tr > td:last-child, +.panel > .table-responsive > .table-bordered > tbody > tr > td:last-child, +.panel > .table-bordered > tfoot > tr > td:last-child, +.panel > .table-responsive > .table-bordered > tfoot > tr > td:last-child { + border-right: 0; +} +.panel > .table-bordered > thead > tr:first-child > td, +.panel > .table-responsive > .table-bordered > thead > tr:first-child > td, +.panel > .table-bordered > tbody > tr:first-child > td, +.panel > .table-responsive > .table-bordered > tbody > tr:first-child > td, +.panel > .table-bordered > thead > tr:first-child > th, +.panel > .table-responsive > .table-bordered > thead > tr:first-child > th, +.panel > .table-bordered > tbody > tr:first-child > th, +.panel > .table-responsive > .table-bordered > tbody > tr:first-child > th { + border-bottom: 0; +} +.panel > .table-bordered > tbody > tr:last-child > td, +.panel > .table-responsive > .table-bordered > tbody > tr:last-child > td, +.panel > .table-bordered > tfoot > tr:last-child > td, +.panel > .table-responsive > .table-bordered > tfoot > tr:last-child > td, +.panel > .table-bordered > tbody > tr:last-child > th, +.panel > .table-responsive > .table-bordered > tbody > tr:last-child > th, +.panel > .table-bordered > tfoot > tr:last-child > th, +.panel > .table-responsive > .table-bordered > tfoot > tr:last-child > th { + border-bottom: 0; +} +.panel > .table-responsive { + margin-bottom: 0; + border: 0; +} +.panel-group { + margin-bottom: 20px; +} +.panel-group .panel { + margin-bottom: 0; + overflow: hidden; + border-radius: 4px; +} +.panel-group .panel + .panel { + margin-top: 5px; +} +.panel-group .panel-heading { + border-bottom: 0; +} +.panel-group .panel-heading + .panel-collapse .panel-body { + border-top: 1px solid #ddd; +} +.panel-group .panel-footer { + border-top: 0; +} +.panel-group .panel-footer + .panel-collapse .panel-body { + border-bottom: 1px solid #ddd; +} +.panel-default { + border-color: #ddd; +} +.panel-default > .panel-heading { + color: #333; + background-color: #f5f5f5; + border-color: #ddd; +} +.panel-default > .panel-heading + .panel-collapse .panel-body { + border-top-color: #ddd; +} +.panel-default > .panel-footer + .panel-collapse .panel-body { + border-bottom-color: #ddd; +} +.panel-primary { + border-color: #428bca; +} +.panel-primary > .panel-heading { + color: #fff; + background-color: #428bca; + border-color: #428bca; +} +.panel-primary > .panel-heading + .panel-collapse .panel-body { + border-top-color: #428bca; +} +.panel-primary > .panel-footer + .panel-collapse .panel-body { + border-bottom-color: #428bca; +} +.panel-success { + border-color: #d6e9c6; +} +.panel-success > .panel-heading { + color: #3c763d; + background-color: #dff0d8; + border-color: #d6e9c6; +} +.panel-success > .panel-heading + .panel-collapse .panel-body { + border-top-color: #d6e9c6; +} +.panel-success > .panel-footer + .panel-collapse .panel-body { + border-bottom-color: #d6e9c6; +} +.panel-info { + border-color: #bce8f1; +} +.panel-info > .panel-heading { + color: #31708f; + background-color: #d9edf7; + border-color: #bce8f1; +} +.panel-info > .panel-heading + .panel-collapse .panel-body { + border-top-color: #bce8f1; +} +.panel-info > .panel-footer + .panel-collapse .panel-body { + border-bottom-color: #bce8f1; +} +.panel-warning { + border-color: #faebcc; +} +.panel-warning > .panel-heading { + color: #8a6d3b; + background-color: #fcf8e3; + border-color: #faebcc; +} +.panel-warning > .panel-heading + .panel-collapse .panel-body { + border-top-color: #faebcc; +} +.panel-warning > .panel-footer + .panel-collapse .panel-body { + border-bottom-color: #faebcc; +} +.panel-danger { + border-color: #ebccd1; +} +.panel-danger > .panel-heading { + color: #a94442; + background-color: #f2dede; + border-color: #ebccd1; +} +.panel-danger > .panel-heading + .panel-collapse .panel-body { + border-top-color: #ebccd1; +} +.panel-danger > .panel-footer + .panel-collapse .panel-body { + border-bottom-color: #ebccd1; +} +.well { + min-height: 20px; + padding: 19px; + margin-bottom: 20px; + background-color: #f5f5f5; + border: 1px solid #e3e3e3; + border-radius: 4px; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .05); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .05); +} +.well blockquote { + border-color: #ddd; + border-color: rgba(0, 0, 0, .15); +} +.well-lg { + padding: 24px; + border-radius: 6px; +} +.well-sm { + padding: 9px; + border-radius: 3px; +} +.close { + float: right; + font-size: 21px; + font-weight: bold; + line-height: 1; + color: #000; + text-shadow: 0 1px 0 #fff; + filter: alpha(opacity=20); + opacity: .2; +} +.close:hover, +.close:focus { + color: #000; + text-decoration: none; + cursor: pointer; + filter: alpha(opacity=50); + opacity: .5; +} +button.close { + -webkit-appearance: none; + padding: 0; + cursor: pointer; + background: transparent; + border: 0; +} +.modal-open { + overflow: hidden; +} +.modal { + position: fixed; + top: 0; + right: 0; + bottom: 0; + left: 0; + z-index: 1050; + display: none; + overflow: auto; + overflow-y: scroll; + -webkit-overflow-scrolling: touch; + outline: 0; +} +.modal.fade .modal-dialog { + -webkit-transition: -webkit-transform .3s ease-out; + -moz-transition: -moz-transform .3s ease-out; + -o-transition: -o-transform .3s ease-out; + transition: transform .3s ease-out; + -webkit-transform: translate(0, -25%); + -ms-transform: translate(0, -25%); + transform: translate(0, -25%); +} +.modal.in .modal-dialog { + -webkit-transform: translate(0, 0); + -ms-transform: translate(0, 0); + transform: translate(0, 0); +} +.modal-dialog { + position: relative; + width: auto; + margin: 10px; +} +.modal-content { + position: relative; + background-color: #fff; + background-clip: padding-box; + border: 1px solid #999; + border: 1px solid rgba(0, 0, 0, .2); + border-radius: 6px; + outline: none; + -webkit-box-shadow: 0 3px 9px rgba(0, 0, 0, .5); + box-shadow: 0 3px 9px rgba(0, 0, 0, .5); +} +.modal-backdrop { + position: fixed; + top: 0; + right: 0; + bottom: 0; + left: 0; + z-index: 1040; + background-color: #000; +} +.modal-backdrop.fade { + filter: alpha(opacity=0); + opacity: 0; +} +.modal-backdrop.in { + filter: alpha(opacity=50); + opacity: .5; +} +.modal-header { + min-height: 16.42857143px; + padding: 15px; + border-bottom: 1px solid #e5e5e5; +} +.modal-header .close { + margin-top: -2px; +} +.modal-title { + margin: 0; + line-height: 1.42857143; +} +.modal-body { + position: relative; + padding: 20px; +} +.modal-footer { + padding: 19px 20px 20px; + margin-top: 15px; + text-align: right; + border-top: 1px solid #e5e5e5; +} +.modal-footer .btn + .btn { + margin-bottom: 0; + margin-left: 5px; +} +.modal-footer .btn-group .btn + .btn { + margin-left: -1px; +} +.modal-footer .btn-block + .btn-block { + margin-left: 0; +} +@media (min-width: 768px) { + .modal-dialog { + width: 600px; + margin: 30px auto; + } + .modal-content { + -webkit-box-shadow: 0 5px 15px rgba(0, 0, 0, .5); + box-shadow: 0 5px 15px rgba(0, 0, 0, .5); + } + .modal-sm { + width: 300px; + } +} +@media (min-width: 992px) { + .modal-lg { + width: 900px; + } +} +.tooltip { + position: absolute; + z-index: 1030; + display: block; + font-size: 12px; + line-height: 1.4; + visibility: visible; + filter: alpha(opacity=0); + opacity: 0; +} +.tooltip.in { + filter: alpha(opacity=90); + opacity: .9; +} +.tooltip.top { + padding: 5px 0; + margin-top: -3px; +} +.tooltip.right { + padding: 0 5px; + margin-left: 3px; +} +.tooltip.bottom { + padding: 5px 0; + margin-top: 3px; +} +.tooltip.left { + padding: 0 5px; + margin-left: -3px; +} +.tooltip-inner { + max-width: 200px; + padding: 3px 8px; + color: #fff; + text-align: center; + text-decoration: none; + background-color: #000; + border-radius: 4px; +} +.tooltip-arrow { + position: absolute; + width: 0; + height: 0; + border-color: transparent; + border-style: solid; +} +.tooltip.top .tooltip-arrow { + bottom: 0; + left: 50%; + margin-left: -5px; + border-width: 5px 5px 0; + border-top-color: #000; +} +.tooltip.top-left .tooltip-arrow { + bottom: 0; + left: 5px; + border-width: 5px 5px 0; + border-top-color: #000; +} +.tooltip.top-right .tooltip-arrow { + right: 5px; + bottom: 0; + border-width: 5px 5px 0; + border-top-color: #000; +} +.tooltip.right .tooltip-arrow { + top: 50%; + left: 0; + margin-top: -5px; + border-width: 5px 5px 5px 0; + border-right-color: #000; +} +.tooltip.left .tooltip-arrow { + top: 50%; + right: 0; + margin-top: -5px; + border-width: 5px 0 5px 5px; + border-left-color: #000; +} +.tooltip.bottom .tooltip-arrow { + top: 0; + left: 50%; + margin-left: -5px; + border-width: 0 5px 5px; + border-bottom-color: #000; +} +.tooltip.bottom-left .tooltip-arrow { + top: 0; + left: 5px; + border-width: 0 5px 5px; + border-bottom-color: #000; +} +.tooltip.bottom-right .tooltip-arrow { + top: 0; + right: 5px; + border-width: 0 5px 5px; + border-bottom-color: #000; +} +.popover { + position: absolute; + top: 0; + left: 0; + z-index: 1010; + display: none; + max-width: 276px; + padding: 1px; + text-align: left; + white-space: normal; + background-color: #fff; + background-clip: padding-box; + border: 1px solid #ccc; + border: 1px solid rgba(0, 0, 0, .2); + border-radius: 6px; + -webkit-box-shadow: 0 5px 10px rgba(0, 0, 0, .2); + box-shadow: 0 5px 10px rgba(0, 0, 0, .2); +} +.popover.top { + margin-top: -10px; +} +.popover.right { + margin-left: 10px; +} +.popover.bottom { + margin-top: 10px; +} +.popover.left { + margin-left: -10px; +} +.popover-title { + padding: 8px 14px; + margin: 0; + font-size: 14px; + font-weight: normal; + line-height: 18px; + background-color: #f7f7f7; + border-bottom: 1px solid #ebebeb; + border-radius: 5px 5px 0 0; +} +.popover-content { + padding: 9px 14px; +} +.popover > .arrow, +.popover > .arrow:after { + position: absolute; + display: block; + width: 0; + height: 0; + border-color: transparent; + border-style: solid; +} +.popover > .arrow { + border-width: 11px; +} +.popover > .arrow:after { + content: ""; + border-width: 10px; +} +.popover.top > .arrow { + bottom: -11px; + left: 50%; + margin-left: -11px; + border-top-color: #999; + border-top-color: rgba(0, 0, 0, .25); + border-bottom-width: 0; +} +.popover.top > .arrow:after { + bottom: 1px; + margin-left: -10px; + content: " "; + border-top-color: #fff; + border-bottom-width: 0; +} +.popover.right > .arrow { + top: 50%; + left: -11px; + margin-top: -11px; + border-right-color: #999; + border-right-color: rgba(0, 0, 0, .25); + border-left-width: 0; +} +.popover.right > .arrow:after { + bottom: -10px; + left: 1px; + content: " "; + border-right-color: #fff; + border-left-width: 0; +} +.popover.bottom > .arrow { + top: -11px; + left: 50%; + margin-left: -11px; + border-top-width: 0; + border-bottom-color: #999; + border-bottom-color: rgba(0, 0, 0, .25); +} +.popover.bottom > .arrow:after { + top: 1px; + margin-left: -10px; + content: " "; + border-top-width: 0; + border-bottom-color: #fff; +} +.popover.left > .arrow { + top: 50%; + right: -11px; + margin-top: -11px; + border-right-width: 0; + border-left-color: #999; + border-left-color: rgba(0, 0, 0, .25); +} +.popover.left > .arrow:after { + right: 1px; + bottom: -10px; + content: " "; + border-right-width: 0; + border-left-color: #fff; +} +.carousel { + position: relative; +} +.carousel-inner { + position: relative; + width: 100%; + overflow: hidden; +} +.carousel-inner > .item { + position: relative; + display: none; + -webkit-transition: .6s ease-in-out left; + transition: .6s ease-in-out left; +} +.carousel-inner > .item > img, +.carousel-inner > .item > a > img { + line-height: 1; +} +.carousel-inner > .active, +.carousel-inner > .next, +.carousel-inner > .prev { + display: block; +} +.carousel-inner > .active { + left: 0; +} +.carousel-inner > .next, +.carousel-inner > .prev { + position: absolute; + top: 0; + width: 100%; +} +.carousel-inner > .next { + left: 100%; +} +.carousel-inner > .prev { + left: -100%; +} +.carousel-inner > .next.left, +.carousel-inner > .prev.right { + left: 0; +} +.carousel-inner > .active.left { + left: -100%; +} +.carousel-inner > .active.right { + left: 100%; +} +.carousel-control { + position: absolute; + top: 0; + bottom: 0; + left: 0; + width: 15%; + font-size: 20px; + color: #fff; + text-align: center; + text-shadow: 0 1px 2px rgba(0, 0, 0, .6); + filter: alpha(opacity=50); + opacity: .5; +} +.carousel-control.left { + background-image: -webkit-linear-gradient(left, color-stop(rgba(0, 0, 0, .5) 0%), color-stop(rgba(0, 0, 0, .0001) 100%)); + background-image: linear-gradient(to right, rgba(0, 0, 0, .5) 0%, rgba(0, 0, 0, .0001) 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#80000000', endColorstr='#00000000', GradientType=1); + background-repeat: repeat-x; +} +.carousel-control.right { + right: 0; + left: auto; + background-image: -webkit-linear-gradient(left, color-stop(rgba(0, 0, 0, .0001) 0%), color-stop(rgba(0, 0, 0, .5) 100%)); + background-image: linear-gradient(to right, rgba(0, 0, 0, .0001) 0%, rgba(0, 0, 0, .5) 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#00000000', endColorstr='#80000000', GradientType=1); + background-repeat: repeat-x; +} +.carousel-control:hover, +.carousel-control:focus { + color: #fff; + text-decoration: none; + filter: alpha(opacity=90); + outline: none; + opacity: .9; +} +.carousel-control .icon-prev, +.carousel-control .icon-next, +.carousel-control .glyphicon-chevron-left, +.carousel-control .glyphicon-chevron-right { + position: absolute; + top: 50%; + z-index: 5; + display: inline-block; +} +.carousel-control .icon-prev, +.carousel-control .glyphicon-chevron-left { + left: 50%; +} +.carousel-control .icon-next, +.carousel-control .glyphicon-chevron-right { + right: 50%; +} +.carousel-control .icon-prev, +.carousel-control .icon-next { + width: 20px; + height: 20px; + margin-top: -10px; + margin-left: -10px; + font-family: serif; +} +.carousel-control .icon-prev:before { + content: '\2039'; +} +.carousel-control .icon-next:before { + content: '\203a'; +} +.carousel-indicators { + position: absolute; + bottom: 10px; + left: 50%; + z-index: 15; + width: 60%; + padding-left: 0; + margin-left: -30%; + text-align: center; + list-style: none; +} +.carousel-indicators li { + display: inline-block; + width: 10px; + height: 10px; + margin: 1px; + text-indent: -999px; + cursor: pointer; + background-color: #000 \9; + background-color: rgba(0, 0, 0, 0); + border: 1px solid #fff; + border-radius: 10px; +} +.carousel-indicators .active { + width: 12px; + height: 12px; + margin: 0; + background-color: #fff; +} +.carousel-caption { + position: absolute; + right: 15%; + bottom: 20px; + left: 15%; + z-index: 10; + padding-top: 20px; + padding-bottom: 20px; + color: #fff; + text-align: center; + text-shadow: 0 1px 2px rgba(0, 0, 0, .6); +} +.carousel-caption .btn { + text-shadow: none; +} +@media screen and (min-width: 768px) { + .carousel-control .glyphicon-chevron-left, + .carousel-control .glyphicon-chevron-right, + .carousel-control .icon-prev, + .carousel-control .icon-next { + width: 30px; + height: 30px; + margin-top: -15px; + margin-left: -15px; + font-size: 30px; + } + .carousel-caption { + right: 20%; + left: 20%; + padding-bottom: 30px; + } + .carousel-indicators { + bottom: 20px; + } +} +.clearfix:before, +.clearfix:after, +.container:before, +.container:after, +.container-fluid:before, +.container-fluid:after, +.row:before, +.row:after, +.form-horizontal .form-group:before, +.form-horizontal .form-group:after, +.btn-toolbar:before, +.btn-toolbar:after, +.btn-group-vertical > .btn-group:before, +.btn-group-vertical > .btn-group:after, +.nav:before, +.nav:after, +.navbar:before, +.navbar:after, +.navbar-header:before, +.navbar-header:after, +.navbar-collapse:before, +.navbar-collapse:after, +.pager:before, +.pager:after, +.panel-body:before, +.panel-body:after, +.modal-footer:before, +.modal-footer:after { + display: table; + content: " "; +} +.clearfix:after, +.container:after, +.container-fluid:after, +.row:after, +.form-horizontal .form-group:after, +.btn-toolbar:after, +.btn-group-vertical > .btn-group:after, +.nav:after, +.navbar:after, +.navbar-header:after, +.navbar-collapse:after, +.pager:after, +.panel-body:after, +.modal-footer:after { + clear: both; +} +.center-block { + display: block; + margin-right: auto; + margin-left: auto; +} +.pull-right { + float: right !important; +} +.pull-left { + float: left !important; +} +.hide { + display: none !important; +} +.show { + display: block !important; +} +.invisible { + visibility: hidden; +} +.text-hide { + font: 0/0 a; + color: transparent; + text-shadow: none; + background-color: transparent; + border: 0; +} +.hidden { + display: none !important; + visibility: hidden !important; +} +.affix { + position: fixed; +} +@-ms-viewport { + width: device-width; +} +.visible-xs, +.visible-sm, +.visible-md, +.visible-lg { + display: none !important; +} +@media (max-width: 767px) { + .visible-xs { + display: block !important; + } + table.visible-xs { + display: table; + } + tr.visible-xs { + display: table-row !important; + } + th.visible-xs, + td.visible-xs { + display: table-cell !important; + } +} +@media (min-width: 768px) and (max-width: 991px) { + .visible-sm { + display: block !important; + } + table.visible-sm { + display: table; + } + tr.visible-sm { + display: table-row !important; + } + th.visible-sm, + td.visible-sm { + display: table-cell !important; + } +} +@media (min-width: 992px) and (max-width: 1199px) { + .visible-md { + display: block !important; + } + table.visible-md { + display: table; + } + tr.visible-md { + display: table-row !important; + } + th.visible-md, + td.visible-md { + display: table-cell !important; + } +} +@media (min-width: 1200px) { + .visible-lg { + display: block !important; + } + table.visible-lg { + display: table; + } + tr.visible-lg { + display: table-row !important; + } + th.visible-lg, + td.visible-lg { + display: table-cell !important; + } +} +@media (max-width: 767px) { + .hidden-xs { + display: none !important; + } +} +@media (min-width: 768px) and (max-width: 991px) { + .hidden-sm { + display: none !important; + } +} +@media (min-width: 992px) and (max-width: 1199px) { + .hidden-md { + display: none !important; + } +} +@media (min-width: 1200px) { + .hidden-lg { + display: none !important; + } +} +.visible-print { + display: none !important; +} +@media print { + .visible-print { + display: block !important; + } + table.visible-print { + display: table; + } + tr.visible-print { + display: table-row !important; + } + th.visible-print, + td.visible-print { + display: table-cell !important; + } +} +@media print { + .hidden-print { + display: none !important; + } +} +/*# sourceMappingURL=bootstrap.css.map */ diff --git a/server/src/main/resources/static/css/font-awesome.css b/server/src/main/resources/static/css/font-awesome.css new file mode 100644 index 00000000000..048cff97398 --- /dev/null +++ b/server/src/main/resources/static/css/font-awesome.css @@ -0,0 +1,1338 @@ +/*! + * Font Awesome 4.0.3 by @davegandy - http://fontawesome.io - @fontawesome + * License - http://fontawesome.io/license (Font: SIL OFL 1.1, CSS: MIT License) + */ +/* FONT PATH + * -------------------------- */ +@font-face { + font-family: 'FontAwesome'; + src: url('../fonts/fontawesome-webfont.eot?v=4.0.3'); + src: url('../fonts/fontawesome-webfont.eot?#iefix&v=4.0.3') format('embedded-opentype'), url('../fonts/fontawesome-webfont.woff?v=4.0.3') format('woff'), url('../fonts/fontawesome-webfont.ttf?v=4.0.3') format('truetype'), url('../fonts/fontawesome-webfont.svg?v=4.0.3#fontawesomeregular') format('svg'); + font-weight: normal; + font-style: normal; +} +.fa { + display: inline-block; + font-family: FontAwesome; + font-style: normal; + font-weight: normal; + line-height: 1; + -webkit-font-smoothing: antialiased; + -moz-osx-font-smoothing: grayscale; +} +/* makes the font 33% larger relative to the icon container */ +.fa-lg { + font-size: 1.3333333333333333em; + line-height: 0.75em; + vertical-align: -15%; +} +.fa-2x { + font-size: 2em; +} +.fa-3x { + font-size: 3em; +} +.fa-4x { + font-size: 4em; +} +.fa-5x { + font-size: 5em; +} +.fa-fw { + width: 1.2857142857142858em; + text-align: center; +} +.fa-ul { + padding-left: 0; + margin-left: 2.142857142857143em; + list-style-type: none; +} +.fa-ul > li { + position: relative; +} +.fa-li { + position: absolute; + left: -2.142857142857143em; + width: 2.142857142857143em; + top: 0.14285714285714285em; + text-align: center; +} +.fa-li.fa-lg { + left: -1.8571428571428572em; +} +.fa-border { + padding: .2em .25em .15em; + border: solid 0.08em #eeeeee; + border-radius: .1em; +} +.pull-right { + float: right; +} +.pull-left { + float: left; +} +.fa.pull-left { + margin-right: .3em; +} +.fa.pull-right { + margin-left: .3em; +} +.fa-spin { + -webkit-animation: spin 2s infinite linear; + -moz-animation: spin 2s infinite linear; + -o-animation: spin 2s infinite linear; + animation: spin 2s infinite linear; +} +@-moz-keyframes spin { + 0% { + -moz-transform: rotate(0deg); + } + 100% { + -moz-transform: rotate(359deg); + } +} +@-webkit-keyframes spin { + 0% { + -webkit-transform: rotate(0deg); + } + 100% { + -webkit-transform: rotate(359deg); + } +} +@-o-keyframes spin { + 0% { + -o-transform: rotate(0deg); + } + 100% { + -o-transform: rotate(359deg); + } +} +@-ms-keyframes spin { + 0% { + -ms-transform: rotate(0deg); + } + 100% { + -ms-transform: rotate(359deg); + } +} +@keyframes spin { + 0% { + transform: rotate(0deg); + } + 100% { + transform: rotate(359deg); + } +} +.fa-rotate-90 { + filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=1); + -webkit-transform: rotate(90deg); + -moz-transform: rotate(90deg); + -ms-transform: rotate(90deg); + -o-transform: rotate(90deg); + transform: rotate(90deg); +} +.fa-rotate-180 { + filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=2); + -webkit-transform: rotate(180deg); + -moz-transform: rotate(180deg); + -ms-transform: rotate(180deg); + -o-transform: rotate(180deg); + transform: rotate(180deg); +} +.fa-rotate-270 { + filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=3); + -webkit-transform: rotate(270deg); + -moz-transform: rotate(270deg); + -ms-transform: rotate(270deg); + -o-transform: rotate(270deg); + transform: rotate(270deg); +} +.fa-flip-horizontal { + filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=0, mirror=1); + -webkit-transform: scale(-1, 1); + -moz-transform: scale(-1, 1); + -ms-transform: scale(-1, 1); + -o-transform: scale(-1, 1); + transform: scale(-1, 1); +} +.fa-flip-vertical { + filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=2, mirror=1); + -webkit-transform: scale(1, -1); + -moz-transform: scale(1, -1); + -ms-transform: scale(1, -1); + -o-transform: scale(1, -1); + transform: scale(1, -1); +} +.fa-stack { + position: relative; + display: inline-block; + width: 2em; + height: 2em; + line-height: 2em; + vertical-align: middle; +} +.fa-stack-1x, +.fa-stack-2x { + position: absolute; + left: 0; + width: 100%; + text-align: center; +} +.fa-stack-1x { + line-height: inherit; +} +.fa-stack-2x { + font-size: 2em; +} +.fa-inverse { + color: #ffffff; +} +/* Font Awesome uses the Unicode Private Use Area (PUA) to ensure screen + readers do not read off random characters that represent icons */ +.fa-glass:before { + content: "\f000"; +} +.fa-music:before { + content: "\f001"; +} +.fa-search:before { + content: "\f002"; +} +.fa-envelope-o:before { + content: "\f003"; +} +.fa-heart:before { + content: "\f004"; +} +.fa-star:before { + content: "\f005"; +} +.fa-star-o:before { + content: "\f006"; +} +.fa-user:before { + content: "\f007"; +} +.fa-film:before { + content: "\f008"; +} +.fa-th-large:before { + content: "\f009"; +} +.fa-th:before { + content: "\f00a"; +} +.fa-th-list:before { + content: "\f00b"; +} +.fa-check:before { + content: "\f00c"; +} +.fa-times:before { + content: "\f00d"; +} +.fa-search-plus:before { + content: "\f00e"; +} +.fa-search-minus:before { + content: "\f010"; +} +.fa-power-off:before { + content: "\f011"; +} +.fa-signal:before { + content: "\f012"; +} +.fa-gear:before, +.fa-cog:before { + content: "\f013"; +} +.fa-trash-o:before { + content: "\f014"; +} +.fa-home:before { + content: "\f015"; +} +.fa-file-o:before { + content: "\f016"; +} +.fa-clock-o:before { + content: "\f017"; +} +.fa-road:before { + content: "\f018"; +} +.fa-download:before { + content: "\f019"; +} +.fa-arrow-circle-o-down:before { + content: "\f01a"; +} +.fa-arrow-circle-o-up:before { + content: "\f01b"; +} +.fa-inbox:before { + content: "\f01c"; +} +.fa-play-circle-o:before { + content: "\f01d"; +} +.fa-rotate-right:before, +.fa-repeat:before { + content: "\f01e"; +} +.fa-refresh:before { + content: "\f021"; +} +.fa-list-alt:before { + content: "\f022"; +} +.fa-lock:before { + content: "\f023"; +} +.fa-flag:before { + content: "\f024"; +} +.fa-headphones:before { + content: "\f025"; +} +.fa-volume-off:before { + content: "\f026"; +} +.fa-volume-down:before { + content: "\f027"; +} +.fa-volume-up:before { + content: "\f028"; +} +.fa-qrcode:before { + content: "\f029"; +} +.fa-barcode:before { + content: "\f02a"; +} +.fa-tag:before { + content: "\f02b"; +} +.fa-tags:before { + content: "\f02c"; +} +.fa-book:before { + content: "\f02d"; +} +.fa-bookmark:before { + content: "\f02e"; +} +.fa-print:before { + content: "\f02f"; +} +.fa-camera:before { + content: "\f030"; +} +.fa-font:before { + content: "\f031"; +} +.fa-bold:before { + content: "\f032"; +} +.fa-italic:before { + content: "\f033"; +} +.fa-text-height:before { + content: "\f034"; +} +.fa-text-width:before { + content: "\f035"; +} +.fa-align-left:before { + content: "\f036"; +} +.fa-align-center:before { + content: "\f037"; +} +.fa-align-right:before { + content: "\f038"; +} +.fa-align-justify:before { + content: "\f039"; +} +.fa-list:before { + content: "\f03a"; +} +.fa-dedent:before, +.fa-outdent:before { + content: "\f03b"; +} +.fa-indent:before { + content: "\f03c"; +} +.fa-video-camera:before { + content: "\f03d"; +} +.fa-picture-o:before { + content: "\f03e"; +} +.fa-pencil:before { + content: "\f040"; +} +.fa-map-marker:before { + content: "\f041"; +} +.fa-adjust:before { + content: "\f042"; +} +.fa-tint:before { + content: "\f043"; +} +.fa-edit:before, +.fa-pencil-square-o:before { + content: "\f044"; +} +.fa-share-square-o:before { + content: "\f045"; +} +.fa-check-square-o:before { + content: "\f046"; +} +.fa-arrows:before { + content: "\f047"; +} +.fa-step-backward:before { + content: "\f048"; +} +.fa-fast-backward:before { + content: "\f049"; +} +.fa-backward:before { + content: "\f04a"; +} +.fa-play:before { + content: "\f04b"; +} +.fa-pause:before { + content: "\f04c"; +} +.fa-stop:before { + content: "\f04d"; +} +.fa-forward:before { + content: "\f04e"; +} +.fa-fast-forward:before { + content: "\f050"; +} +.fa-step-forward:before { + content: "\f051"; +} +.fa-eject:before { + content: "\f052"; +} +.fa-chevron-left:before { + content: "\f053"; +} +.fa-chevron-right:before { + content: "\f054"; +} +.fa-plus-circle:before { + content: "\f055"; +} +.fa-minus-circle:before { + content: "\f056"; +} +.fa-times-circle:before { + content: "\f057"; +} +.fa-check-circle:before { + content: "\f058"; +} +.fa-question-circle:before { + content: "\f059"; +} +.fa-info-circle:before { + content: "\f05a"; +} +.fa-crosshairs:before { + content: "\f05b"; +} +.fa-times-circle-o:before { + content: "\f05c"; +} +.fa-check-circle-o:before { + content: "\f05d"; +} +.fa-ban:before { + content: "\f05e"; +} +.fa-arrow-left:before { + content: "\f060"; +} +.fa-arrow-right:before { + content: "\f061"; +} +.fa-arrow-up:before { + content: "\f062"; +} +.fa-arrow-down:before { + content: "\f063"; +} +.fa-mail-forward:before, +.fa-share:before { + content: "\f064"; +} +.fa-expand:before { + content: "\f065"; +} +.fa-compress:before { + content: "\f066"; +} +.fa-plus:before { + content: "\f067"; +} +.fa-minus:before { + content: "\f068"; +} +.fa-asterisk:before { + content: "\f069"; +} +.fa-exclamation-circle:before { + content: "\f06a"; +} +.fa-gift:before { + content: "\f06b"; +} +.fa-leaf:before { + content: "\f06c"; +} +.fa-fire:before { + content: "\f06d"; +} +.fa-eye:before { + content: "\f06e"; +} +.fa-eye-slash:before { + content: "\f070"; +} +.fa-warning:before, +.fa-exclamation-triangle:before { + content: "\f071"; +} +.fa-plane:before { + content: "\f072"; +} +.fa-calendar:before { + content: "\f073"; +} +.fa-random:before { + content: "\f074"; +} +.fa-comment:before { + content: "\f075"; +} +.fa-magnet:before { + content: "\f076"; +} +.fa-chevron-up:before { + content: "\f077"; +} +.fa-chevron-down:before { + content: "\f078"; +} +.fa-retweet:before { + content: "\f079"; +} +.fa-shopping-cart:before { + content: "\f07a"; +} +.fa-folder:before { + content: "\f07b"; +} +.fa-folder-open:before { + content: "\f07c"; +} +.fa-arrows-v:before { + content: "\f07d"; +} +.fa-arrows-h:before { + content: "\f07e"; +} +.fa-bar-chart-o:before { + content: "\f080"; +} +.fa-twitter-square:before { + content: "\f081"; +} +.fa-facebook-square:before { + content: "\f082"; +} +.fa-camera-retro:before { + content: "\f083"; +} +.fa-key:before { + content: "\f084"; +} +.fa-gears:before, +.fa-cogs:before { + content: "\f085"; +} +.fa-comments:before { + content: "\f086"; +} +.fa-thumbs-o-up:before { + content: "\f087"; +} +.fa-thumbs-o-down:before { + content: "\f088"; +} +.fa-star-half:before { + content: "\f089"; +} +.fa-heart-o:before { + content: "\f08a"; +} +.fa-sign-out:before { + content: "\f08b"; +} +.fa-linkedin-square:before { + content: "\f08c"; +} +.fa-thumb-tack:before { + content: "\f08d"; +} +.fa-external-link:before { + content: "\f08e"; +} +.fa-sign-in:before { + content: "\f090"; +} +.fa-trophy:before { + content: "\f091"; +} +.fa-github-square:before { + content: "\f092"; +} +.fa-upload:before { + content: "\f093"; +} +.fa-lemon-o:before { + content: "\f094"; +} +.fa-phone:before { + content: "\f095"; +} +.fa-square-o:before { + content: "\f096"; +} +.fa-bookmark-o:before { + content: "\f097"; +} +.fa-phone-square:before { + content: "\f098"; +} +.fa-twitter:before { + content: "\f099"; +} +.fa-facebook:before { + content: "\f09a"; +} +.fa-github:before { + content: "\f09b"; +} +.fa-unlock:before { + content: "\f09c"; +} +.fa-credit-card:before { + content: "\f09d"; +} +.fa-rss:before { + content: "\f09e"; +} +.fa-hdd-o:before { + content: "\f0a0"; +} +.fa-bullhorn:before { + content: "\f0a1"; +} +.fa-bell:before { + content: "\f0f3"; +} +.fa-certificate:before { + content: "\f0a3"; +} +.fa-hand-o-right:before { + content: "\f0a4"; +} +.fa-hand-o-left:before { + content: "\f0a5"; +} +.fa-hand-o-up:before { + content: "\f0a6"; +} +.fa-hand-o-down:before { + content: "\f0a7"; +} +.fa-arrow-circle-left:before { + content: "\f0a8"; +} +.fa-arrow-circle-right:before { + content: "\f0a9"; +} +.fa-arrow-circle-up:before { + content: "\f0aa"; +} +.fa-arrow-circle-down:before { + content: "\f0ab"; +} +.fa-globe:before { + content: "\f0ac"; +} +.fa-wrench:before { + content: "\f0ad"; +} +.fa-tasks:before { + content: "\f0ae"; +} +.fa-filter:before { + content: "\f0b0"; +} +.fa-briefcase:before { + content: "\f0b1"; +} +.fa-arrows-alt:before { + content: "\f0b2"; +} +.fa-group:before, +.fa-users:before { + content: "\f0c0"; +} +.fa-chain:before, +.fa-link:before { + content: "\f0c1"; +} +.fa-cloud:before { + content: "\f0c2"; +} +.fa-flask:before { + content: "\f0c3"; +} +.fa-cut:before, +.fa-scissors:before { + content: "\f0c4"; +} +.fa-copy:before, +.fa-files-o:before { + content: "\f0c5"; +} +.fa-paperclip:before { + content: "\f0c6"; +} +.fa-save:before, +.fa-floppy-o:before { + content: "\f0c7"; +} +.fa-square:before { + content: "\f0c8"; +} +.fa-bars:before { + content: "\f0c9"; +} +.fa-list-ul:before { + content: "\f0ca"; +} +.fa-list-ol:before { + content: "\f0cb"; +} +.fa-strikethrough:before { + content: "\f0cc"; +} +.fa-underline:before { + content: "\f0cd"; +} +.fa-table:before { + content: "\f0ce"; +} +.fa-magic:before { + content: "\f0d0"; +} +.fa-truck:before { + content: "\f0d1"; +} +.fa-pinterest:before { + content: "\f0d2"; +} +.fa-pinterest-square:before { + content: "\f0d3"; +} +.fa-google-plus-square:before { + content: "\f0d4"; +} +.fa-google-plus:before { + content: "\f0d5"; +} +.fa-money:before { + content: "\f0d6"; +} +.fa-caret-down:before { + content: "\f0d7"; +} +.fa-caret-up:before { + content: "\f0d8"; +} +.fa-caret-left:before { + content: "\f0d9"; +} +.fa-caret-right:before { + content: "\f0da"; +} +.fa-columns:before { + content: "\f0db"; +} +.fa-unsorted:before, +.fa-sort:before { + content: "\f0dc"; +} +.fa-sort-down:before, +.fa-sort-asc:before { + content: "\f0dd"; +} +.fa-sort-up:before, +.fa-sort-desc:before { + content: "\f0de"; +} +.fa-envelope:before { + content: "\f0e0"; +} +.fa-linkedin:before { + content: "\f0e1"; +} +.fa-rotate-left:before, +.fa-undo:before { + content: "\f0e2"; +} +.fa-legal:before, +.fa-gavel:before { + content: "\f0e3"; +} +.fa-dashboard:before, +.fa-tachometer:before { + content: "\f0e4"; +} +.fa-comment-o:before { + content: "\f0e5"; +} +.fa-comments-o:before { + content: "\f0e6"; +} +.fa-flash:before, +.fa-bolt:before { + content: "\f0e7"; +} +.fa-sitemap:before { + content: "\f0e8"; +} +.fa-umbrella:before { + content: "\f0e9"; +} +.fa-paste:before, +.fa-clipboard:before { + content: "\f0ea"; +} +.fa-lightbulb-o:before { + content: "\f0eb"; +} +.fa-exchange:before { + content: "\f0ec"; +} +.fa-cloud-download:before { + content: "\f0ed"; +} +.fa-cloud-upload:before { + content: "\f0ee"; +} +.fa-user-md:before { + content: "\f0f0"; +} +.fa-stethoscope:before { + content: "\f0f1"; +} +.fa-suitcase:before { + content: "\f0f2"; +} +.fa-bell-o:before { + content: "\f0a2"; +} +.fa-coffee:before { + content: "\f0f4"; +} +.fa-cutlery:before { + content: "\f0f5"; +} +.fa-file-text-o:before { + content: "\f0f6"; +} +.fa-building-o:before { + content: "\f0f7"; +} +.fa-hospital-o:before { + content: "\f0f8"; +} +.fa-ambulance:before { + content: "\f0f9"; +} +.fa-medkit:before { + content: "\f0fa"; +} +.fa-fighter-jet:before { + content: "\f0fb"; +} +.fa-beer:before { + content: "\f0fc"; +} +.fa-h-square:before { + content: "\f0fd"; +} +.fa-plus-square:before { + content: "\f0fe"; +} +.fa-angle-double-left:before { + content: "\f100"; +} +.fa-angle-double-right:before { + content: "\f101"; +} +.fa-angle-double-up:before { + content: "\f102"; +} +.fa-angle-double-down:before { + content: "\f103"; +} +.fa-angle-left:before { + content: "\f104"; +} +.fa-angle-right:before { + content: "\f105"; +} +.fa-angle-up:before { + content: "\f106"; +} +.fa-angle-down:before { + content: "\f107"; +} +.fa-desktop:before { + content: "\f108"; +} +.fa-laptop:before { + content: "\f109"; +} +.fa-tablet:before { + content: "\f10a"; +} +.fa-mobile-phone:before, +.fa-mobile:before { + content: "\f10b"; +} +.fa-circle-o:before { + content: "\f10c"; +} +.fa-quote-left:before { + content: "\f10d"; +} +.fa-quote-right:before { + content: "\f10e"; +} +.fa-spinner:before { + content: "\f110"; +} +.fa-circle:before { + content: "\f111"; +} +.fa-mail-reply:before, +.fa-reply:before { + content: "\f112"; +} +.fa-github-alt:before { + content: "\f113"; +} +.fa-folder-o:before { + content: "\f114"; +} +.fa-folder-open-o:before { + content: "\f115"; +} +.fa-smile-o:before { + content: "\f118"; +} +.fa-frown-o:before { + content: "\f119"; +} +.fa-meh-o:before { + content: "\f11a"; +} +.fa-gamepad:before { + content: "\f11b"; +} +.fa-keyboard-o:before { + content: "\f11c"; +} +.fa-flag-o:before { + content: "\f11d"; +} +.fa-flag-checkered:before { + content: "\f11e"; +} +.fa-terminal:before { + content: "\f120"; +} +.fa-code:before { + content: "\f121"; +} +.fa-reply-all:before { + content: "\f122"; +} +.fa-mail-reply-all:before { + content: "\f122"; +} +.fa-star-half-empty:before, +.fa-star-half-full:before, +.fa-star-half-o:before { + content: "\f123"; +} +.fa-location-arrow:before { + content: "\f124"; +} +.fa-crop:before { + content: "\f125"; +} +.fa-code-fork:before { + content: "\f126"; +} +.fa-unlink:before, +.fa-chain-broken:before { + content: "\f127"; +} +.fa-question:before { + content: "\f128"; +} +.fa-info:before { + content: "\f129"; +} +.fa-exclamation:before { + content: "\f12a"; +} +.fa-superscript:before { + content: "\f12b"; +} +.fa-subscript:before { + content: "\f12c"; +} +.fa-eraser:before { + content: "\f12d"; +} +.fa-puzzle-piece:before { + content: "\f12e"; +} +.fa-microphone:before { + content: "\f130"; +} +.fa-microphone-slash:before { + content: "\f131"; +} +.fa-shield:before { + content: "\f132"; +} +.fa-calendar-o:before { + content: "\f133"; +} +.fa-fire-extinguisher:before { + content: "\f134"; +} +.fa-rocket:before { + content: "\f135"; +} +.fa-maxcdn:before { + content: "\f136"; +} +.fa-chevron-circle-left:before { + content: "\f137"; +} +.fa-chevron-circle-right:before { + content: "\f138"; +} +.fa-chevron-circle-up:before { + content: "\f139"; +} +.fa-chevron-circle-down:before { + content: "\f13a"; +} +.fa-html5:before { + content: "\f13b"; +} +.fa-css3:before { + content: "\f13c"; +} +.fa-anchor:before { + content: "\f13d"; +} +.fa-unlock-alt:before { + content: "\f13e"; +} +.fa-bullseye:before { + content: "\f140"; +} +.fa-ellipsis-h:before { + content: "\f141"; +} +.fa-ellipsis-v:before { + content: "\f142"; +} +.fa-rss-square:before { + content: "\f143"; +} +.fa-play-circle:before { + content: "\f144"; +} +.fa-ticket:before { + content: "\f145"; +} +.fa-minus-square:before { + content: "\f146"; +} +.fa-minus-square-o:before { + content: "\f147"; +} +.fa-level-up:before { + content: "\f148"; +} +.fa-level-down:before { + content: "\f149"; +} +.fa-check-square:before { + content: "\f14a"; +} +.fa-pencil-square:before { + content: "\f14b"; +} +.fa-external-link-square:before { + content: "\f14c"; +} +.fa-share-square:before { + content: "\f14d"; +} +.fa-compass:before { + content: "\f14e"; +} +.fa-toggle-down:before, +.fa-caret-square-o-down:before { + content: "\f150"; +} +.fa-toggle-up:before, +.fa-caret-square-o-up:before { + content: "\f151"; +} +.fa-toggle-right:before, +.fa-caret-square-o-right:before { + content: "\f152"; +} +.fa-euro:before, +.fa-eur:before { + content: "\f153"; +} +.fa-gbp:before { + content: "\f154"; +} +.fa-dollar:before, +.fa-usd:before { + content: "\f155"; +} +.fa-rupee:before, +.fa-inr:before { + content: "\f156"; +} +.fa-cny:before, +.fa-rmb:before, +.fa-yen:before, +.fa-jpy:before { + content: "\f157"; +} +.fa-ruble:before, +.fa-rouble:before, +.fa-rub:before { + content: "\f158"; +} +.fa-won:before, +.fa-krw:before { + content: "\f159"; +} +.fa-bitcoin:before, +.fa-btc:before { + content: "\f15a"; +} +.fa-file:before { + content: "\f15b"; +} +.fa-file-text:before { + content: "\f15c"; +} +.fa-sort-alpha-asc:before { + content: "\f15d"; +} +.fa-sort-alpha-desc:before { + content: "\f15e"; +} +.fa-sort-amount-asc:before { + content: "\f160"; +} +.fa-sort-amount-desc:before { + content: "\f161"; +} +.fa-sort-numeric-asc:before { + content: "\f162"; +} +.fa-sort-numeric-desc:before { + content: "\f163"; +} +.fa-thumbs-up:before { + content: "\f164"; +} +.fa-thumbs-down:before { + content: "\f165"; +} +.fa-youtube-square:before { + content: "\f166"; +} +.fa-youtube:before { + content: "\f167"; +} +.fa-xing:before { + content: "\f168"; +} +.fa-xing-square:before { + content: "\f169"; +} +.fa-youtube-play:before { + content: "\f16a"; +} +.fa-dropbox:before { + content: "\f16b"; +} +.fa-stack-overflow:before { + content: "\f16c"; +} +.fa-instagram:before { + content: "\f16d"; +} +.fa-flickr:before { + content: "\f16e"; +} +.fa-adn:before { + content: "\f170"; +} +.fa-bitbucket:before { + content: "\f171"; +} +.fa-bitbucket-square:before { + content: "\f172"; +} +.fa-tumblr:before { + content: "\f173"; +} +.fa-tumblr-square:before { + content: "\f174"; +} +.fa-long-arrow-down:before { + content: "\f175"; +} +.fa-long-arrow-up:before { + content: "\f176"; +} +.fa-long-arrow-left:before { + content: "\f177"; +} +.fa-long-arrow-right:before { + content: "\f178"; +} +.fa-apple:before { + content: "\f179"; +} +.fa-windows:before { + content: "\f17a"; +} +.fa-android:before { + content: "\f17b"; +} +.fa-linux:before { + content: "\f17c"; +} +.fa-dribbble:before { + content: "\f17d"; +} +.fa-skype:before { + content: "\f17e"; +} +.fa-foursquare:before { + content: "\f180"; +} +.fa-trello:before { + content: "\f181"; +} +.fa-female:before { + content: "\f182"; +} +.fa-male:before { + content: "\f183"; +} +.fa-gittip:before { + content: "\f184"; +} +.fa-sun-o:before { + content: "\f185"; +} +.fa-moon-o:before { + content: "\f186"; +} +.fa-archive:before { + content: "\f187"; +} +.fa-bug:before { + content: "\f188"; +} +.fa-vk:before { + content: "\f189"; +} +.fa-weibo:before { + content: "\f18a"; +} +.fa-renren:before { + content: "\f18b"; +} +.fa-pagelines:before { + content: "\f18c"; +} +.fa-stack-exchange:before { + content: "\f18d"; +} +.fa-arrow-circle-o-right:before { + content: "\f18e"; +} +.fa-arrow-circle-o-left:before { + content: "\f190"; +} +.fa-toggle-left:before, +.fa-caret-square-o-left:before { + content: "\f191"; +} +.fa-dot-circle-o:before { + content: "\f192"; +} +.fa-wheelchair:before { + content: "\f193"; +} +.fa-vimeo-square:before { + content: "\f194"; +} +.fa-turkish-lira:before, +.fa-try:before { + content: "\f195"; +} +.fa-plus-square-o:before { + content: "\f196"; +} diff --git a/server/src/main/resources/static/druid.css b/server/src/main/resources/static/druid.css new file mode 100644 index 00000000000..91a368174aa --- /dev/null +++ b/server/src/main/resources/static/druid.css @@ -0,0 +1,985 @@ +@import url(//fonts.googleapis.com/css?family=Roboto:400,700,400italic,500); +/* line 17, ../../../../../../usr/local/Cellar/ruby/2.0.0-p195/lib/ruby/gems/2.0.0/gems/compass-0.12.2/frameworks/compass/stylesheets/compass/reset/_utilities.scss */ +html, body, div, span, applet, object, iframe, +h1, h2, h3, h4, h5, h6, p, blockquote, pre, +a, abbr, acronym, address, big, cite, code, +del, dfn, em, img, ins, kbd, q, s, samp, +small, strike, strong, sub, sup, tt, var, +b, u, i, center, +dl, dt, dd, ol, ul, li, +fieldset, form, label, legend, +table, caption, tbody, tfoot, thead, tr, th, td, +article, aside, canvas, details, embed, +figure, figcaption, footer, header, hgroup, +menu, nav, output, ruby, section, summary, +time, mark, audio, video { + margin: 0; + padding: 0; + border: 0; + font: inherit; + font-size: 100%; + vertical-align: baseline; +} + +/* line 22, ../../../../../../usr/local/Cellar/ruby/2.0.0-p195/lib/ruby/gems/2.0.0/gems/compass-0.12.2/frameworks/compass/stylesheets/compass/reset/_utilities.scss */ +html { + line-height: 1; +} + +/* line 24, ../../../../../../usr/local/Cellar/ruby/2.0.0-p195/lib/ruby/gems/2.0.0/gems/compass-0.12.2/frameworks/compass/stylesheets/compass/reset/_utilities.scss */ +ol, ul { + list-style: none; +} + +/* line 26, ../../../../../../usr/local/Cellar/ruby/2.0.0-p195/lib/ruby/gems/2.0.0/gems/compass-0.12.2/frameworks/compass/stylesheets/compass/reset/_utilities.scss */ +table { + border-collapse: collapse; + border-spacing: 0; +} + +/* line 28, ../../../../../../usr/local/Cellar/ruby/2.0.0-p195/lib/ruby/gems/2.0.0/gems/compass-0.12.2/frameworks/compass/stylesheets/compass/reset/_utilities.scss */ +caption, th, td { + text-align: left; + font-weight: normal; + vertical-align: middle; +} + +/* line 30, ../../../../../../usr/local/Cellar/ruby/2.0.0-p195/lib/ruby/gems/2.0.0/gems/compass-0.12.2/frameworks/compass/stylesheets/compass/reset/_utilities.scss */ +q, blockquote { + quotes: none; +} +/* line 103, ../../../../../../usr/local/Cellar/ruby/2.0.0-p195/lib/ruby/gems/2.0.0/gems/compass-0.12.2/frameworks/compass/stylesheets/compass/reset/_utilities.scss */ +q:before, q:after, blockquote:before, blockquote:after { + content: ""; + content: none; +} + +/* line 32, ../../../../../../usr/local/Cellar/ruby/2.0.0-p195/lib/ruby/gems/2.0.0/gems/compass-0.12.2/frameworks/compass/stylesheets/compass/reset/_utilities.scss */ +a img { + border: none; +} + +/* line 116, ../../../../../../usr/local/Cellar/ruby/2.0.0-p195/lib/ruby/gems/2.0.0/gems/compass-0.12.2/frameworks/compass/stylesheets/compass/reset/_utilities.scss */ +article, aside, details, figcaption, figure, footer, header, hgroup, menu, nav, section, summary { + display: block; +} + +/* line 37, ../src/client/druid.scss */ +html { + font-family: Roboto,Helvetica,Arial,freesans,clean,sans-serif; + width: 100%; + height: 100%; + font-size: 13px; + cursor: default; + background: #22222A; +} +/* line 46, ../src/client/druid.scss */ +html a, html a:link, html a:visited { + text-decoration: none; + color: #d8d8dd; +} +/* line 50, ../src/client/druid.scss */ +html a:hover { + color: #fff; + text-decoration: underline; +} + +/* line 56, ../src/client/druid.scss */ +body { + color: #858597; + background: #22222a; + height: 100%; +} +/* line 61, ../src/client/druid.scss */ +body header { + min-height: 40px; + max-height: 40px; + padding-top: 4px; + width: 100%; + flex: 1; + -webkit-flex: 1; + display: flex; + display: -webkit-flex; + flex-direction: row; + -webkit-flex-flow: row nowrap; + justify-content: flex-start; + -webkit-justify-content: flex-start; + align-items: baseline; +} +/* line 77, ../src/client/druid.scss */ +body header > * { + align-self: flex-start; +} +/* line 80, ../src/client/druid.scss */ +body header h1 { + padding-left: 8px; + min-width: 239px; + color: #fefefe; + font-size: 24px; + font-size: 24px; + line-height: 1.5; + color: #858597; + border-bottom: none; +} +/* line 89, ../src/client/druid.scss */ +body header h1 span.data-source-name { + color: #fefefe; +} +/* line 92, ../src/client/druid.scss */ +body header h1 a:link, body header h1 a:visited { + color: #fefefe; +} +/* line 95, ../src/client/druid.scss */ +body header h1 a:hover { + color: #8968ad; +} +/* line 98, ../src/client/druid.scss */ +body header h1 .summary { + font-size: 14px; +} +/* line 103, ../src/client/druid.scss */ +body header site-nav { + align-items: baseline; + display: flex; + display: -webkit-flex; + flex-direction: row; + -webkit-flex-flow: row nowrap; +} +/* line 109, ../src/client/druid.scss */ +body header site-nav > * { + align-self: flex-start; +} +/* line 112, ../src/client/druid.scss */ +body header site-nav .nav { + font-size: 16px; + width: 239px; + min-width: 239px; + max-width: 239px; +} +/* line 117, ../src/client/druid.scss */ +body header site-nav .nav li a { + color: #d8d8dd; + padding: 8px 15px; +} +/* line 120, ../src/client/druid.scss */ +body header site-nav .nav li a:hover { + color: #fefefe; + background: #858597; +} +/* line 126, ../src/client/druid.scss */ +body header site-nav .nav li.active a { + background: #3f6e94; +} +/* line 128, ../src/client/druid.scss */ +body header site-nav .nav li.active a:hover { + background: #3f6e94; + color: #22222a; +} +/* line 135, ../src/client/druid.scss */ +body header site-nav div.og-console-links { + line-height: 1.3; + border-left: 1px solid #1b1a20; + padding: 0px 4px; +} +/* line 139, ../src/client/druid.scss */ +body header site-nav div.og-console-links a { + /* unvisited link */ + color: #9a7795; + /* visited link */ + /* mouse over link */ + /* selected link */ +} +/* line 141, ../src/client/druid.scss */ +body header site-nav div.og-console-links a:visited { + color: #9a7795; +} +/* line 142, ../src/client/druid.scss */ +body header site-nav div.og-console-links a:hover { + color: #b25fa5; +} +/* line 143, ../src/client/druid.scss */ +body header site-nav div.og-console-links a:active { + color: #b25fa5; +} +/* line 149, ../src/client/druid.scss */ +body .primary-view .breadcrumb { + font-size: 18px; + padding: 8px; + margin: 0px; + background: #1b1a20; +} +/* line 154, ../src/client/druid.scss */ +body .primary-view .breadcrumb > li + li:before { + color: #d8d8dd; +} +/* line 157, ../src/client/druid.scss */ +body .primary-view .breadcrumb a, body .primary-view .breadcrumb a:visited { + color: #858597; +} +/* line 160, ../src/client/druid.scss */ +body .primary-view .breadcrumb > .active { + color: #fefefe; +} +/* line 165, ../src/client/druid.scss */ +body .cluster-view, body .data-sources-view, body .data-source-view { + display: flex; + display: -webkit-flex; + height: 100vh; + width: 100vw; + flex-direction: column; + -webkit-flex-flow: column nowrap; +} +/* line 173, ../src/client/druid.scss */ +body .cluster-view h1, body .cluster-view h2, body .data-sources-view h1, body .data-sources-view h2, body .data-source-view h1, body .data-source-view h2 { + color: #858597; + background: #2E2E38; + line-height: 1.5; + border-bottom: 1px solid #1b1a20; + padding: 0px 8px; +} +/* line 181, ../src/client/druid.scss */ +body .cluster-view #middle, body .data-sources-view #middle, body .data-source-view #middle { + flex: 1; + -webkit-flex: 1; + width: 100vw; + height: 100%; + display: flex; + display: -webkit-flex; + flex-direction: row; + -webkit-flex-flow: row nowrap; + border-top: 1px solid #1b1a20; +} +/* line 194, ../src/client/druid.scss */ +body .tier.hot { + color: #f16065; +} +/* line 197, ../src/client/druid.scss */ +body .tier._default_tier { + color: #5ca0d7; +} +/* line 202, ../src/client/druid.scss */ +body .data-source-view .rules, body .data-source-view .timeline { + border-top: 1px solid #1b1a20; + display: flex; + display: -webkit-flex; + justify-content: flex-start; + -webkit-justify-content: flex-start; +} +/* line 210, ../src/client/druid.scss */ +body .data-source-view .rules h2, body .data-source-view .timeline h2 { + font-size: 18px; + border-bottom: none; + border-right: 1px solid #1b1a20; + min-width: 240px; + max-width: 240px; + width: 240px; +} +/* line 217, ../src/client/druid.scss */ +body .data-source-view .rules h2 > div, body .data-source-view .timeline h2 > div { + margin-left: 1em; + font-size: 14px; +} +/* line 224, ../src/client/druid.scss */ +body .data-source-view .timeline { + height: 150px; +} +/* line 228, ../src/client/druid.scss */ +body .data-source-view .timeline h2 .download { + margin-top: 8px; +} +/* line 232, ../src/client/druid.scss */ +body .data-source-view .timeline > div { + width: 100%; +} +/* line 237, ../src/client/druid.scss */ +body .data-source-view .rules { + height: 75px; +} +/* line 240, ../src/client/druid.scss */ +body .data-source-view .rules .rule { + padding: 4px 8px; +} +/* line 242, ../src/client/druid.scss */ +body .data-source-view .rules .rule .default { + color: #e69545; +} +/* line 246, ../src/client/druid.scss */ +body .data-source-view .rules .rule .type .load { + color: #5ca0d7; +} +/* line 249, ../src/client/druid.scss */ +body .data-source-view .rules .rule .type .drop { + color: #f16065; +} +/* line 253, ../src/client/druid.scss */ +body .data-source-view .rules .rule .period, body .data-source-view .rules .rule .interval { + color: #fefefe; +} +/* line 259, ../src/client/druid.scss */ +body .data-source-view #intervals-segments { + flex: 1; + -webkit-flex: 1; + display: flex; + display: -webkit-flex; + width: 100vw; + flex-direction: row; + -webkit-flex-flow: row nowrap; +} +/* line 268, ../src/client/druid.scss */ +body .data-source-view #intervals-segments h2 { + padding-top: 1em; + padding-bottom: 0.5em; +} +/* line 273, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .dynamic-list { + bottom: 0; + left: 0; + overflow: auto; + position: absolute; + right: 0; + top: 0; +} +/* line 281, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .time-sep { + color: rgba(133, 133, 151, 0.3); +} +/* line 285, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-master { + flex: 1; + -webkit-flex: 1; + max-width: 240px; + width: 240px; + border-right: 1px solid #1b1a20; + display: block; + position: relative; + font-size: 14px; +} +/* line 295, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-master input { + width: 8em; +} +/* line 298, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-master table { + margin-left: 16px; +} +/* line 300, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-master table .count, body .data-source-view #intervals-segments .intervals-master table .size { + text-align: right; +} +/* line 303, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-master table td, body .data-source-view #intervals-segments .intervals-master table th { + padding: 2px 4px; +} +/* line 306, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-master table tbody { + line-height: 1.3; +} +/* line 309, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-master table tbody tr:hover { + color: #d8d8dd; + background: #1b1a20; +} +/* line 313, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-master table tbody tr td { + padding: 2px 4px; +} +/* line 316, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-master table tbody tr:nth-child(3n) { + border-bottom: 1px solid #3c3c47; +} +/* line 323, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-detail { + flex: 1; + -webkit-flex: 1; + font-size: 14px; +} +/* line 327, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-detail .segment, body .data-source-view #intervals-segments .intervals-detail .loading, body .data-source-view #intervals-segments .intervals-detail .info { + background: #2E2E38; + border: 1px solid #1b1a20; + border-radius: 15px; + padding: 8px; + margin: 8px 8px 0px 8px; + max-width: 600px; + font-size: 13px; +} +/* line 336, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-detail .segment { + display: flex; + display: -webkit-flex; + flex-direction: column; + -webkit-flex-flow: column nowrap; +} +/* line 342, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-detail .segment .segment-id { + text-align: center; +} +/* line 344, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-detail .segment .segment-id a:link, body .data-source-view #intervals-segments .intervals-detail .segment .segment-id a:visited { + color: #2a6496; +} +/* line 347, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-detail .segment .segment-id a:hover { + color: #428bca; +} +/* line 355, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-detail .segment .s3 i { + color: #e47911; +} +/* line 360, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-detail .segment .summary { + font-size: 16px; +} +/* line 364, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-detail .segment .summary :nth-child(3), body .data-source-view #intervals-segments .intervals-detail .segment .specifics :nth-child(3) { + text-align: right; +} +/* line 368, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-detail .segment > * { + width: 100%; + flex: 1; + -webkit-flex: 1; + margin: 4px 0px; + display: flex; + display: -webkit-flex; + flex-direction: row; + -webkit-flex-flow: row nowrap; +} +/* line 379, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-detail .segment > * > * { + flex: 1; + -webkit-flex: 1; +} +/* line 385, ../src/client/druid.scss */ +body .data-source-view #intervals-segments .intervals-detail .loading { + display: flex; + display: -webkit-flex; + flex-direction: row; + -webkit-flex-flow: row nowrap; + justify-content: flex-start; + -webkit-justify-content: flex-start; +} +/* line 403, ../src/client/druid.scss */ +body .cluster-view #data-sources h2, body .data-sources-view #data-sources h2 { + font-size: 24px; +} +/* line 406, ../src/client/druid.scss */ +body .cluster-view #data-sources table, body .data-sources-view #data-sources table { + border-collapse: inherit; + padding: 0px 8px; +} +/* line 409, ../src/client/druid.scss */ +body .cluster-view #data-sources table td:first-child, body .cluster-view #data-sources table th:first-child, body .data-sources-view #data-sources table td:first-child, body .data-sources-view #data-sources table th:first-child { + width: 16px; +} +/* line 412, ../src/client/druid.scss */ +body .cluster-view #data-sources table td.name, body .data-sources-view #data-sources table td.name { + font-size: 16px; + line-height: 1.5; +} +/* line 415, ../src/client/druid.scss */ +body .cluster-view #data-sources table td.name > a:after, body .data-sources-view #data-sources table td.name > a:after { + content: "\00a0"; +} +/* line 419, ../src/client/druid.scss */ +body .cluster-view #data-sources table td.name .rule-editor-link, body .data-sources-view #data-sources table td.name .rule-editor-link { + font-size: 85%; +} +/* line 421, ../src/client/druid.scss */ +body .cluster-view #data-sources table td.name .rule-editor-link a, body .data-sources-view #data-sources table td.name .rule-editor-link a { + color: #c8c8c8; +} +/* line 423, ../src/client/druid.scss */ +body .cluster-view #data-sources table td.name .rule-editor-link a:hover, body .data-sources-view #data-sources table td.name .rule-editor-link a:hover { + color: #fefefe; +} +/* line 429, ../src/client/druid.scss */ +body .cluster-view #data-sources table td.rules, body .data-sources-view #data-sources table td.rules { + width: 240px; +} +/* line 433, ../src/client/druid.scss */ +body .cluster-view #data-sources tr.data-source, body .data-sources-view #data-sources tr.data-source { + line-height: 1.3; +} +/* line 436, ../src/client/druid.scss */ +body .cluster-view #data-sources tr.data-source.unhappy .name, body .data-sources-view #data-sources tr.data-source.unhappy .name { + font-weight: bold; + color: #EEE; +} +/* line 440, ../src/client/druid.scss */ +body .cluster-view #data-sources tr.data-source .load-status, body .cluster-view #data-sources tr.data-source .replication, body .data-sources-view #data-sources tr.data-source .load-status, body .data-sources-view #data-sources tr.data-source .replication { + padding-right: 0.2em; +} +/* line 444, ../src/client/druid.scss */ +body .cluster-view #data-sources tr.data-source .load-status.loaded, body .data-sources-view #data-sources tr.data-source .load-status.loaded { + color: #5ca0d7; +} +/* line 447, ../src/client/druid.scss */ +body .cluster-view #data-sources tr.data-source .load-status.one-percent, body .data-sources-view #data-sources tr.data-source .load-status.one-percent { + color: #c6c631; +} +/* line 450, ../src/client/druid.scss */ +body .cluster-view #data-sources tr.data-source .load-status.unloaded, body .data-sources-view #data-sources tr.data-source .load-status.unloaded { + color: #f16065; +} +/* line 456, ../src/client/druid.scss */ +body .cluster-view #data-sources tr.key td, body .data-sources-view #data-sources tr.key td { + padding-top: 1em; +} +/* line 464, ../src/client/druid.scss */ +body .cluster-view #data-sources { + width: 240px; + border-right: 1px solid #1b1a20; +} +/* line 470, ../src/client/druid.scss */ +body .cluster-view #tiers { + vertical-align: top; +} +/* line 473, ../src/client/druid.scss */ +body .cluster-view #tiers .tier { + width: 240px; + display: inline-block; + vertical-align: top; + border-right: 1px solid #1b1a20; +} +/* line 479, ../src/client/druid.scss */ +body .cluster-view #tiers .tier h2 { + font-size: 24px; +} +/* line 481, ../src/client/druid.scss */ +body .cluster-view #tiers .tier h2 .queue { + font-size: 12px; +} +/* line 483, ../src/client/druid.scss */ +body .cluster-view #tiers .tier h2 .queue.load { + color: #EEE; +} +/* line 485, ../src/client/druid.scss */ +body .cluster-view #tiers .tier h2 .queue.load .bytes { + color: #f16065; + font-weight: bold; +} +/* line 496, ../src/client/druid.scss */ +body .cluster-view #tiers .tier tier-capacity g.capacity rect.maxSize, body .cluster-view #tiers .tier tr.node rect.maxSize { + fill: #5ca0d7; + opacity: 0.75; +} +/* line 500, ../src/client/druid.scss */ +body .cluster-view #tiers .tier tier-capacity g.capacity rect.currSize, body .cluster-view #tiers .tier tr.node rect.currSize { + fill: #f16065; +} +/* line 506, ../src/client/druid.scss */ +body .cluster-view #tiers .tier tier-capacity { + padding: 0px 4px; + padding-top: 0.5em; + padding-bottom: 1em; + display: flex; + display: -webkit-flex; + flex-direction: row; + -webkit-flex-flow: row nowrap; +} +/* line 516, ../src/client/druid.scss */ +body .cluster-view #tiers .tier tier-capacity div.labels { + width: 168px; + height: 180px; + padding-left: 0.5em; +} +/* line 521, ../src/client/druid.scss */ +body .cluster-view #tiers .tier tier-capacity div.labels .value { + color: #c8c8c8; +} +/* line 525, ../src/client/druid.scss */ +body .cluster-view #tiers .tier tier-capacity div.labels .top-labels > * { + margin-bottom: 4px; +} +/* line 530, ../src/client/druid.scss */ +body .cluster-view #tiers .tier tier-capacity div.labels .bottom-labels { + position: absolute; + bottom: 0px; +} +/* line 533, ../src/client/druid.scss */ +body .cluster-view #tiers .tier tier-capacity div.labels .bottom-labels > * { + margin-top: 4px; +} +/* line 538, ../src/client/druid.scss */ +body .cluster-view #tiers .tier tier-capacity div.labels .resilience { + line-height: 1; + font-size: 24px; + font-weight: bold; + color: #fefefe; +} +/* line 544, ../src/client/druid.scss */ +body .cluster-view #tiers .tier tier-capacity div.labels .free { + font-size: 18px; + font-weight: bold; +} +/* line 547, ../src/client/druid.scss */ +body .cluster-view #tiers .tier tier-capacity div.labels .free .value { + color: #fefefe; +} +/* line 551, ../src/client/druid.scss */ +body .cluster-view #tiers .tier tier-capacity div.labels .maxSize { + font-size: 18px; +} +/* line 556, ../src/client/druid.scss */ +body .cluster-view #tiers .tier tier-capacity div.labels .currSize { + font-size: 16px; +} +/* line 565, ../src/client/druid.scss */ +body .cluster-view #tiers .tier g.nodes .node { + fill: #ebeb3a; + opacity: 0.66; +} +/* line 572, ../src/client/druid.scss */ +body .cluster-view #tiers .tier div.capacity-history svg { + margin-top: 1em; +} +/* line 573, ../src/client/druid.scss */ +body .cluster-view #tiers .tier div.capacity-history text { + fill: white; + opacity: 0.75; +} +/* line 577, ../src/client/druid.scss */ +body .cluster-view #tiers .tier div.capacity-history path.maxSize { + fill: #5ca0d7; + stroke: none; + opacity: 0.75; +} +/* line 582, ../src/client/druid.scss */ +body .cluster-view #tiers .tier div.capacity-history path.currSize { + fill: #f16065; + stroke: none; +} +/* line 587, ../src/client/druid.scss */ +body .cluster-view #tiers .tier div.capacity-history .axis line { + stroke: #22222a; + opacity: 0.15; + shape-rendering: crispEdges; +} +/* line 592, ../src/client/druid.scss */ +body .cluster-view #tiers .tier div.capacity-history .axis text { + font-size: 7px; +} +/* line 596, ../src/client/druid.scss */ +body .cluster-view #tiers .tier div.capacity-history text.label { + font-size: 16px; + font-weight: bold; + text-anchor: middle; +} +/* line 601, ../src/client/druid.scss */ +body .cluster-view #tiers .tier div.capacity-history a.dash-link { + cursor: pointer; +} +/* line 605, ../src/client/druid.scss */ +body .cluster-view #tiers .tier div.capacity-history rect.delta.positive { + fill: #3cb349; +} +/* line 608, ../src/client/druid.scss */ +body .cluster-view #tiers .tier div.capacity-history rect.delta.negative { + fill: #f16065; +} +/* line 614, ../src/client/druid.scss */ +body .cluster-view #tiers .tier div.growth { + line-height: 1.3; +} +/* line 616, ../src/client/druid.scss */ +body .cluster-view #tiers .tier div.growth span.value { + color: #C8C8C8; +} +/* line 621, ../src/client/druid.scss */ +body .cluster-view #tiers .tier tier-nodes table { + margin-left: 4px; +} +/* line 624, ../src/client/druid.scss */ +body .cluster-view #tiers .tier tier-nodes table tr.node text { + fill: white; + alignment-baseline: central; +} +/* line 628, ../src/client/druid.scss */ +body .cluster-view #tiers .tier tier-nodes table tr.node td.host { + padding-left: 0.25em; +} +/* line 639, ../src/client/druid.scss */ +body .data-sources-view #data-sources { + width: 100%; +} +/* line 641, ../src/client/druid.scss */ +body .data-sources-view #data-sources > table { + width: 100%; +} +/* line 645, ../src/client/druid.scss */ +body .data-sources-view #data-sources > table td:first-child, body .data-sources-view #data-sources > table th:first-child { + width: 16px; +} +/* line 648, ../src/client/druid.scss */ +body .data-sources-view #data-sources > table td:nth-child(2), body .data-sources-view #data-sources > table th:nth-child(2) { + width: 224px; +} +/* line 653, ../src/client/druid.scss */ +body .data-sources-view #data-sources > table tbody tr:hover { + background: #2E2E38; +} +/* line 657, ../src/client/druid.scss */ +body .data-sources-view #data-sources > table td.footprint { + padding: 4px 6px; +} +/* line 662, ../src/client/druid.scss */ +body .data-sources-view .disabled-data-sources { + padding: 0px 8px; +} +/* line 664, ../src/client/druid.scss */ +body .data-sources-view .disabled-data-sources li { + font-size: 16px; + line-height: 1.75; +} +/* line 674, ../src/client/druid.scss */ +body svg.rules-timeline ._default_tier rect, body svg.timeline ._default_tier rect { + fill: #5ca0d7; +} +/* line 679, ../src/client/druid.scss */ +body svg.rules-timeline .hot rect, body svg.timeline .hot rect { + fill: #f16065; +} +/* line 684, ../src/client/druid.scss */ +body svg.rules-timeline .days rect, body svg.timeline .days rect { + fill: #33a680; +} +/* line 686, ../src/client/druid.scss */ +body svg.rules-timeline .days rect:hover, body svg.timeline .days rect:hover { + stroke-width: 2px; + stroke: #c6c631; +} +/* line 693, ../src/client/druid.scss */ +body svg.rules-timeline text, body svg.timeline text { + text-anchor: end; +} +/* line 699, ../src/client/druid.scss */ +body svg.rules-timeline.with-footprint .load rect, body svg.rules-timeline.with-footprint .drop rect, body svg.timeline.with-footprint .load rect, body svg.timeline.with-footprint .drop rect { + opacity: 0.33; +} +/* line 706, ../src/client/druid.scss */ +body svg.rules-timeline .rules text, body svg.timeline .rules text { + opacity: 0.75; + fill: #eee; +} +/* line 712, ../src/client/druid.scss */ +body svg.rules-timeline .axis, body svg.timeline .axis { + shape-rendering: crispEdges; +} +/* line 716, ../src/client/druid.scss */ +body svg.rules-timeline .x.axis text, body svg.timeline .x.axis text { + font-size: 11px; + fill: #858597; +} +/* line 720, ../src/client/druid.scss */ +body svg.rules-timeline .x.axis line, body svg.timeline .x.axis line { + stroke: #e69545; + opacity: 0.5; +} +/* line 725, ../src/client/druid.scss */ +body svg.rules-timeline .x.axis .minor, body svg.timeline .x.axis .minor { + stroke-opacity: .5; +} +/* line 728, ../src/client/druid.scss */ +body svg.rules-timeline .x.axis path, body svg.timeline .x.axis path { + display: none; +} +/* line 735, ../src/client/druid.scss */ +body svg.timeline .rules rect { + opacity: 0.33; +} +/* line 785, ../src/client/druid.scss */ +body .rules-editor { + flex-direction: column; + -webkit-flex-flow: column nowrap; +} +/* line 789, ../src/client/druid.scss */ +body .rules-editor h1 { + font-size: 18px; +} +/* line 792, ../src/client/druid.scss */ +body .rules-editor .rule { + height: 120px; + margin-bottom: 10px; +} +/* line 796, ../src/client/druid.scss */ +body .rules-editor .rule:not(:first-child) { + border-top: 1px solid #858597; + padding-top: 8px; + height: 129px; +} +/* line 801, ../src/client/druid.scss */ +body .rules-editor .rule .summary { + font-size: 16px; + margin-bottom: 10px; +} +/* line 804, ../src/client/druid.scss */ +body .rules-editor .rule .summary .rule { + color: #fefefe; +} +/* line 808, ../src/client/druid.scss */ +body .rules-editor .rule .edit { + display: flex; + display: -webkit-flex; + margin-bottom: 1em; + flex-direction: row; + justify-content: flex-start; +} +/* line 814, ../src/client/druid.scss */ +body .rules-editor .rule .edit > * { + flex: initial; + -webkit-flex: initial; + margin-right: 1em; +} +/* line 818, ../src/client/druid.scss */ +body .rules-editor .rule .edit > *:last-child { + margin-right: 0px; +} +/* line 822, ../src/client/druid.scss */ +body .rules-editor .rule .edit .interval-period { + width: 200px; +} +/* line 824, ../src/client/druid.scss */ +body .rules-editor .rule .edit .interval-period input { + margin-bottom: 10px; +} +/* line 827, ../src/client/druid.scss */ +body .rules-editor .rule .edit .interval-period .alert { + margin-bottom: 0; +} +/* line 831, ../src/client/druid.scss */ +body .rules-editor .rule .edit .replicants { + width: 160px; +} +/* line 833, ../src/client/druid.scss */ +body .rules-editor .rule .edit .replicants .replicant { + white-space: nowrap; + margin-bottom: 5px; +} +/* line 836, ../src/client/druid.scss */ +body .rules-editor .rule .edit .replicants .replicant input[type=number] { + width: 2.5em; +} +/* line 839, ../src/client/druid.scss */ +body .rules-editor .rule .edit .replicants .replicant button { + vertical-align: baseline; +} +/* line 846, ../src/client/druid.scss */ +body .rules-editor .alert { + padding: 10px 12px; +} + +/* line 852, ../src/client/druid.scss */ +#floatingCirclesG { + position: relative; + width: 128px; + height: 128px; + -webkit-transform: scale(0.6); + transform: scale(0.6); +} + +/* line 860, ../src/client/druid.scss */ +.f_circleG { + position: absolute; + background-color: #2E2E38; + height: 23px; + width: 23px; + -webkit-border-radius: 12px; + -webkit-animation-name: f_fadeG; + -webkit-animation-duration: 1.04s; + -webkit-animation-iteration-count: infinite; + -webkit-animation-direction: linear; + border-radius: 12px; + animation-name: f_fadeG; + animation-duration: 1.04s; + animation-iteration-count: infinite; + animation-direction: linear; +} + +/* line 877, ../src/client/druid.scss */ +#frotateG_01 { + left: 0; + top: 52px; + -webkit-animation-delay: 0.39s; + animation-delay: 0.39s; +} + +/* line 884, ../src/client/druid.scss */ +#frotateG_02 { + left: 15px; + top: 15px; + -webkit-animation-delay: 0.52s; + animation-delay: 0.52s; +} + +/* line 891, ../src/client/druid.scss */ +#frotateG_03 { + left: 52px; + top: 0; + -webkit-animation-delay: 0.65s; + animation-delay: 0.65s; +} + +/* line 898, ../src/client/druid.scss */ +#frotateG_04 { + right: 15px; + top: 15px; + -webkit-animation-delay: 0.78s; + animation-delay: 0.78s; +} + +/* line 905, ../src/client/druid.scss */ +#frotateG_05 { + right: 0; + top: 52px; + -webkit-animation-delay: 0.91s; + animation-delay: 0.91s; +} + +/* line 912, ../src/client/druid.scss */ +#frotateG_06 { + right: 15px; + bottom: 15px; + -webkit-animation-delay: 1.04s; + animation-delay: 1.04s; +} + +/* line 919, ../src/client/druid.scss */ +#frotateG_07 { + left: 52px; + bottom: 0; + -webkit-animation-delay: 1.17s; + animation-delay: 1.17s; +} + +/* line 926, ../src/client/druid.scss */ +#frotateG_08 { + left: 15px; + bottom: 15px; + -webkit-animation-delay: 1.3s; + animation-delay: 1.3s; +} + +@-webkit-keyframes f_fadeG { + /* line 934, ../src/client/druid.scss */ + 0% { + background-color: #858597; + } + + /* line 937, ../src/client/druid.scss */ + 100% { + background-color: #2E2E38; + } +} + +@keyframes f_fadeG { + /* line 943, ../src/client/druid.scss */ + 0% { + background-color: #858597; + } + + /* line 946, ../src/client/druid.scss */ + 100% { + background-color: #2E2E38; + } +} diff --git a/server/src/main/resources/static/druid.js b/server/src/main/resources/static/druid.js new file mode 100644 index 00000000000..92cce5f149e --- /dev/null +++ b/server/src/main/resources/static/druid.js @@ -0,0 +1,12761 @@ +(function e(t,n,r){function s(o,u){if(!n[o]){if(!t[o]){var a=typeof require=="function"&&require;if(!u&&a)return a(o,!0);if(i)return i(o,!0);throw new Error("Cannot find module '"+o+"'")}var f=n[o]={exports:{}};t[o][0].call(f.exports,function(e){var n=t[o][1][e];return s(n?n:e)},f,f.exports,e,t,n,r)}return n[o].exports}var i=typeof require=="function"&&require;for(var o=0;o=0&&(g=b(f,c))}function e(){g&&(b.cancel(g),g=null)}function f(){h?(a.next(),d()):a.pause()}var g,h,i=this,j=i.slides=[],k=-1;i.currentSlide=null;var l=!1;i.select=function(e,f){function g(){if(!l){if(i.currentSlide&&angular.isString(f)&&!a.noTransition&&e.$element){e.$element.addClass(f);{e.$element[0].offsetWidth}angular.forEach(j,function(a){angular.extend(a,{direction:"",entering:!1,leaving:!1,active:!1})}),angular.extend(e,{direction:f,active:!0,entering:!0}),angular.extend(i.currentSlide||{},{direction:f,leaving:!0}),a.$currentTransition=c(e.$element,{}),function(b,c){a.$currentTransition.then(function(){h(b,c)},function(){h(b,c)})}(e,i.currentSlide)}else h(e,i.currentSlide);i.currentSlide=e,k=m,d()}}function h(b,c){angular.extend(b,{direction:"",active:!0,leaving:!1,entering:!1}),angular.extend(c||{},{direction:"",active:!1,leaving:!1,entering:!1}),a.$currentTransition=null}var m=j.indexOf(e);void 0===f&&(f=m>k?"next":"prev"),e&&e!==i.currentSlide&&(a.$currentTransition?(a.$currentTransition.cancel(),b(g)):g())},a.$on("$destroy",function(){l=!0}),i.indexOfSlide=function(a){return j.indexOf(a)},a.next=function(){var b=(k+1)%j.length;return a.$currentTransition?void 0:i.select(j[b],"next")},a.prev=function(){var b=0>k-1?j.length-1:k-1;return a.$currentTransition?void 0:i.select(j[b],"prev")},a.select=function(a){i.select(a)},a.isActive=function(a){return i.currentSlide===a},a.slides=function(){return j},a.$watch("interval",d),a.$on("$destroy",e),a.play=function(){h||(h=!0,d())},a.pause=function(){a.noPause||(h=!1,e())},i.addSlide=function(b,c){b.$element=c,j.push(b),1===j.length||b.active?(i.select(j[j.length-1]),1==j.length&&a.play()):b.active=!1},i.removeSlide=function(a){var b=j.indexOf(a);j.splice(b,1),j.length>0&&a.active?b>=j.length?i.select(j[b-1]):i.select(j[b]):k>b&&k--}}]).directive("carousel",[function(){return{restrict:"EA",transclude:!0,replace:!0,controller:"CarouselController",require:"carousel",templateUrl:"template/carousel/carousel.html",scope:{interval:"=",noTransition:"=",noPause:"="}}}]).directive("slide",["$parse",function(a){return{require:"^carousel",restrict:"EA",transclude:!0,replace:!0,templateUrl:"template/carousel/slide.html",scope:{},link:function(b,c,d,e){if(d.active){var f=a(d.active),g=f.assign,h=b.active=f(b.$parent);b.$watch(function(){var a=f(b.$parent);return a!==b.active&&(a!==h?h=b.active=a:g(b.$parent,a=h=b.active)),a})}e.addSlide(b,c),b.$on("$destroy",function(){e.removeSlide(b)}),b.$watch("active",function(a){a&&e.select(b)})}}}]),angular.module("ui.bootstrap.position",[]).factory("$position",["$document","$window",function(a,b){function c(a,c){return a.currentStyle?a.currentStyle[c]:b.getComputedStyle?b.getComputedStyle(a)[c]:a.style[c]}function d(a){return"static"===(c(a,"position")||"static")}var e=function(b){for(var c=a[0],e=b.offsetParent||c;e&&e!==c&&d(e);)e=e.offsetParent;return e||c};return{position:function(b){var c=this.offset(b),d={top:0,left:0},f=e(b[0]);f!=a[0]&&(d=this.offset(angular.element(f)),d.top+=f.clientTop-f.scrollTop,d.left+=f.clientLeft-f.scrollLeft);var g=b[0].getBoundingClientRect();return{width:g.width||b.prop("offsetWidth"),height:g.height||b.prop("offsetHeight"),top:c.top-d.top,left:c.left-d.left}},offset:function(c){var d=c[0].getBoundingClientRect();return{width:d.width||c.prop("offsetWidth"),height:d.height||c.prop("offsetHeight"),top:d.top+(b.pageYOffset||a[0].body.scrollTop||a[0].documentElement.scrollTop),left:d.left+(b.pageXOffset||a[0].body.scrollLeft||a[0].documentElement.scrollLeft)}}}}]),angular.module("ui.bootstrap.datepicker",["ui.bootstrap.position"]).constant("datepickerConfig",{dayFormat:"dd",monthFormat:"MMMM",yearFormat:"yyyy",dayHeaderFormat:"EEE",dayTitleFormat:"MMMM yyyy",monthTitleFormat:"yyyy",showWeeks:!0,startingDay:0,yearRange:20,minDate:null,maxDate:null}).controller("DatepickerController",["$scope","$attrs","dateFilter","datepickerConfig",function(a,b,c,d){function e(b,c){return angular.isDefined(b)?a.$parent.$eval(b):c}function f(a,b){return new Date(a,b,0).getDate()}function g(a,b){for(var c=new Array(b),d=a,e=0;b>e;)c[e++]=new Date(d),d.setDate(d.getDate()+1);return c}function h(a,b,d,e){return{date:a,label:c(a,b),selected:!!d,secondary:!!e}}var i={day:e(b.dayFormat,d.dayFormat),month:e(b.monthFormat,d.monthFormat),year:e(b.yearFormat,d.yearFormat),dayHeader:e(b.dayHeaderFormat,d.dayHeaderFormat),dayTitle:e(b.dayTitleFormat,d.dayTitleFormat),monthTitle:e(b.monthTitleFormat,d.monthTitleFormat)},j=e(b.startingDay,d.startingDay),k=e(b.yearRange,d.yearRange);this.minDate=d.minDate?new Date(d.minDate):null,this.maxDate=d.maxDate?new Date(d.maxDate):null,this.modes=[{name:"day",getVisibleDates:function(a,b){var d=a.getFullYear(),e=a.getMonth(),k=new Date(d,e,1),l=j-k.getDay(),m=l>0?7-l:-l,n=new Date(k),o=0;m>0&&(n.setDate(-m+1),o+=m),o+=f(d,e+1),o+=(7-o%7)%7;for(var p=g(n,o),q=new Array(7),r=0;o>r;r++){var s=new Date(p[r]);p[r]=h(s,i.day,b&&b.getDate()===s.getDate()&&b.getMonth()===s.getMonth()&&b.getFullYear()===s.getFullYear(),s.getMonth()!==e)}for(var t=0;7>t;t++)q[t]=c(p[t].date,i.dayHeader);return{objects:p,title:c(a,i.dayTitle),labels:q}},compare:function(a,b){return new Date(a.getFullYear(),a.getMonth(),a.getDate())-new Date(b.getFullYear(),b.getMonth(),b.getDate())},split:7,step:{months:1}},{name:"month",getVisibleDates:function(a,b){for(var d=new Array(12),e=a.getFullYear(),f=0;12>f;f++){var g=new Date(e,f,1);d[f]=h(g,i.month,b&&b.getMonth()===f&&b.getFullYear()===e)}return{objects:d,title:c(a,i.monthTitle)}},compare:function(a,b){return new Date(a.getFullYear(),a.getMonth())-new Date(b.getFullYear(),b.getMonth())},split:3,step:{years:1}},{name:"year",getVisibleDates:function(a,b){for(var c=new Array(k),d=a.getFullYear(),e=parseInt((d-1)/k,10)*k+1,f=0;k>f;f++){var g=new Date(e+f,0,1);c[f]=h(g,i.year,b&&b.getFullYear()===g.getFullYear())}return{objects:c,title:[c[0].label,c[k-1].label].join(" - ")}},compare:function(a,b){return a.getFullYear()-b.getFullYear()},split:5,step:{years:k}}],this.isDisabled=function(b,c){var d=this.modes[c||0];return this.minDate&&d.compare(b,this.minDate)<0||this.maxDate&&d.compare(b,this.maxDate)>0||a.dateDisabled&&a.dateDisabled({date:b,mode:d.name})}}]).directive("datepicker",["dateFilter","$parse","datepickerConfig","$log",function(a,b,c,d){return{restrict:"EA",replace:!0,templateUrl:"template/datepicker/datepicker.html",scope:{dateDisabled:"&"},require:["datepicker","?^ngModel"],controller:"DatepickerController",link:function(a,e,f,g){function h(){a.showWeekNumbers=0===o&&q}function i(a,b){for(var c=[];a.length>0;)c.push(a.splice(0,b));return c}function j(b){var c=null,e=!0;n.$modelValue&&(c=new Date(n.$modelValue),isNaN(c)?(e=!1,d.error('Datepicker directive: "ng-model" value must be a Date object, a number of milliseconds since 01.01.1970 or a string representing an RFC2822 or ISO 8601 date.')):b&&(p=c)),n.$setValidity("date",e);var f=m.modes[o],g=f.getVisibleDates(p,c);angular.forEach(g.objects,function(a){a.disabled=m.isDisabled(a.date,o)}),n.$setValidity("date-disabled",!c||!m.isDisabled(c)),a.rows=i(g.objects,f.split),a.labels=g.labels||[],a.title=g.title}function k(a){o=a,h(),j()}function l(a){var b=new Date(a);b.setDate(b.getDate()+4-(b.getDay()||7));var c=b.getTime();return b.setMonth(0),b.setDate(1),Math.floor(Math.round((c-b)/864e5)/7)+1}var m=g[0],n=g[1];if(n){var o=0,p=new Date,q=c.showWeeks;f.showWeeks?a.$parent.$watch(b(f.showWeeks),function(a){q=!!a,h()}):h(),f.min&&a.$parent.$watch(b(f.min),function(a){m.minDate=a?new Date(a):null,j()}),f.max&&a.$parent.$watch(b(f.max),function(a){m.maxDate=a?new Date(a):null,j()}),n.$render=function(){j(!0)},a.select=function(a){if(0===o){var b=n.$modelValue?new Date(n.$modelValue):new Date(0,0,0,0,0,0,0);b.setFullYear(a.getFullYear(),a.getMonth(),a.getDate()),n.$setViewValue(b),j(!0)}else p=a,k(o-1)},a.move=function(a){var b=m.modes[o].step;p.setMonth(p.getMonth()+a*(b.months||0)),p.setFullYear(p.getFullYear()+a*(b.years||0)),j()},a.toggleMode=function(){k((o+1)%m.modes.length)},a.getWeekNumber=function(b){return 0===o&&a.showWeekNumbers&&7===b.length?l(b[0].date):null}}}}}]).constant("datepickerPopupConfig",{dateFormat:"yyyy-MM-dd",currentText:"Today",toggleWeeksText:"Weeks",clearText:"Clear",closeText:"Done",closeOnDateSelection:!0,appendToBody:!1,showButtonBar:!0}).directive("datepickerPopup",["$compile","$parse","$document","$position","dateFilter","datepickerPopupConfig","datepickerConfig",function(a,b,c,d,e,f,g){return{restrict:"EA",require:"ngModel",link:function(h,i,j,k){function l(a){u?u(h,!!a):q.isOpen=!!a}function m(a){if(a){if(angular.isDate(a))return k.$setValidity("date",!0),a;if(angular.isString(a)){var b=new Date(a);return isNaN(b)?(k.$setValidity("date",!1),void 0):(k.$setValidity("date",!0),b)}return k.$setValidity("date",!1),void 0}return k.$setValidity("date",!0),null}function n(a,c,d){a&&(h.$watch(b(a),function(a){q[c]=a}),y.attr(d||c,c))}function o(){q.position=s?d.offset(i):d.position(i),q.position.top=q.position.top+i.prop("offsetHeight")}var p,q=h.$new(),r=angular.isDefined(j.closeOnDateSelection)?h.$eval(j.closeOnDateSelection):f.closeOnDateSelection,s=angular.isDefined(j.datepickerAppendToBody)?h.$eval(j.datepickerAppendToBody):f.appendToBody;j.$observe("datepickerPopup",function(a){p=a||f.dateFormat,k.$render()}),q.showButtonBar=angular.isDefined(j.showButtonBar)?h.$eval(j.showButtonBar):f.showButtonBar,h.$on("$destroy",function(){C.remove(),q.$destroy()}),j.$observe("currentText",function(a){q.currentText=angular.isDefined(a)?a:f.currentText}),j.$observe("toggleWeeksText",function(a){q.toggleWeeksText=angular.isDefined(a)?a:f.toggleWeeksText}),j.$observe("clearText",function(a){q.clearText=angular.isDefined(a)?a:f.clearText}),j.$observe("closeText",function(a){q.closeText=angular.isDefined(a)?a:f.closeText});var t,u;j.isOpen&&(t=b(j.isOpen),u=t.assign,h.$watch(t,function(a){q.isOpen=!!a})),q.isOpen=t?t(h):!1;var v=function(a){q.isOpen&&a.target!==i[0]&&q.$apply(function(){l(!1)})},w=function(){q.$apply(function(){l(!0)})},x=angular.element("
");x.attr({"ng-model":"date","ng-change":"dateSelection()"});var y=angular.element(x.children()[0]),z={};j.datepickerOptions&&(z=h.$eval(j.datepickerOptions),y.attr(angular.extend({},z))),k.$parsers.unshift(m),q.dateSelection=function(a){angular.isDefined(a)&&(q.date=a),k.$setViewValue(q.date),k.$render(),r&&l(!1)},i.bind("input change keyup",function(){q.$apply(function(){q.date=k.$modelValue})}),k.$render=function(){var a=k.$viewValue?e(k.$viewValue,p):"";i.val(a),q.date=k.$modelValue},n(j.min,"min"),n(j.max,"max"),j.showWeeks?n(j.showWeeks,"showWeeks","show-weeks"):(q.showWeeks="show-weeks"in z?z["show-weeks"]:g.showWeeks,y.attr("show-weeks","showWeeks")),j.dateDisabled&&y.attr("date-disabled",j.dateDisabled);var A=!1,B=!1;q.$watch("isOpen",function(a){a?(o(),c.bind("click",v),B&&i.unbind("focus",w),i[0].focus(),A=!0):(A&&c.unbind("click",v),i.bind("focus",w),B=!0),u&&u(h,a)}),q.today=function(){q.dateSelection(new Date)},q.clear=function(){q.dateSelection(null)};var C=a(x)(q);s?c.find("body").append(C):i.after(C)}}}]).directive("datepickerPopupWrap",function(){return{restrict:"EA",replace:!0,transclude:!0,templateUrl:"template/datepicker/popup.html",link:function(a,b){b.bind("click",function(a){a.preventDefault(),a.stopPropagation()})}}}),angular.module("ui.bootstrap.dropdownToggle",[]).directive("dropdownToggle",["$document","$location",function(a){var b=null,c=angular.noop;return{restrict:"CA",link:function(d,e){d.$watch("$location.path",function(){c()}),e.parent().bind("click",function(){c()}),e.bind("click",function(d){var f=e===b;d.preventDefault(),d.stopPropagation(),b&&c(),f||e.hasClass("disabled")||e.prop("disabled")||(e.parent().addClass("open"),b=e,c=function(d){d&&(d.preventDefault(),d.stopPropagation()),a.unbind("click",c),e.parent().removeClass("open"),c=angular.noop,b=null},a.bind("click",c))})}}}]),angular.module("ui.bootstrap.modal",["ui.bootstrap.transition"]).factory("$$stackedMap",function(){return{createNew:function(){var a=[];return{add:function(b,c){a.push({key:b,value:c})},get:function(b){for(var c=0;c0)}function i(){if(k&&-1==g()){var a=l;j(k,l,150,function(){a.$destroy(),a=null}),k=void 0,l=void 0}}function j(c,d,e,f){function g(){g.done||(g.done=!0,c.remove(),f&&f())}d.animate=!1;var h=a.transitionEndEventName;if(h){var i=b(g,e);c.bind(h,function(){b.cancel(i),g(),d.$apply()})}else b(g,0)}var k,l,m="modal-open",n=f.createNew(),o={};return e.$watch(g,function(a){l&&(l.index=a)}),c.bind("keydown",function(a){var b;27===a.which&&(b=n.top(),b&&b.value.keyboard&&e.$apply(function(){o.dismiss(b.key)}))}),o.open=function(a,b){n.add(a,{deferred:b.deferred,modalScope:b.scope,backdrop:b.backdrop,keyboard:b.keyboard});var f=c.find("body").eq(0),h=g();h>=0&&!k&&(l=e.$new(!0),l.index=h,k=d("
")(l),f.append(k));var i=angular.element("
");i.attr("window-class",b.windowClass),i.attr("index",n.length()-1),i.attr("animate","animate"),i.html(b.content);var j=d(i)(b.scope);n.top().value.modalDomEl=j,f.append(j),f.addClass(m)},o.close=function(a,b){var c=n.get(a).value;c&&(c.deferred.resolve(b),h(a))},o.dismiss=function(a,b){var c=n.get(a).value;c&&(c.deferred.reject(b),h(a))},o.dismissAll=function(a){for(var b=this.getTop();b;)this.dismiss(b.key,a),b=this.getTop()},o.getTop=function(){return n.top()},o}]).provider("$modal",function(){var a={options:{backdrop:!0,keyboard:!0},$get:["$injector","$rootScope","$q","$http","$templateCache","$controller","$modalStack",function(b,c,d,e,f,g,h){function i(a){return a.template?d.when(a.template):e.get(a.templateUrl,{cache:f}).then(function(a){return a.data})}function j(a){var c=[];return angular.forEach(a,function(a){(angular.isFunction(a)||angular.isArray(a))&&c.push(d.when(b.invoke(a)))}),c}var k={};return k.open=function(b){var e=d.defer(),f=d.defer(),k={result:e.promise,opened:f.promise,close:function(a){h.close(k,a)},dismiss:function(a){h.dismiss(k,a)}};if(b=angular.extend({},a.options,b),b.resolve=b.resolve||{},!b.template&&!b.templateUrl)throw new Error("One of template or templateUrl options is required.");var l=d.all([i(b)].concat(j(b.resolve)));return l.then(function(a){var d=(b.scope||c).$new();d.$close=k.close,d.$dismiss=k.dismiss;var f,i={},j=1;b.controller&&(i.$scope=d,i.$modalInstance=k,angular.forEach(b.resolve,function(b,c){i[c]=a[j++]}),f=g(b.controller,i)),h.open(k,{scope:d,deferred:e,content:a[0],backdrop:b.backdrop,keyboard:b.keyboard,windowClass:b.windowClass})},function(a){e.reject(a)}),l.then(function(){f.resolve(!0)},function(){f.reject(!1)}),k},k}]};return a}),angular.module("ui.bootstrap.pagination",[]).controller("PaginationController",["$scope","$attrs","$parse","$interpolate",function(a,b,c,d){var e=this,f=b.numPages?c(b.numPages).assign:angular.noop;this.init=function(d){b.itemsPerPage?a.$parent.$watch(c(b.itemsPerPage),function(b){e.itemsPerPage=parseInt(b,10),a.totalPages=e.calculateTotalPages()}):this.itemsPerPage=d},this.noPrevious=function(){return 1===this.page},this.noNext=function(){return this.page===a.totalPages},this.isActive=function(a){return this.page===a},this.calculateTotalPages=function(){var b=this.itemsPerPage<1?1:Math.ceil(a.totalItems/this.itemsPerPage);return Math.max(b||0,1)},this.getAttributeValue=function(b,c,e){return angular.isDefined(b)?e?d(b)(a.$parent):a.$parent.$eval(b):c},this.render=function(){this.page=parseInt(a.page,10)||1,this.page>0&&this.page<=a.totalPages&&(a.pages=this.getPages(this.page,a.totalPages))},a.selectPage=function(b){!e.isActive(b)&&b>0&&b<=a.totalPages&&(a.page=b,a.onSelectPage({page:b}))},a.$watch("page",function(){e.render()}),a.$watch("totalItems",function(){a.totalPages=e.calculateTotalPages()}),a.$watch("totalPages",function(b){f(a.$parent,b),e.page>b?a.selectPage(b):e.render()})}]).constant("paginationConfig",{itemsPerPage:10,boundaryLinks:!1,directionLinks:!0,firstText:"First",previousText:"Previous",nextText:"Next",lastText:"Last",rotate:!0}).directive("pagination",["$parse","paginationConfig",function(a,b){return{restrict:"EA",scope:{page:"=",totalItems:"=",onSelectPage:" &"},controller:"PaginationController",templateUrl:"template/pagination/pagination.html",replace:!0,link:function(c,d,e,f){function g(a,b,c,d){return{number:a,text:b,active:c,disabled:d}}var h,i=f.getAttributeValue(e.boundaryLinks,b.boundaryLinks),j=f.getAttributeValue(e.directionLinks,b.directionLinks),k=f.getAttributeValue(e.firstText,b.firstText,!0),l=f.getAttributeValue(e.previousText,b.previousText,!0),m=f.getAttributeValue(e.nextText,b.nextText,!0),n=f.getAttributeValue(e.lastText,b.lastText,!0),o=f.getAttributeValue(e.rotate,b.rotate);f.init(b.itemsPerPage),e.maxSize&&c.$parent.$watch(a(e.maxSize),function(a){h=parseInt(a,10),f.render()}),f.getPages=function(a,b){var c=[],d=1,e=b,p=angular.isDefined(h)&&b>h;p&&(o?(d=Math.max(a-Math.floor(h/2),1),e=d+h-1,e>b&&(e=b,d=e-h+1)):(d=(Math.ceil(a/h)-1)*h+1,e=Math.min(d+h-1,b)));for(var q=d;e>=q;q++){var r=g(q,q,f.isActive(q),!1);c.push(r)}if(p&&!o){if(d>1){var s=g(d-1,"...",!1,!1);c.unshift(s)}if(b>e){var t=g(e+1,"...",!1,!1);c.push(t)}}if(j){var u=g(a-1,l,!1,f.noPrevious());c.unshift(u);var v=g(a+1,m,!1,f.noNext());c.push(v)}if(i){var w=g(1,k,!1,f.noPrevious());c.unshift(w);var x=g(b,n,!1,f.noNext());c.push(x)}return c}}}}]).constant("pagerConfig",{itemsPerPage:10,previousText:"« Previous",nextText:"Next »",align:!0}).directive("pager",["pagerConfig",function(a){return{restrict:"EA",scope:{page:"=",totalItems:"=",onSelectPage:" &"},controller:"PaginationController",templateUrl:"template/pagination/pager.html",replace:!0,link:function(b,c,d,e){function f(a,b,c,d,e){return{number:a,text:b,disabled:c,previous:i&&d,next:i&&e}}var g=e.getAttributeValue(d.previousText,a.previousText,!0),h=e.getAttributeValue(d.nextText,a.nextText,!0),i=e.getAttributeValue(d.align,a.align);e.init(a.itemsPerPage),e.getPages=function(a){return[f(a-1,g,e.noPrevious(),!0,!1),f(a+1,h,e.noNext(),!1,!0)]}}}}]),angular.module("ui.bootstrap.tooltip",["ui.bootstrap.position","ui.bootstrap.bindHtml"]).provider("$tooltip",function(){function a(a){var b=/[A-Z]/g,c="-";return a.replace(b,function(a,b){return(b?c:"")+a.toLowerCase()})}var b={placement:"top",animation:!0,popupDelay:0},c={mouseenter:"mouseleave",click:"click",focus:"blur"},d={};this.options=function(a){angular.extend(d,a)},this.setTriggers=function(a){angular.extend(c,a)},this.$get=["$window","$compile","$timeout","$parse","$document","$position","$interpolate",function(e,f,g,h,i,j,k){return function(e,l,m){function n(a){var b=a||o.trigger||m,d=c[b]||b;return{show:b,hide:d}}var o=angular.extend({},b,d),p=a(e),q=k.startSymbol(),r=k.endSymbol(),s="
';return{restrict:"EA",scope:!0,compile:function(){var a=f(s);return function(b,c,d){function f(){b.tt_isOpen?m():k()}function k(){(!z||b.$eval(d[l+"Enable"]))&&(b.tt_popupDelay?(v=g(p,b.tt_popupDelay,!1),v.then(function(a){a()})):p()())}function m(){b.$apply(function(){q()})}function p(){return b.tt_content?(r(),u&&g.cancel(u),t.css({top:0,left:0,display:"block"}),w?i.find("body").append(t):c.after(t),A(),b.tt_isOpen=!0,b.$digest(),A):angular.noop}function q(){b.tt_isOpen=!1,g.cancel(v),b.tt_animation?u=g(s,500):s()}function r(){t&&s(),t=a(b,function(){}),b.$digest()}function s(){t&&(t.remove(),t=null)}var t,u,v,w=angular.isDefined(o.appendToBody)?o.appendToBody:!1,x=n(void 0),y=!1,z=angular.isDefined(d[l+"Enable"]),A=function(){var a,d,e,f;switch(a=w?j.offset(c):j.position(c),d=t.prop("offsetWidth"),e=t.prop("offsetHeight"),b.tt_placement){case"right":f={top:a.top+a.height/2-e/2,left:a.left+a.width};break;case"bottom":f={top:a.top+a.height,left:a.left+a.width/2-d/2};break;case"left":f={top:a.top+a.height/2-e/2,left:a.left-d};break;default:f={top:a.top-e,left:a.left+a.width/2-d/2}}f.top+="px",f.left+="px",t.css(f)};b.tt_isOpen=!1,d.$observe(e,function(a){b.tt_content=a,!a&&b.tt_isOpen&&q()}),d.$observe(l+"Title",function(a){b.tt_title=a}),d.$observe(l+"Placement",function(a){b.tt_placement=angular.isDefined(a)?a:o.placement}),d.$observe(l+"PopupDelay",function(a){var c=parseInt(a,10);b.tt_popupDelay=isNaN(c)?o.popupDelay:c});var B=function(){y&&(c.unbind(x.show,k),c.unbind(x.hide,m))};d.$observe(l+"Trigger",function(a){B(),x=n(a),x.show===x.hide?c.bind(x.show,f):(c.bind(x.show,k),c.bind(x.hide,m)),y=!0});var C=b.$eval(d[l+"Animation"]);b.tt_animation=angular.isDefined(C)?!!C:o.animation,d.$observe(l+"AppendToBody",function(a){w=angular.isDefined(a)?h(a)(b):w}),w&&b.$on("$locationChangeSuccess",function(){b.tt_isOpen&&q()}),b.$on("$destroy",function(){g.cancel(u),g.cancel(v),B(),s()})}}}}}]}).directive("tooltipPopup",function(){return{restrict:"EA",replace:!0,scope:{content:"@",placement:"@",animation:"&",isOpen:"&"},templateUrl:"template/tooltip/tooltip-popup.html"}}).directive("tooltip",["$tooltip",function(a){return a("tooltip","tooltip","mouseenter")}]).directive("tooltipHtmlUnsafePopup",function(){return{restrict:"EA",replace:!0,scope:{content:"@",placement:"@",animation:"&",isOpen:"&"},templateUrl:"template/tooltip/tooltip-html-unsafe-popup.html"}}).directive("tooltipHtmlUnsafe",["$tooltip",function(a){return a("tooltipHtmlUnsafe","tooltip","mouseenter")}]),angular.module("ui.bootstrap.popover",["ui.bootstrap.tooltip"]).directive("popoverPopup",function(){return{restrict:"EA",replace:!0,scope:{title:"@",content:"@",placement:"@",animation:"&",isOpen:"&"},templateUrl:"template/popover/popover.html"}}).directive("popover",["$tooltip",function(a){return a("popover","popover","click")}]),angular.module("ui.bootstrap.progressbar",["ui.bootstrap.transition"]).constant("progressConfig",{animate:!0,max:100}).controller("ProgressController",["$scope","$attrs","progressConfig","$transition",function(a,b,c,d){var e=this,f=[],g=angular.isDefined(b.max)?a.$parent.$eval(b.max):c.max,h=angular.isDefined(b.animate)?a.$parent.$eval(b.animate):c.animate;this.addBar=function(a,b){var c=0,d=a.$parent.$index;angular.isDefined(d)&&f[d]&&(c=f[d].value),f.push(a),this.update(b,a.value,c),a.$watch("value",function(a,c){a!==c&&e.update(b,a,c)}),a.$on("$destroy",function(){e.removeBar(a)})},this.update=function(a,b,c){var e=this.getPercentage(b);h?(a.css("width",this.getPercentage(c)+"%"),d(a,{width:e+"%"})):a.css({transition:"none",width:e+"%"})},this.removeBar=function(a){f.splice(f.indexOf(a),1)},this.getPercentage=function(a){return Math.round(100*a/g)}}]).directive("progress",function(){return{restrict:"EA",replace:!0,transclude:!0,controller:"ProgressController",require:"progress",scope:{},template:'
'}}).directive("bar",function(){return{restrict:"EA",replace:!0,transclude:!0,require:"^progress",scope:{value:"=",type:"@"},templateUrl:"template/progressbar/bar.html",link:function(a,b,c,d){d.addBar(a,b)}}}).directive("progressbar",function(){return{restrict:"EA",replace:!0,transclude:!0,controller:"ProgressController",scope:{value:"=",type:"@"},templateUrl:"template/progressbar/progressbar.html",link:function(a,b,c,d){d.addBar(a,angular.element(b.children()[0]))}}}),angular.module("ui.bootstrap.rating",[]).constant("ratingConfig",{max:5,stateOn:null,stateOff:null}).controller("RatingController",["$scope","$attrs","$parse","ratingConfig",function(a,b,c,d){this.maxRange=angular.isDefined(b.max)?a.$parent.$eval(b.max):d.max,this.stateOn=angular.isDefined(b.stateOn)?a.$parent.$eval(b.stateOn):d.stateOn,this.stateOff=angular.isDefined(b.stateOff)?a.$parent.$eval(b.stateOff):d.stateOff,this.createRateObjects=function(a){for(var b={stateOn:this.stateOn,stateOff:this.stateOff},c=0,d=a.length;d>c;c++)a[c]=angular.extend({index:c},b,a[c]);return a},a.range=angular.isDefined(b.ratingStates)?this.createRateObjects(angular.copy(a.$parent.$eval(b.ratingStates))):this.createRateObjects(new Array(this.maxRange)),a.rate=function(b){a.value===b||a.readonly||(a.value=b) +},a.enter=function(b){a.readonly||(a.val=b),a.onHover({value:b})},a.reset=function(){a.val=angular.copy(a.value),a.onLeave()},a.$watch("value",function(b){a.val=b}),a.readonly=!1,b.readonly&&a.$parent.$watch(c(b.readonly),function(b){a.readonly=!!b})}]).directive("rating",function(){return{restrict:"EA",scope:{value:"=",onHover:"&",onLeave:"&"},controller:"RatingController",templateUrl:"template/rating/rating.html",replace:!0}}),angular.module("ui.bootstrap.tabs",[]).controller("TabsetController",["$scope",function(a){var b=this,c=b.tabs=a.tabs=[];b.select=function(a){angular.forEach(c,function(a){a.active=!1}),a.active=!0},b.addTab=function(a){c.push(a),(1===c.length||a.active)&&b.select(a)},b.removeTab=function(a){var d=c.indexOf(a);if(a.active&&c.length>1){var e=d==c.length-1?d-1:d+1;b.select(c[e])}c.splice(d,1)}}]).directive("tabset",function(){return{restrict:"EA",transclude:!0,replace:!0,scope:{},controller:"TabsetController",templateUrl:"template/tabs/tabset.html",link:function(a,b,c){a.vertical=angular.isDefined(c.vertical)?a.$parent.$eval(c.vertical):!1,a.justified=angular.isDefined(c.justified)?a.$parent.$eval(c.justified):!1,a.type=angular.isDefined(c.type)?a.$parent.$eval(c.type):"tabs"}}}).directive("tab",["$parse",function(a){return{require:"^tabset",restrict:"EA",replace:!0,templateUrl:"template/tabs/tab.html",transclude:!0,scope:{heading:"@",onSelect:"&select",onDeselect:"&deselect"},controller:function(){},compile:function(b,c,d){return function(b,c,e,f){var g,h;e.active?(g=a(e.active),h=g.assign,b.$parent.$watch(g,function(a,c){a!==c&&(b.active=!!a)}),b.active=g(b.$parent)):h=g=angular.noop,b.$watch("active",function(a){h(b.$parent,a),a?(f.select(b),b.onSelect()):b.onDeselect()}),b.disabled=!1,e.disabled&&b.$parent.$watch(a(e.disabled),function(a){b.disabled=!!a}),b.select=function(){b.disabled||(b.active=!0)},f.addTab(b),b.$on("$destroy",function(){f.removeTab(b)}),b.$transcludeFn=d}}}}]).directive("tabHeadingTransclude",[function(){return{restrict:"A",require:"^tab",link:function(a,b){a.$watch("headingElement",function(a){a&&(b.html(""),b.append(a))})}}}]).directive("tabContentTransclude",function(){function a(a){return a.tagName&&(a.hasAttribute("tab-heading")||a.hasAttribute("data-tab-heading")||"tab-heading"===a.tagName.toLowerCase()||"data-tab-heading"===a.tagName.toLowerCase())}return{restrict:"A",require:"^tabset",link:function(b,c,d){var e=b.$eval(d.tabContentTransclude);e.$transcludeFn(e.$parent,function(b){angular.forEach(b,function(b){a(b)?e.headingElement=b:c.append(b)})})}}}),angular.module("ui.bootstrap.timepicker",[]).constant("timepickerConfig",{hourStep:1,minuteStep:1,showMeridian:!0,meridians:null,readonlyInput:!1,mousewheel:!0}).directive("timepicker",["$parse","$log","timepickerConfig","$locale",function(a,b,c,d){return{restrict:"EA",require:"?^ngModel",replace:!0,scope:{},templateUrl:"template/timepicker/timepicker.html",link:function(e,f,g,h){function i(){var a=parseInt(e.hours,10),b=e.showMeridian?a>0&&13>a:a>=0&&24>a;return b?(e.showMeridian&&(12===a&&(a=0),e.meridian===q[1]&&(a+=12)),a):void 0}function j(){var a=parseInt(e.minutes,10);return a>=0&&60>a?a:void 0}function k(a){return angular.isDefined(a)&&a.toString().length<2?"0"+a:a}function l(a){m(),h.$setViewValue(new Date(p)),n(a)}function m(){h.$setValidity("time",!0),e.invalidHours=!1,e.invalidMinutes=!1}function n(a){var b=p.getHours(),c=p.getMinutes();e.showMeridian&&(b=0===b||12===b?12:b%12),e.hours="h"===a?b:k(b),e.minutes="m"===a?c:k(c),e.meridian=p.getHours()<12?q[0]:q[1]}function o(a){var b=new Date(p.getTime()+6e4*a);p.setHours(b.getHours(),b.getMinutes()),l()}if(h){var p=new Date,q=angular.isDefined(g.meridians)?e.$parent.$eval(g.meridians):c.meridians||d.DATETIME_FORMATS.AMPMS,r=c.hourStep;g.hourStep&&e.$parent.$watch(a(g.hourStep),function(a){r=parseInt(a,10)});var s=c.minuteStep;g.minuteStep&&e.$parent.$watch(a(g.minuteStep),function(a){s=parseInt(a,10)}),e.showMeridian=c.showMeridian,g.showMeridian&&e.$parent.$watch(a(g.showMeridian),function(a){if(e.showMeridian=!!a,h.$error.time){var b=i(),c=j();angular.isDefined(b)&&angular.isDefined(c)&&(p.setHours(b),l())}else n()});var t=f.find("input"),u=t.eq(0),v=t.eq(1),w=angular.isDefined(g.mousewheel)?e.$eval(g.mousewheel):c.mousewheel;if(w){var x=function(a){a.originalEvent&&(a=a.originalEvent);var b=a.wheelDelta?a.wheelDelta:-a.deltaY;return a.detail||b>0};u.bind("mousewheel wheel",function(a){e.$apply(x(a)?e.incrementHours():e.decrementHours()),a.preventDefault()}),v.bind("mousewheel wheel",function(a){e.$apply(x(a)?e.incrementMinutes():e.decrementMinutes()),a.preventDefault()})}if(e.readonlyInput=angular.isDefined(g.readonlyInput)?e.$eval(g.readonlyInput):c.readonlyInput,e.readonlyInput)e.updateHours=angular.noop,e.updateMinutes=angular.noop;else{var y=function(a,b){h.$setViewValue(null),h.$setValidity("time",!1),angular.isDefined(a)&&(e.invalidHours=a),angular.isDefined(b)&&(e.invalidMinutes=b)};e.updateHours=function(){var a=i();angular.isDefined(a)?(p.setHours(a),l("h")):y(!0)},u.bind("blur",function(){!e.validHours&&e.hours<10&&e.$apply(function(){e.hours=k(e.hours)})}),e.updateMinutes=function(){var a=j();angular.isDefined(a)?(p.setMinutes(a),l("m")):y(void 0,!0)},v.bind("blur",function(){!e.invalidMinutes&&e.minutes<10&&e.$apply(function(){e.minutes=k(e.minutes)})})}h.$render=function(){var a=h.$modelValue?new Date(h.$modelValue):null;isNaN(a)?(h.$setValidity("time",!1),b.error('Timepicker directive: "ng-model" value must be a Date object, a number of milliseconds since 01.01.1970 or a string representing an RFC2822 or ISO 8601 date.')):(a&&(p=a),m(),n())},e.incrementHours=function(){o(60*r)},e.decrementHours=function(){o(60*-r)},e.incrementMinutes=function(){o(s)},e.decrementMinutes=function(){o(-s)},e.toggleMeridian=function(){o(720*(p.getHours()<12?1:-1))}}}}}]),angular.module("ui.bootstrap.typeahead",["ui.bootstrap.position","ui.bootstrap.bindHtml"]).factory("typeaheadParser",["$parse",function(a){var b=/^\s*(.*?)(?:\s+as\s+(.*?))?\s+for\s+(?:([\$\w][\$\w\d]*))\s+in\s+(.*)$/;return{parse:function(c){var d=c.match(b);if(!d)throw new Error("Expected typeahead specification in form of '_modelValue_ (as _label_)? for _item_ in _collection_' but got '"+c+"'.");return{itemName:d[3],source:a(d[4]),viewMapper:a(d[2]||d[1]),modelMapper:a(d[1])}}}}]).directive("typeahead",["$compile","$parse","$q","$timeout","$document","$position","typeaheadParser",function(a,b,c,d,e,f,g){var h=[9,13,27,38,40];return{require:"ngModel",link:function(i,j,k,l){var m,n=i.$eval(k.typeaheadMinLength)||1,o=i.$eval(k.typeaheadWaitMs)||0,p=i.$eval(k.typeaheadEditable)!==!1,q=b(k.typeaheadLoading).assign||angular.noop,r=b(k.typeaheadOnSelect),s=k.typeaheadInputFormatter?b(k.typeaheadInputFormatter):void 0,t=k.typeaheadAppendToBody?b(k.typeaheadAppendToBody):!1,u=b(k.ngModel).assign,v=g.parse(k.typeahead),w=angular.element("
");w.attr({matches:"matches",active:"activeIdx",select:"select(activeIdx)",query:"query",position:"position"}),angular.isDefined(k.typeaheadTemplateUrl)&&w.attr("template-url",k.typeaheadTemplateUrl);var x=i.$new();i.$on("$destroy",function(){x.$destroy()});var y=function(){x.matches=[],x.activeIdx=-1},z=function(a){var b={$viewValue:a};q(i,!0),c.when(v.source(i,b)).then(function(c){if(a===l.$viewValue&&m){if(c.length>0){x.activeIdx=0,x.matches.length=0;for(var d=0;d=n?o>0?(A&&d.cancel(A),A=d(function(){z(a)},o)):z(a):(q(i,!1),y()),p?a:a?(l.$setValidity("editable",!1),void 0):(l.$setValidity("editable",!0),a)}),l.$formatters.push(function(a){var b,c,d={};return s?(d.$model=a,s(i,d)):(d[v.itemName]=a,b=v.viewMapper(i,d),d[v.itemName]=void 0,c=v.viewMapper(i,d),b!==c?b:a)}),x.select=function(a){var b,c,d={};d[v.itemName]=c=x.matches[a].model,b=v.modelMapper(i,d),u(i,b),l.$setValidity("editable",!0),r(i,{$item:c,$model:b,$label:v.viewMapper(i,d)}),y(),j[0].focus()},j.bind("keydown",function(a){0!==x.matches.length&&-1!==h.indexOf(a.which)&&(a.preventDefault(),40===a.which?(x.activeIdx=(x.activeIdx+1)%x.matches.length,x.$digest()):38===a.which?(x.activeIdx=(x.activeIdx?x.activeIdx:x.matches.length)-1,x.$digest()):13===a.which||9===a.which?x.$apply(function(){x.select(x.activeIdx)}):27===a.which&&(a.stopPropagation(),y(),x.$digest()))}),j.bind("blur",function(){m=!1});var B=function(a){j[0]!==a.target&&(y(),x.$digest())};e.bind("click",B),i.$on("$destroy",function(){e.unbind("click",B)});var C=a(w)(x);t?e.find("body").append(C):j.after(C)}}}]).directive("typeaheadPopup",function(){return{restrict:"EA",scope:{matches:"=",query:"=",active:"=",position:"=",select:"&"},replace:!0,templateUrl:"template/typeahead/typeahead-popup.html",link:function(a,b,c){a.templateUrl=c.templateUrl,a.isOpen=function(){return a.matches.length>0},a.isActive=function(b){return a.active==b},a.selectActive=function(b){a.active=b},a.selectMatch=function(b){a.select({activeIdx:b})}}}}).directive("typeaheadMatch",["$http","$templateCache","$compile","$parse",function(a,b,c,d){return{restrict:"EA",scope:{index:"=",match:"=",query:"="},link:function(e,f,g){var h=d(g.templateUrl)(e.$parent)||"template/typeahead/typeahead-match.html";a.get(h,{cache:b}).success(function(a){f.replaceWith(c(a.trim())(e))})}}}]).filter("typeaheadHighlight",function(){function a(a){return a.replace(/([.?*+^$[\]\\(){}|-])/g,"\\$1")}return function(b,c){return c?b.replace(new RegExp(a(c),"gi"),"$&"):b}}),angular.module("template/accordion/accordion-group.html",[]).run(["$templateCache",function(a){a.put("template/accordion/accordion-group.html",'
\n
\n

\n {{heading}}\n

\n
\n
\n
\n
\n
')}]),angular.module("template/accordion/accordion.html",[]).run(["$templateCache",function(a){a.put("template/accordion/accordion.html",'
')}]),angular.module("template/alert/alert.html",[]).run(["$templateCache",function(a){a.put("template/alert/alert.html","
\n \n
\n
\n")}]),angular.module("template/carousel/carousel.html",[]).run(["$templateCache",function(a){a.put("template/carousel/carousel.html",'\n')}]),angular.module("template/carousel/slide.html",[]).run(["$templateCache",function(a){a.put("template/carousel/slide.html","
\n")}]),angular.module("template/datepicker/datepicker.html",[]).run(["$templateCache",function(a){a.put("template/datepicker/datepicker.html",'\n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n
#{{label}}
{{ getWeekNumber(row) }}\n \n
\n')}]),angular.module("template/datepicker/popup.html",[]).run(["$templateCache",function(a){a.put("template/datepicker/popup.html","
    \n
  • \n"+'
  • \n \n \n \n \n \n \n
  • \n
\n')}]),angular.module("template/modal/backdrop.html",[]).run(["$templateCache",function(a){a.put("template/modal/backdrop.html",'')}]),angular.module("template/modal/window.html",[]).run(["$templateCache",function(a){a.put("template/modal/window.html",'')}]),angular.module("template/pagination/pager.html",[]).run(["$templateCache",function(a){a.put("template/pagination/pager.html",'')}]),angular.module("template/pagination/pagination.html",[]).run(["$templateCache",function(a){a.put("template/pagination/pagination.html",'')}]),angular.module("template/tooltip/tooltip-html-unsafe-popup.html",[]).run(["$templateCache",function(a){a.put("template/tooltip/tooltip-html-unsafe-popup.html",'
\n
\n
\n
\n')}]),angular.module("template/tooltip/tooltip-popup.html",[]).run(["$templateCache",function(a){a.put("template/tooltip/tooltip-popup.html",'
\n
\n
\n
\n')}]),angular.module("template/popover/popover.html",[]).run(["$templateCache",function(a){a.put("template/popover/popover.html",'
\n
\n\n
\n

\n
\n
\n
\n')}]),angular.module("template/progressbar/bar.html",[]).run(["$templateCache",function(a){a.put("template/progressbar/bar.html",'
')}]),angular.module("template/progressbar/progress.html",[]).run(["$templateCache",function(a){a.put("template/progressbar/progress.html",'
')}]),angular.module("template/progressbar/progressbar.html",[]).run(["$templateCache",function(a){a.put("template/progressbar/progressbar.html",'
')}]),angular.module("template/rating/rating.html",[]).run(["$templateCache",function(a){a.put("template/rating/rating.html",'\n \n')}]),angular.module("template/tabs/tab.html",[]).run(["$templateCache",function(a){a.put("template/tabs/tab.html",'
  • \n {{heading}}\n
  • \n')}]),angular.module("template/tabs/tabset-titles.html",[]).run(["$templateCache",function(a){a.put("template/tabs/tabset-titles.html","
      \n
    \n")}]),angular.module("template/tabs/tabset.html",[]).run(["$templateCache",function(a){a.put("template/tabs/tabset.html",'\n
    \n \n
    \n
    \n
    \n
    \n
    \n')}]),angular.module("template/timepicker/timepicker.html",[]).run(["$templateCache",function(a){a.put("template/timepicker/timepicker.html",'\n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n
     
    \n \n :\n \n
     
    \n')}]),angular.module("template/typeahead/typeahead-match.html",[]).run(["$templateCache",function(a){a.put("template/typeahead/typeahead-match.html",'')}]),angular.module("template/typeahead/typeahead-popup.html",[]).run(["$templateCache",function(a){a.put("template/typeahead/typeahead-popup.html","
      \n"+'
    • \n
      \n
    • \n
    ')}]); +},{}],2:[function(require,module,exports){ +/* + AngularJS v1.2.16 + (c) 2010-2014 Google, Inc. http://angularjs.org + License: MIT +*/ +(function(p,h,q){'use strict';function E(a){var e=[];s(e,h.noop).chars(a);return e.join("")}function k(a){var e={};a=a.split(",");var d;for(d=0;d=c;d--)e.end&&e.end(f[d]);f.length=c}}var b,g,f=[],l=a;for(f.last=function(){return f[f.length-1]};a;){g=!0;if(f.last()&&x[f.last()])a=a.replace(RegExp("(.*)<\\s*\\/\\s*"+f.last()+"[^>]*>","i"),function(b,a){a=a.replace(H,"$1").replace(I,"$1");e.chars&&e.chars(r(a));return""}),c("",f.last());else{if(0===a.indexOf("\x3c!--"))b=a.indexOf("--",4),0<=b&&a.lastIndexOf("--\x3e",b)===b&&(e.comment&&e.comment(a.substring(4,b)),a=a.substring(b+3),g=!1);else if(y.test(a)){if(b=a.match(y))a= +a.replace(b[0],""),g=!1}else if(J.test(a)){if(b=a.match(z))a=a.substring(b[0].length),b[0].replace(z,c),g=!1}else K.test(a)&&(b=a.match(A))&&(a=a.substring(b[0].length),b[0].replace(A,d),g=!1);g&&(b=a.indexOf("<"),g=0>b?a:a.substring(0,b),a=0>b?"":a.substring(b),e.chars&&e.chars(r(g)))}if(a==l)throw L("badparse",a);l=a}c()}function r(a){if(!a)return"";var e=M.exec(a);a=e[1];var d=e[3];if(e=e[2])n.innerHTML=e.replace(//g,">")}function s(a,e){var d=!1,c=h.bind(a,a.push);return{start:function(a,g,f){a=h.lowercase(a);!d&&x[a]&&(d=a);d||!0!==C[a]||(c("<"),c(a),h.forEach(g,function(d,f){var g=h.lowercase(f),k="img"===a&&"src"===g||"background"===g;!0!==O[g]||!0===D[g]&&!e(d,k)||(c(" "),c(f),c('="'),c(B(d)),c('"'))}),c(f?"/>":">"))},end:function(a){a=h.lowercase(a);d||!0!==C[a]||(c(""));a==d&&(d=!1)},chars:function(a){d|| +c(B(a))}}}var L=h.$$minErr("$sanitize"),A=/^<\s*([\w:-]+)((?:\s+[\w:-]+(?:\s*=\s*(?:(?:"[^"]*")|(?:'[^']*')|[^>\s]+))?)*)\s*(\/?)\s*>/,z=/^<\s*\/\s*([\w:-]+)[^>]*>/,G=/([\w:-]+)(?:\s*=\s*(?:(?:"((?:[^"])*)")|(?:'((?:[^'])*)')|([^>\s]+)))?/g,K=/^]*?)>/i,I=/]/,d=/^mailto:/;return function(c,b){function g(a){a&&m.push(E(a))}function f(a,c){m.push("');g(c);m.push("")}if(!c)return c;for(var l,k=c,m=[],n,p;l=k.match(e);)n=l[0],l[2]==l[3]&&(n="mailto:"+n),p=l.index,g(k.substr(0,p)),f(n,l[0].replace(d,"")),k=k.substring(p+l[0].length);g(k);return a(m.join(""))}}])})(window,window.angular); +//# sourceMappingURL=angular-sanitize.min.js.map + +},{}],3:[function(require,module,exports){ +/** + * State-based routing for AngularJS + * @version v0.2.10 + * @link http://angular-ui.github.com/ + * @license MIT License, http://www.opensource.org/licenses/MIT + */ +"undefined"!=typeof module&&"undefined"!=typeof exports&&module.exports===exports&&(module.exports="ui.router"),function(a,b,c){"use strict";function d(a,b){return I(new(I(function(){},{prototype:a})),b)}function e(a){return H(arguments,function(b){b!==a&&H(b,function(b,c){a.hasOwnProperty(c)||(a[c]=b)})}),a}function f(a,b){var c=[];for(var d in a.path){if(a.path[d]!==b.path[d])break;c.push(a.path[d])}return c}function g(a,b){if(Array.prototype.indexOf)return a.indexOf(b,Number(arguments[2])||0);var c=a.length>>>0,d=Number(arguments[2])||0;for(d=0>d?Math.ceil(d):Math.floor(d),0>d&&(d+=c);c>d;d++)if(d in a&&a[d]===b)return d;return-1}function h(a,b,c,d){var e,h=f(c,d),i={},j=[];for(var k in h)if(h[k].params&&h[k].params.length){e=h[k].params;for(var l in e)g(j,e[l])>=0||(j.push(e[l]),i[e[l]]=a[e[l]])}return I({},i,b)}function i(a,b){var c={};return H(a,function(a){var d=b[a];c[a]=null!=d?String(d):null}),c}function j(a,b,c){if(!c){c=[];for(var d in a)c.push(d)}for(var e=0;e "));if(o[c]=d,E(a))m.push(c,[function(){return b.get(a)}],h);else{var e=b.annotate(a);H(e,function(a){a!==c&&g.hasOwnProperty(a)&&k(g[a],a)}),m.push(c,a,e)}n.pop(),o[c]=f}}function l(a){return F(a)&&a.then&&a.$$promises}if(!F(g))throw new Error("'invocables' must be an object");var m=[],n=[],o={};return H(g,k),g=n=o=null,function(d,f,g){function h(){--s||(t||e(r,f.$$values),p.$$values=r,p.$$promises=!0,o.resolve(r))}function k(a){p.$$failure=a,o.reject(a)}function n(c,e,f){function i(a){l.reject(a),k(a)}function j(){if(!C(p.$$failure))try{l.resolve(b.invoke(e,g,r)),l.promise.then(function(a){r[c]=a,h()},i)}catch(a){i(a)}}var l=a.defer(),m=0;H(f,function(a){q.hasOwnProperty(a)&&!d.hasOwnProperty(a)&&(m++,q[a].then(function(b){r[a]=b,--m||j()},i))}),m||j(),q[c]=l.promise}if(l(d)&&g===c&&(g=f,f=d,d=null),d){if(!F(d))throw new Error("'locals' must be an object")}else d=i;if(f){if(!l(f))throw new Error("'parent' must be a promise returned by $resolve.resolve()")}else f=j;var o=a.defer(),p=o.promise,q=p.$$promises={},r=I({},d),s=1+m.length/3,t=!1;if(C(f.$$failure))return k(f.$$failure),p;f.$$values?(t=e(r,f.$$values),h()):(I(q,f.$$promises),f.then(h,k));for(var u=0,v=m.length;v>u;u+=3)d.hasOwnProperty(m[u])?h():n(m[u],m[u+1],m[u+2]);return p}},this.resolve=function(a,b,c,d){return this.study(a)(b,c,d)}}function m(a,b,c){this.fromConfig=function(a,b,c){return C(a.template)?this.fromString(a.template,b):C(a.templateUrl)?this.fromUrl(a.templateUrl,b):C(a.templateProvider)?this.fromProvider(a.templateProvider,b,c):null},this.fromString=function(a,b){return D(a)?a(b):a},this.fromUrl=function(c,d){return D(c)&&(c=c(d)),null==c?null:a.get(c,{cache:b}).then(function(a){return a.data})},this.fromProvider=function(a,b,d){return c.invoke(a,null,d||{params:b})}}function n(a){function b(b){if(!/^\w+(-+\w+)*$/.test(b))throw new Error("Invalid parameter name '"+b+"' in pattern '"+a+"'");if(f[b])throw new Error("Duplicate parameter name '"+b+"' in pattern '"+a+"'");f[b]=!0,j.push(b)}function c(a){return a.replace(/[\\\[\]\^$*+?.()|{}]/g,"\\$&")}var d,e=/([:*])(\w+)|\{(\w+)(?:\:((?:[^{}\\]+|\\.|\{(?:[^{}\\]+|\\.)*\})+))?\}/g,f={},g="^",h=0,i=this.segments=[],j=this.params=[];this.source=a;for(var k,l,m;(d=e.exec(a))&&(k=d[2]||d[3],l=d[4]||("*"==d[1]?".*":"[^/]*"),m=a.substring(h,d.index),!(m.indexOf("?")>=0));)g+=c(m)+"("+l+")",b(k),i.push(m),h=e.lastIndex;m=a.substring(h);var n=m.indexOf("?");if(n>=0){var o=this.sourceSearch=m.substring(n);m=m.substring(0,n),this.sourcePath=a.substring(0,h+n),H(o.substring(1).split(/[&?]/),b)}else this.sourcePath=a,this.sourceSearch="";g+=c(m)+"$",i.push(m),this.regexp=new RegExp(g),this.prefix=i[0]}function o(){this.compile=function(a){return new n(a)},this.isMatcher=function(a){return F(a)&&D(a.exec)&&D(a.format)&&D(a.concat)},this.$get=function(){return this}}function p(a){function b(a){var b=/^\^((?:\\[^a-zA-Z0-9]|[^\\\[\]\^$*+?.()|{}]+)*)/.exec(a.source);return null!=b?b[1].replace(/\\(.)/g,"$1"):""}function c(a,b){return a.replace(/\$(\$|\d{1,2})/,function(a,c){return b["$"===c?0:Number(c)]})}function d(a,b,c){if(!c)return!1;var d=a.invoke(b,b,{$match:c});return C(d)?d:!0}var e=[],f=null;this.rule=function(a){if(!D(a))throw new Error("'rule' must be a function");return e.push(a),this},this.otherwise=function(a){if(E(a)){var b=a;a=function(){return b}}else if(!D(a))throw new Error("'rule' must be a function");return f=a,this},this.when=function(e,f){var g,h=E(f);if(E(e)&&(e=a.compile(e)),!h&&!D(f)&&!G(f))throw new Error("invalid 'handler' in when()");var i={matcher:function(b,c){return h&&(g=a.compile(c),c=["$match",function(a){return g.format(a)}]),I(function(a,e){return d(a,c,b.exec(e.path(),e.search()))},{prefix:E(b.prefix)?b.prefix:""})},regex:function(a,e){if(a.global||a.sticky)throw new Error("when() RegExp must not be global or sticky");return h&&(g=e,e=["$match",function(a){return c(g,a)}]),I(function(b,c){return d(b,e,a.exec(c.path()))},{prefix:b(a)})}},j={matcher:a.isMatcher(e),regex:e instanceof RegExp};for(var k in j)if(j[k])return this.rule(i[k](e,f));throw new Error("invalid 'what' in when()")},this.$get=["$location","$rootScope","$injector",function(a,b,c){function d(b){function d(b){var d=b(c,a);return d?(E(d)&&a.replace().url(d),!0):!1}if(!b||!b.defaultPrevented){var g,h=e.length;for(g=0;h>g;g++)if(d(e[g]))return;f&&d(f)}}return b.$on("$locationChangeSuccess",d),{sync:function(){d()}}}]}function q(a,e,f){function g(a){return 0===a.indexOf(".")||0===a.indexOf("^")}function l(a,b){var d=E(a),e=d?a:a.name,f=g(e);if(f){if(!b)throw new Error("No reference point given for path '"+e+"'");for(var h=e.split("."),i=0,j=h.length,k=b;j>i;i++)if(""!==h[i]||0!==i){if("^"!==h[i])break;if(!k.parent)throw new Error("Path '"+e+"' not valid for state '"+b.name+"'");k=k.parent}else k=b;h=h.slice(i).join("."),e=k.name+(k.name&&h?".":"")+h}var l=w[e];return!l||!d&&(d||l!==a&&l.self!==a)?c:l}function m(a,b){x[a]||(x[a]=[]),x[a].push(b)}function n(b){b=d(b,{self:b,resolve:b.resolve||{},toString:function(){return this.name}});var c=b.name;if(!E(c)||c.indexOf("@")>=0)throw new Error("State must have a valid name");if(w.hasOwnProperty(c))throw new Error("State '"+c+"'' is already defined");var e=-1!==c.indexOf(".")?c.substring(0,c.lastIndexOf(".")):E(b.parent)?b.parent:"";if(e&&!w[e])return m(e,b.self);for(var f in z)D(z[f])&&(b[f]=z[f](b,z.$delegates[f]));if(w[c]=b,!b[y]&&b.url&&a.when(b.url,["$match","$stateParams",function(a,c){v.$current.navigable==b&&j(a,c)||v.transitionTo(b,a,{location:!1})}]),x[c])for(var g=0;g-1}function p(a){var b=a.split("."),c=v.$current.name.split(".");if("**"===b[0]&&(c=c.slice(c.indexOf(b[1])),c.unshift("**")),"**"===b[b.length-1]&&(c.splice(c.indexOf(b[b.length-2])+1,Number.MAX_VALUE),c.push("**")),b.length!=c.length)return!1;for(var d=0,e=b.length;e>d;d++)"*"===b[d]&&(c[d]="*");return c.join("")===b.join("")}function q(a,b){return E(a)&&!C(b)?z[a]:D(b)&&E(a)?(z[a]&&!z.$delegates[a]&&(z.$delegates[a]=z[a]),z[a]=b,this):this}function r(a,b){return F(a)?b=a:b.name=a,n(b),this}function s(a,e,g,m,n,q,r,s,x){function z(){r.url()!==M&&(r.url(M),r.replace())}function A(a,c,d,f,h){var i=d?c:k(a.params,c),j={$stateParams:i};h.resolve=n.resolve(a.resolve,j,h.resolve,a);var l=[h.resolve.then(function(a){h.globals=a})];return f&&l.push(f),H(a.views,function(c,d){var e=c.resolve&&c.resolve!==a.resolve?c.resolve:{};e.$template=[function(){return g.load(d,{view:c,locals:j,params:i,notify:!1})||""}],l.push(n.resolve(e,j,h.resolve,a).then(function(f){if(D(c.controllerProvider)||G(c.controllerProvider)){var g=b.extend({},e,j);f.$$controller=m.invoke(c.controllerProvider,null,g)}else f.$$controller=c.controller;f.$$state=a,f.$$controllerAs=c.controllerAs,h[d]=f}))}),e.all(l).then(function(){return h})}var B=e.reject(new Error("transition superseded")),F=e.reject(new Error("transition prevented")),K=e.reject(new Error("transition aborted")),L=e.reject(new Error("transition failed")),M=r.url(),N=x.baseHref();return u.locals={resolve:null,globals:{$stateParams:{}}},v={params:{},current:u.self,$current:u,transition:null},v.reload=function(){v.transitionTo(v.current,q,{reload:!0,inherit:!1,notify:!1})},v.go=function(a,b,c){return this.transitionTo(a,b,I({inherit:!0,relative:v.$current},c))},v.transitionTo=function(b,c,f){c=c||{},f=I({location:!0,inherit:!1,relative:null,notify:!0,reload:!1,$retry:!1},f||{});var g,k=v.$current,n=v.params,o=k.path,p=l(b,f.relative);if(!C(p)){var s={to:b,toParams:c,options:f};if(g=a.$broadcast("$stateNotFound",s,k.self,n),g.defaultPrevented)return z(),K;if(g.retry){if(f.$retry)return z(),L;var w=v.transition=e.when(g.retry);return w.then(function(){return w!==v.transition?B:(s.options.$retry=!0,v.transitionTo(s.to,s.toParams,s.options))},function(){return K}),z(),w}if(b=s.to,c=s.toParams,f=s.options,p=l(b,f.relative),!C(p)){if(f.relative)throw new Error("Could not resolve '"+b+"' from state '"+f.relative+"'");throw new Error("No such state '"+b+"'")}}if(p[y])throw new Error("Cannot transition to abstract state '"+b+"'");f.inherit&&(c=h(q,c||{},v.$current,p)),b=p;var x,D,E=b.path,G=u.locals,H=[];for(x=0,D=E[x];D&&D===o[x]&&j(c,n,D.ownParams)&&!f.reload;x++,D=E[x])G=H[x]=D.locals;if(t(b,k,G,f))return b.self.reloadOnSearch!==!1&&z(),v.transition=null,e.when(v.current);if(c=i(b.params,c||{}),f.notify&&(g=a.$broadcast("$stateChangeStart",b.self,c,k.self,n),g.defaultPrevented))return z(),F;for(var N=e.when(G),O=x;O=x;d--)g=o[d],g.self.onExit&&m.invoke(g.self.onExit,g.self,g.locals.globals),g.locals=null;for(d=x;d1||b.ctrlKey||b.metaKey||b.shiftKey||f.attr("target")||(c(function(){a.go(i.state,j,o)}),b.preventDefault())})}}}function y(a,b,c){return{restrict:"A",controller:["$scope","$element","$attrs",function(d,e,f){function g(){a.$current.self===i&&h()?e.addClass(l):e.removeClass(l)}function h(){return!k||j(k,b)}var i,k,l;l=c(f.uiSrefActive||"",!1)(d),this.$$setStateInfo=function(b,c){i=a.get(b,w(e)),k=c,g()},d.$on("$stateChangeSuccess",g)}]}}function z(a){return function(b){return a.is(b)}}function A(a){return function(b){return a.includes(b)}}function B(a,b){function e(a){this.locals=a.locals.globals,this.params=this.locals.$stateParams}function f(){this.locals=null,this.params=null}function g(c,g){if(null!=g.redirectTo){var h,j=g.redirectTo;if(E(j))h=j;else{if(!D(j))throw new Error("Invalid 'redirectTo' in when()");h=function(a,b){return j(a,b.path(),b.search())}}b.when(c,h)}else a.state(d(g,{parent:null,name:"route:"+encodeURIComponent(c),url:c,onEnter:e,onExit:f}));return i.push(g),this}function h(a,b,d){function e(a){return""!==a.name?a:c}var f={routes:i,params:d,current:c};return b.$on("$stateChangeStart",function(a,c,d,f){b.$broadcast("$routeChangeStart",e(c),e(f))}),b.$on("$stateChangeSuccess",function(a,c,d,g){f.current=e(c),b.$broadcast("$routeChangeSuccess",e(c),e(g)),J(d,f.params)}),b.$on("$stateChangeError",function(a,c,d,f,g,h){b.$broadcast("$routeChangeError",e(c),e(f),h)}),f}var i=[];e.$inject=["$$state"],this.when=g,this.$get=h,h.$inject=["$state","$rootScope","$routeParams"]}var C=b.isDefined,D=b.isFunction,E=b.isString,F=b.isObject,G=b.isArray,H=b.forEach,I=b.extend,J=b.copy;b.module("ui.router.util",["ng"]),b.module("ui.router.router",["ui.router.util"]),b.module("ui.router.state",["ui.router.router","ui.router.util"]),b.module("ui.router",["ui.router.state"]),b.module("ui.router.compat",["ui.router"]),l.$inject=["$q","$injector"],b.module("ui.router.util").service("$resolve",l),m.$inject=["$http","$templateCache","$injector"],b.module("ui.router.util").service("$templateFactory",m),n.prototype.concat=function(a){return new n(this.sourcePath+a+this.sourceSearch)},n.prototype.toString=function(){return this.source},n.prototype.exec=function(a,b){var c=this.regexp.exec(a);if(!c)return null;var d,e=this.params,f=e.length,g=this.segments.length-1,h={};if(g!==c.length-1)throw new Error("Unbalanced capture group in route '"+this.source+"'");for(d=0;g>d;d++)h[e[d]]=c[d+1];for(;f>d;d++)h[e[d]]=b[e[d]];return h},n.prototype.parameters=function(){return this.params},n.prototype.format=function(a){var b=this.segments,c=this.params;if(!a)return b.join("");var d,e,f,g=b.length-1,h=c.length,i=b[0];for(d=0;g>d;d++)f=a[c[d]],null!=f&&(i+=encodeURIComponent(f)),i+=b[d+1];for(;h>d;d++)f=a[c[d]],null!=f&&(i+=(e?"&":"?")+c[d]+"="+encodeURIComponent(f),e=!0);return i},b.module("ui.router.util").provider("$urlMatcherFactory",o),p.$inject=["$urlMatcherFactoryProvider"],b.module("ui.router.router").provider("$urlRouter",p),q.$inject=["$urlRouterProvider","$urlMatcherFactoryProvider","$locationProvider"],b.module("ui.router.state").value("$stateParams",{}).provider("$state",q),r.$inject=[],b.module("ui.router.state").provider("$view",r),b.module("ui.router.state").provider("$uiViewScroll",s),t.$inject=["$state","$injector","$uiViewScroll"],u.$inject=["$compile","$controller","$state"],b.module("ui.router.state").directive("uiView",t),b.module("ui.router.state").directive("uiView",u),x.$inject=["$state","$timeout"],y.$inject=["$state","$stateParams","$interpolate"],b.module("ui.router.state").directive("uiSref",x).directive("uiSrefActive",y),z.$inject=["$state"],A.$inject=["$state"],b.module("ui.router.state").filter("isState",z).filter("includedByState",A),B.$inject=["$stateProvider","$urlRouterProvider"],b.module("ui.router.compat").provider("$route",B).directive("ngView",t)}(window,window.angular); +},{}],4:[function(require,module,exports){ +/* + AngularJS v1.2.16 + (c) 2010-2014 Google, Inc. http://angularjs.org + License: MIT +*/ +(function(O,U,s){'use strict';function t(b){return function(){var a=arguments[0],c,a="["+(b?b+":":"")+a+"] http://errors.angularjs.org/1.2.16/"+(b?b+"/":"")+a;for(c=1;c").append(b).html();try{return 3===b[0].nodeType?K(c):c.match(/^(<[^>]+>)/)[1].replace(/^<([\w\-]+)/, +function(a,b){return"<"+K(b)})}catch(d){return K(c)}}function Xb(b){try{return decodeURIComponent(b)}catch(a){}}function Yb(b){var a={},c,d;q((b||"").split("&"),function(b){b&&(c=b.split("="),d=Xb(c[0]),B(d)&&(b=B(c[1])?Xb(c[1]):!0,a[d]?M(a[d])?a[d].push(b):a[d]=[a[d],b]:a[d]=b))});return a}function Zb(b){var a=[];q(b,function(b,d){M(b)?q(b,function(b){a.push(za(d,!0)+(!0===b?"":"="+za(b,!0)))}):a.push(za(d,!0)+(!0===b?"":"="+za(b,!0)))});return a.length?a.join("&"):""}function wb(b){return za(b, +!0).replace(/%26/gi,"&").replace(/%3D/gi,"=").replace(/%2B/gi,"+")}function za(b,a){return encodeURIComponent(b).replace(/%40/gi,"@").replace(/%3A/gi,":").replace(/%24/g,"$").replace(/%2C/gi,",").replace(/%20/g,a?"%20":"+")}function Wc(b,a){function c(a){a&&d.push(a)}var d=[b],e,g,f=["ng:app","ng-app","x-ng-app","data-ng-app"],h=/\sng[:\-]app(:\s*([\w\d_]+);?)?\s/;q(f,function(a){f[a]=!0;c(U.getElementById(a));a=a.replace(":","\\:");b.querySelectorAll&&(q(b.querySelectorAll("."+a),c),q(b.querySelectorAll("."+ +a+"\\:"),c),q(b.querySelectorAll("["+a+"]"),c))});q(d,function(a){if(!e){var b=h.exec(" "+a.className+" ");b?(e=a,g=(b[2]||"").replace(/\s+/g,",")):q(a.attributes,function(b){!e&&f[b.name]&&(e=a,g=b.value)})}});e&&a(e,g?[g]:[])}function $b(b,a){var c=function(){b=y(b);if(b.injector()){var c=b[0]===U?"document":ha(b);throw Pa("btstrpd",c);}a=a||[];a.unshift(["$provide",function(a){a.value("$rootElement",b)}]);a.unshift("ng");c=ac(a);c.invoke(["$rootScope","$rootElement","$compile","$injector","$animate", +function(a,b,c,d,e){a.$apply(function(){b.data("$injector",d);c(b)(a)})}]);return c},d=/^NG_DEFER_BOOTSTRAP!/;if(O&&!d.test(O.name))return c();O.name=O.name.replace(d,"");Ea.resumeBootstrap=function(b){q(b,function(b){a.push(b)});c()}}function fb(b,a){a=a||"_";return b.replace(Xc,function(b,d){return(d?a:"")+b.toLowerCase()})}function xb(b,a,c){if(!b)throw Pa("areq",a||"?",c||"required");return b}function Ra(b,a,c){c&&M(b)&&(b=b[b.length-1]);xb(P(b),a,"not a function, got "+(b&&"object"==typeof b? +b.constructor.name||"Object":typeof b));return b}function Aa(b,a){if("hasOwnProperty"===b)throw Pa("badname",a);}function bc(b,a,c){if(!a)return b;a=a.split(".");for(var d,e=b,g=a.length,f=0;f "+e[1]+a.replace(le,"<$1>")+e[2]; +d.removeChild(d.firstChild);for(a=e[0];a--;)d=d.lastChild;a=0;for(e=d.childNodes.length;a=S?(c.preventDefault=null,c.stopPropagation=null,c.isDefaultPrevented=null):(delete c.preventDefault,delete c.stopPropagation,delete c.isDefaultPrevented)};c.elem=b;return c}function Ia(b){var a=typeof b,c;"object"==a&&null!==b?"function"==typeof(c=b.$$hashKey)?c=b.$$hashKey():c===s&&(c=b.$$hashKey=bb()):c=b;return a+":"+c}function Va(b){q(b,this.put,this)}function oc(b){var a,c;"function"==typeof b?(a=b.$inject)||(a=[],b.length&&(c=b.toString().replace(oe, +""),c=c.match(pe),q(c[1].split(qe),function(b){b.replace(re,function(b,c,d){a.push(d)})})),b.$inject=a):M(b)?(c=b.length-1,Ra(b[c],"fn"),a=b.slice(0,c)):Ra(b,"fn",!0);return a}function ac(b){function a(a){return function(b,c){if(X(b))q(b,Rb(a));else return a(b,c)}}function c(a,b){Aa(a,"service");if(P(b)||M(b))b=n.instantiate(b);if(!b.$get)throw Wa("pget",a);return m[a+h]=b}function d(a,b){return c(a,{$get:b})}function e(a){var b=[],c,d,g,h;q(a,function(a){if(!k.get(a)){k.put(a,!0);try{if(w(a))for(c= +Sa(a),b=b.concat(e(c.requires)).concat(c._runBlocks),d=c._invokeQueue,g=0,h=d.length;g 4096 bytes)!"));else{if(l.cookie!==da)for(da=l.cookie,d=da.split("; "),Q={},g=0;gk&&this.remove(p.key),b},get:function(a){if(k").parent()[0])});var g=L(a,b,a,c,d,e);ma(a,"ng-scope");return function(b,c,d){xb(b,"scope");var e=c?Ja.clone.call(a):a;q(d,function(a,b){e.data("$"+b+"Controller",a)});d=0;for(var f=e.length;darguments.length&& +(b=a,a=s);D&&(c=lb);return p(a,b,c)}var I,x,v,A,R,H,lb={},da;I=c===g?d:Ub(d,new Hb(y(g),d.$attr));x=I.$$element;if(Q){var T=/^\s*([@=&])(\??)\s*(\w*)\s*$/;f=y(g);H=e.$new(!0);ia&&ia===Q.$$originalDirective?f.data("$isolateScope",H):f.data("$isolateScopeNoTemplate",H);ma(f,"ng-isolate-scope");q(Q.scope,function(a,c){var d=a.match(T)||[],g=d[3]||c,f="?"==d[2],d=d[1],l,m,n,p;H.$$isolateBindings[c]=d+g;switch(d){case "@":I.$observe(g,function(a){H[c]=a});I.$$observers[g].$$scope=e;I[g]&&(H[c]=b(I[g])(e)); +break;case "=":if(f&&!I[g])break;m=r(I[g]);p=m.literal?xa:function(a,b){return a===b};n=m.assign||function(){l=H[c]=m(e);throw ja("nonassign",I[g],Q.name);};l=H[c]=m(e);H.$watch(function(){var a=m(e);p(a,H[c])||(p(a,l)?n(e,a=H[c]):H[c]=a);return l=a},null,m.literal);break;case "&":m=r(I[g]);H[c]=function(a){return m(e,a)};break;default:throw ja("iscp",Q.name,c,a);}})}da=p&&u;L&&q(L,function(a){var b={$scope:a===Q||a.$$isolateScope?H:e,$element:x,$attrs:I,$transclude:da},c;R=a.controller;"@"==R&&(R= +I[a.name]);c=z(R,b);lb[a.name]=c;D||x.data("$"+a.name+"Controller",c);a.controllerAs&&(b.$scope[a.controllerAs]=c)});f=0;for(v=l.length;fG.priority)break;if(V=G.scope)A=A||G,G.templateUrl||(K("new/isolated scope",Q,G,Z),X(V)&&(Q=G));t=G.name;!G.templateUrl&&G.controller&&(V=G.controller,L=L||{},K("'"+t+"' controller",L[t],G,Z),L[t]=G);if(V=G.transclude)E=!0,G.$$tlb||(K("transclusion",T,G,Z),T=G),"element"==V?(D=!0,v=G.priority, +V=H(c,ra,W),Z=d.$$element=y(U.createComment(" "+t+": "+d[t]+" ")),c=Z[0],mb(g,y(ya.call(V,0)),c),Xa=x(V,e,v,f&&f.name,{nonTlbTranscludeDirective:T})):(V=y(Eb(c)).contents(),Z.empty(),Xa=x(V,e));if(G.template)if(K("template",ia,G,Z),ia=G,V=P(G.template)?G.template(Z,d):G.template,V=Y(V),G.replace){f=G;V=Cb.test(V)?y(V):[];c=V[0];if(1!=V.length||1!==c.nodeType)throw ja("tplrt",t,"");mb(g,Z,c);S={$attr:{}};V=da(c,[],S);var $=a.splice(N+1,a.length-(N+1));Q&&pc(V);a=a.concat(V).concat($);B(d,S);S=a.length}else Z.html(V); +if(G.templateUrl)K("template",ia,G,Z),ia=G,G.replace&&(f=G),J=C(a.splice(N,a.length-N),Z,d,g,Xa,l,n,{controllerDirectives:L,newIsolateScopeDirective:Q,templateDirective:ia,nonTlbTranscludeDirective:T}),S=a.length;else if(G.compile)try{O=G.compile(Z,d,Xa),P(O)?u(null,O,ra,W):O&&u(O.pre,O.post,ra,W)}catch(aa){m(aa,ha(Z))}G.terminal&&(J.terminal=!0,v=Math.max(v,G.priority))}J.scope=A&&!0===A.scope;J.transclude=E&&Xa;p.hasElementTranscludeDirective=D;return J}function pc(a){for(var b=0,c=a.length;bp.priority)&&-1!=p.restrict.indexOf(g)&&(n&&(p=Tb(p,{$$start:n,$$end:r})),b.push(p),k=p)}catch(F){m(F)}}return k}function B(a,b){var c=b.$attr,d=a.$attr,e=a.$$element;q(a,function(d,e){"$"!=e.charAt(0)&&(b[e]&&(d+=("style"===e?";":" ")+b[e]),a.$set(e,d,!0,c[e]))});q(b,function(b,g){"class"==g?(ma(e,b),a["class"]=(a["class"]? +a["class"]+" ":"")+b):"style"==g?(e.attr("style",e.attr("style")+";"+b),a.style=(a.style?a.style+";":"")+b):"$"==g.charAt(0)||a.hasOwnProperty(g)||(a[g]=b,d[g]=c[g])})}function C(a,b,c,d,e,g,f,l){var k=[],m,r,z=b[0],u=a.shift(),F=D({},u,{templateUrl:null,transclude:null,replace:null,$$originalDirective:u}),x=P(u.templateUrl)?u.templateUrl(b,c):u.templateUrl;b.empty();n.get(v.getTrustedResourceUrl(x),{cache:p}).success(function(n){var p,J;n=Y(n);if(u.replace){n=Cb.test(n)?y(n):[];p=n[0];if(1!=n.length|| +1!==p.nodeType)throw ja("tplrt",u.name,x);n={$attr:{}};mb(d,b,p);var v=da(p,[],n);X(u.scope)&&pc(v);a=v.concat(a);B(c,n)}else p=z,b.html(n);a.unshift(F);m=ia(a,p,c,e,b,u,g,f,l);q(d,function(a,c){a==p&&(d[c]=b[0])});for(r=L(b[0].childNodes,e);k.length;){n=k.shift();J=k.shift();var A=k.shift(),R=k.shift(),v=b[0];if(J!==z){var H=J.className;l.hasElementTranscludeDirective&&u.replace||(v=Eb(p));mb(A,y(J),v);ma(y(v),H)}J=m.transclude?Q(n,m.transclude):R;m(r,n,v,d,J)}k=null}).error(function(a,b,c,d){throw ja("tpload", +d.url);});return function(a,b,c,d,e){k?(k.push(b),k.push(c),k.push(d),k.push(e)):m(r,b,c,d,e)}}function E(a,b){var c=b.priority-a.priority;return 0!==c?c:a.name!==b.name?a.namea.status? +b:n.reject(b)}var d={method:"get",transformRequest:e.transformRequest,transformResponse:e.transformResponse},g=function(a){function b(a){var c;q(a,function(b,d){P(b)&&(c=b(),null!=c?a[d]=c:delete a[d])})}var c=e.headers,d=D({},a.headers),g,f,c=D({},c.common,c[K(a.method)]);b(c);b(d);a:for(g in c){a=K(g);for(f in d)if(K(f)===a)continue a;d[g]=c[g]}return d}(a);D(d,a);d.headers=g;d.method=Fa(d.method);(a=Ib(d.url)?b.cookies()[d.xsrfCookieName||e.xsrfCookieName]:s)&&(g[d.xsrfHeaderName||e.xsrfHeaderName]= +a);var f=[function(a){g=a.headers;var b=uc(a.data,tc(g),a.transformRequest);E(a.data)&&q(g,function(a,b){"content-type"===K(b)&&delete g[b]});E(a.withCredentials)&&!E(e.withCredentials)&&(a.withCredentials=e.withCredentials);return z(a,b,g).then(c,c)},s],h=n.when(d);for(q(v,function(a){(a.request||a.requestError)&&f.unshift(a.request,a.requestError);(a.response||a.responseError)&&f.push(a.response,a.responseError)});f.length;){a=f.shift();var k=f.shift(),h=h.then(a,k)}h.success=function(a){h.then(function(b){a(b.data, +b.status,b.headers,d)});return h};h.error=function(a){h.then(null,function(b){a(b.data,b.status,b.headers,d)});return h};return h}function z(b,c,g){function f(a,b,c,e){v&&(200<=a&&300>a?v.put(s,[a,b,sc(c),e]):v.remove(s));l(b,a,c,e);d.$$phase||d.$apply()}function l(a,c,d,e){c=Math.max(c,0);(200<=c&&300>c?p.resolve:p.reject)({data:a,status:c,headers:tc(d),config:b,statusText:e})}function k(){var a=db(r.pendingRequests,b);-1!==a&&r.pendingRequests.splice(a,1)}var p=n.defer(),z=p.promise,v,q,s=u(b.url, +b.params);r.pendingRequests.push(b);z.then(k,k);(b.cache||e.cache)&&(!1!==b.cache&&"GET"==b.method)&&(v=X(b.cache)?b.cache:X(e.cache)?e.cache:F);if(v)if(q=v.get(s),B(q)){if(q.then)return q.then(k,k),q;M(q)?l(q[1],q[0],ba(q[2]),q[3]):l(q,200,{},"OK")}else v.put(s,z);E(q)&&a(b.method,s,c,f,g,b.timeout,b.withCredentials,b.responseType);return z}function u(a,b){if(!b)return a;var c=[];Sc(b,function(a,b){null===a||E(a)||(M(a)||(a=[a]),q(a,function(a){X(a)&&(a=qa(a));c.push(za(b)+"="+za(a))}))});0=S&&(!b.match(/^(get|post|head|put|delete|options)$/i)||!O.XMLHttpRequest))return new O.ActiveXObject("Microsoft.XMLHTTP");if(O.XMLHttpRequest)return new O.XMLHttpRequest;throw t("$httpBackend")("noxhr");}function Ud(){this.$get=["$browser","$window","$document",function(b,a,c){return ve(b,ue,b.defer,a.angular.callbacks,c[0])}]}function ve(b,a,c,d,e){function g(a,b){var c=e.createElement("script"),d=function(){c.onreadystatechange= +c.onload=c.onerror=null;e.body.removeChild(c);b&&b()};c.type="text/javascript";c.src=a;S&&8>=S?c.onreadystatechange=function(){/loaded|complete/.test(c.readyState)&&d()}:c.onload=c.onerror=function(){d()};e.body.appendChild(c);return d}var f=-1;return function(e,l,k,m,n,p,r,z){function u(){v=f;A&&A();x&&x.abort()}function F(a,d,e,g,f){L&&c.cancel(L);A=x=null;0===d&&(d=e?200:"file"==sa(l).protocol?404:0);a(1223===d?204:d,e,g,f||"");b.$$completeOutstandingRequest(C)}var v;b.$$incOutstandingRequestCount(); +l=l||b.url();if("jsonp"==K(e)){var J="_"+(d.counter++).toString(36);d[J]=function(a){d[J].data=a};var A=g(l.replace("JSON_CALLBACK","angular.callbacks."+J),function(){d[J].data?F(m,200,d[J].data):F(m,v||-2);d[J]=Ea.noop})}else{var x=a(e);x.open(e,l,!0);q(n,function(a,b){B(a)&&x.setRequestHeader(b,a)});x.onreadystatechange=function(){if(x&&4==x.readyState){var a=null,b=null;v!==f&&(a=x.getAllResponseHeaders(),b="response"in x?x.response:x.responseText);F(m,v||x.status,b,a,x.statusText||"")}};r&&(x.withCredentials= +!0);if(z)try{x.responseType=z}catch(s){if("json"!==z)throw s;}x.send(k||null)}if(0=h&&(n.resolve(r),m(p.$$intervalId),delete e[p.$$intervalId]);z||b.$apply()},f);e[p.$$intervalId]=n;return p}var e={};d.cancel=function(a){return a&&a.$$intervalId in e?(e[a.$$intervalId].reject("canceled"),clearInterval(a.$$intervalId),delete e[a.$$intervalId], +!0):!1};return d}]}function ad(){this.$get=function(){return{id:"en-us",NUMBER_FORMATS:{DECIMAL_SEP:".",GROUP_SEP:",",PATTERNS:[{minInt:1,minFrac:0,maxFrac:3,posPre:"",posSuf:"",negPre:"-",negSuf:"",gSize:3,lgSize:3},{minInt:1,minFrac:2,maxFrac:2,posPre:"\u00a4",posSuf:"",negPre:"(\u00a4",negSuf:")",gSize:3,lgSize:3}],CURRENCY_SYM:"$"},DATETIME_FORMATS:{MONTH:"January February March April May June July August September October November December".split(" "),SHORTMONTH:"Jan Feb Mar Apr May Jun Jul Aug Sep Oct Nov Dec".split(" "), +DAY:"Sunday Monday Tuesday Wednesday Thursday Friday Saturday".split(" "),SHORTDAY:"Sun Mon Tue Wed Thu Fri Sat".split(" "),AMPMS:["AM","PM"],medium:"MMM d, y h:mm:ss a","short":"M/d/yy h:mm a",fullDate:"EEEE, MMMM d, y",longDate:"MMMM d, y",mediumDate:"MMM d, y",shortDate:"M/d/yy",mediumTime:"h:mm:ss a",shortTime:"h:mm a"},pluralCat:function(b){return 1===b?"one":"other"}}}}function wc(b){b=b.split("/");for(var a=b.length;a--;)b[a]=wb(b[a]);return b.join("/")}function xc(b,a,c){b=sa(b,c);a.$$protocol= +b.protocol;a.$$host=b.hostname;a.$$port=Y(b.port)||we[b.protocol]||null}function yc(b,a,c){var d="/"!==b.charAt(0);d&&(b="/"+b);b=sa(b,c);a.$$path=decodeURIComponent(d&&"/"===b.pathname.charAt(0)?b.pathname.substring(1):b.pathname);a.$$search=Yb(b.search);a.$$hash=decodeURIComponent(b.hash);a.$$path&&"/"!=a.$$path.charAt(0)&&(a.$$path="/"+a.$$path)}function oa(b,a){if(0===a.indexOf(b))return a.substr(b.length)}function Ya(b){var a=b.indexOf("#");return-1==a?b:b.substr(0,a)}function Jb(b){return b.substr(0, +Ya(b).lastIndexOf("/")+1)}function zc(b,a){this.$$html5=!0;a=a||"";var c=Jb(b);xc(b,this,b);this.$$parse=function(a){var e=oa(c,a);if(!w(e))throw Kb("ipthprfx",a,c);yc(e,this,b);this.$$path||(this.$$path="/");this.$$compose()};this.$$compose=function(){var a=Zb(this.$$search),b=this.$$hash?"#"+wb(this.$$hash):"";this.$$url=wc(this.$$path)+(a?"?"+a:"")+b;this.$$absUrl=c+this.$$url.substr(1)};this.$$rewrite=function(d){var e;if((e=oa(b,d))!==s)return d=e,(e=oa(a,e))!==s?c+(oa("/",e)||e):b+d;if((e=oa(c, +d))!==s)return c+e;if(c==d+"/")return c}}function Lb(b,a){var c=Jb(b);xc(b,this,b);this.$$parse=function(d){var e=oa(b,d)||oa(c,d),e="#"==e.charAt(0)?oa(a,e):this.$$html5?e:"";if(!w(e))throw Kb("ihshprfx",d,a);yc(e,this,b);d=this.$$path;var g=/^\/?.*?:(\/.*)/;0===e.indexOf(b)&&(e=e.replace(b,""));g.exec(e)||(d=(e=g.exec(d))?e[1]:d);this.$$path=d;this.$$compose()};this.$$compose=function(){var c=Zb(this.$$search),e=this.$$hash?"#"+wb(this.$$hash):"";this.$$url=wc(this.$$path)+(c?"?"+c:"")+e;this.$$absUrl= +b+(this.$$url?a+this.$$url:"")};this.$$rewrite=function(a){if(Ya(b)==Ya(a))return a}}function Ac(b,a){this.$$html5=!0;Lb.apply(this,arguments);var c=Jb(b);this.$$rewrite=function(d){var e;if(b==Ya(d))return d;if(e=oa(c,d))return b+a+e;if(c===d+"/")return c}}function nb(b){return function(){return this[b]}}function Bc(b,a){return function(c){if(E(c))return this[b];this[b]=a(c);this.$$compose();return this}}function Vd(){var b="",a=!1;this.hashPrefix=function(a){return B(a)?(b=a,this):b};this.html5Mode= +function(b){return B(b)?(a=b,this):a};this.$get=["$rootScope","$browser","$sniffer","$rootElement",function(c,d,e,g){function f(a){c.$broadcast("$locationChangeSuccess",h.absUrl(),a)}var h,l=d.baseHref(),k=d.url();a?(l=k.substring(0,k.indexOf("/",k.indexOf("//")+2))+(l||"/"),e=e.history?zc:Ac):(l=Ya(k),e=Lb);h=new e(l,"#"+b);h.$$parse(h.$$rewrite(k));g.on("click",function(a){if(!a.ctrlKey&&!a.metaKey&&2!=a.which){for(var b=y(a.target);"a"!==K(b[0].nodeName);)if(b[0]===g[0]||!(b=b.parent())[0])return; +var e=b.prop("href");X(e)&&"[object SVGAnimatedString]"===e.toString()&&(e=sa(e.animVal).href);var f=h.$$rewrite(e);e&&(!b.attr("target")&&f&&!a.isDefaultPrevented())&&(a.preventDefault(),f!=d.url()&&(h.$$parse(f),c.$apply(),O.angular["ff-684208-preventDefault"]=!0))}});h.absUrl()!=k&&d.url(h.absUrl(),!0);d.onUrlChange(function(a){h.absUrl()!=a&&(c.$evalAsync(function(){var b=h.absUrl();h.$$parse(a);c.$broadcast("$locationChangeStart",a,b).defaultPrevented?(h.$$parse(b),d.url(b)):f(b)}),c.$$phase|| +c.$digest())});var m=0;c.$watch(function(){var a=d.url(),b=h.$$replace;m&&a==h.absUrl()||(m++,c.$evalAsync(function(){c.$broadcast("$locationChangeStart",h.absUrl(),a).defaultPrevented?h.$$parse(a):(d.url(h.absUrl(),b),f(a))}));h.$$replace=!1;return m});return h}]}function Wd(){var b=!0,a=this;this.debugEnabled=function(a){return B(a)?(b=a,this):b};this.$get=["$window",function(c){function d(a){a instanceof Error&&(a.stack?a=a.message&&-1===a.stack.indexOf(a.message)?"Error: "+a.message+"\n"+a.stack: +a.stack:a.sourceURL&&(a=a.message+"\n"+a.sourceURL+":"+a.line));return a}function e(a){var b=c.console||{},e=b[a]||b.log||C;a=!1;try{a=!!e.apply}catch(l){}return a?function(){var a=[];q(arguments,function(b){a.push(d(b))});return e.apply(b,a)}:function(a,b){e(a,null==b?"":b)}}return{log:e("log"),info:e("info"),warn:e("warn"),error:e("error"),debug:function(){var c=e("debug");return function(){b&&c.apply(a,arguments)}}()}}]}function fa(b,a){if("constructor"===b)throw Ba("isecfld",a);return b}function Za(b, +a){if(b){if(b.constructor===b)throw Ba("isecfn",a);if(b.document&&b.location&&b.alert&&b.setInterval)throw Ba("isecwindow",a);if(b.children&&(b.nodeName||b.prop&&b.attr&&b.find))throw Ba("isecdom",a);}return b}function ob(b,a,c,d,e){e=e||{};a=a.split(".");for(var g,f=0;1e?Cc(d[0],d[1],d[2],d[3],d[4],c,a):function(b,g){var f=0,h;do h=Cc(d[f++],d[f++],d[f++],d[f++],d[f++],c,a)(b,g),g=s,b=h;while(fa)for(b in l++,e)e.hasOwnProperty(b)&&!d.hasOwnProperty(b)&&(q--,delete e[b])}else e!==d&&(e=d,l++);return l},function(){p?(p=!1,b(d,d,c)):b(d,f,c);if(h)if(X(d))if(ab(d)){f=Array(d.length);for(var a=0;as&&(y=4-s,Q[y]||(Q[y]=[]),H=P(d.exp)?"fn: "+(d.exp.name||d.exp.toString()):d.exp,H+="; newVal: "+qa(g)+"; oldVal: "+qa(f),Q[y].push(H));else if(d===c){x=!1;break a}}catch(w){p.$$phase= +null,e(w)}if(!(h=L.$$childHead||L!==this&&L.$$nextSibling))for(;L!==this&&!(h=L.$$nextSibling);)L=L.$parent}while(L=h);if((x||k.length)&&!s--)throw p.$$phase=null,a("infdig",b,qa(Q));}while(x||k.length);for(p.$$phase=null;m.length;)try{m.shift()()}catch(T){e(T)}},$destroy:function(){if(!this.$$destroyed){var a=this.$parent;this.$broadcast("$destroy");this.$$destroyed=!0;this!==p&&(q(this.$$listenerCount,eb(null,m,this)),a.$$childHead==this&&(a.$$childHead=this.$$nextSibling),a.$$childTail==this&& +(a.$$childTail=this.$$prevSibling),this.$$prevSibling&&(this.$$prevSibling.$$nextSibling=this.$$nextSibling),this.$$nextSibling&&(this.$$nextSibling.$$prevSibling=this.$$prevSibling),this.$parent=this.$$nextSibling=this.$$prevSibling=this.$$childHead=this.$$childTail=this.$root=null,this.$$listeners={},this.$$watchers=this.$$asyncQueue=this.$$postDigestQueue=[],this.$destroy=this.$digest=this.$apply=C,this.$on=this.$watch=function(){return C})}},$eval:function(a,b){return g(a)(this,b)},$evalAsync:function(a){p.$$phase|| +p.$$asyncQueue.length||f.defer(function(){p.$$asyncQueue.length&&p.$digest()});this.$$asyncQueue.push({scope:this,expression:a})},$$postDigest:function(a){this.$$postDigestQueue.push(a)},$apply:function(a){try{return l("$apply"),this.$eval(a)}catch(b){e(b)}finally{p.$$phase=null;try{p.$digest()}catch(c){throw e(c),c;}}},$on:function(a,b){var c=this.$$listeners[a];c||(this.$$listeners[a]=c=[]);c.push(b);var d=this;do d.$$listenerCount[a]||(d.$$listenerCount[a]=0),d.$$listenerCount[a]++;while(d=d.$parent); +var e=this;return function(){c[db(c,b)]=null;m(e,1,a)}},$emit:function(a,b){var c=[],d,g=this,f=!1,h={name:a,targetScope:g,stopPropagation:function(){f=!0},preventDefault:function(){h.defaultPrevented=!0},defaultPrevented:!1},l=[h].concat(ya.call(arguments,1)),k,m;do{d=g.$$listeners[a]||c;h.currentScope=g;k=0;for(m=d.length;kc.msieDocumentMode)throw ua("iequirks");var e=ba(ga);e.isEnabled=function(){return b};e.trustAs=d.trustAs;e.getTrusted=d.getTrusted;e.valueOf=d.valueOf;b||(e.trustAs=e.getTrusted=function(a,b){return b},e.valueOf=Da);e.parseAs=function(b,c){var d=a(c);return d.literal&&d.constant?d:function(a,c){return e.getTrusted(b, +d(a,c))}};var g=e.parseAs,f=e.getTrusted,h=e.trustAs;q(ga,function(a,b){var c=K(b);e[Ta("parse_as_"+c)]=function(b){return g(a,b)};e[Ta("get_trusted_"+c)]=function(b){return f(a,b)};e[Ta("trust_as_"+c)]=function(b){return h(a,b)}});return e}]}function be(){this.$get=["$window","$document",function(b,a){var c={},d=Y((/android (\d+)/.exec(K((b.navigator||{}).userAgent))||[])[1]),e=/Boxee/i.test((b.navigator||{}).userAgent),g=a[0]||{},f=g.documentMode,h,l=/^(Moz|webkit|O|ms)(?=[A-Z])/,k=g.body&&g.body.style, +m=!1,n=!1;if(k){for(var p in k)if(m=l.exec(p)){h=m[0];h=h.substr(0,1).toUpperCase()+h.substr(1);break}h||(h="WebkitOpacity"in k&&"webkit");m=!!("transition"in k||h+"Transition"in k);n=!!("animation"in k||h+"Animation"in k);!d||m&&n||(m=w(g.body.style.webkitTransition),n=w(g.body.style.webkitAnimation))}return{history:!(!b.history||!b.history.pushState||4>d||e),hashchange:"onhashchange"in b&&(!f||7b;b=Math.abs(b);var f=b+"",h="",l=[],k=!1;if(-1!==f.indexOf("e")){var m=f.match(/([\d\.]+)e(-?)(\d+)/);m&&"-"==m[2]&&m[3]>e+1?f="0":(h=f,k=!0)}if(k)0b)&&(h=b.toFixed(e)); +else{f=(f.split(Nc)[1]||"").length;E(e)&&(e=Math.min(Math.max(a.minFrac,f),a.maxFrac));f=Math.pow(10,e);b=Math.round(b*f)/f;b=(""+b).split(Nc);f=b[0];b=b[1]||"";var m=0,n=a.lgSize,p=a.gSize;if(f.length>=n+p)for(m=f.length-n,k=0;kb&&(d="-",b=-b);for(b=""+b;b.length-c)e+=c;0===e&&-12==c&&(e=12);return Ob(e,a,d)}}function pb(b,a){return function(c,d){var e=c["get"+b](),g=Fa(a?"SHORT"+b:b);return d[g][e]}}function Jc(b){function a(a){var b;if(b=a.match(c)){a=new Date(0);var g=0,f=0,h=b[8]?a.setUTCFullYear:a.setFullYear,l=b[8]?a.setUTCHours:a.setHours;b[9]&&(g=Y(b[9]+b[10]),f=Y(b[9]+b[11])); +h.call(a,Y(b[1]),Y(b[2])-1,Y(b[3]));g=Y(b[4]||0)-g;f=Y(b[5]||0)-f;h=Y(b[6]||0);b=Math.round(1E3*parseFloat("0."+(b[7]||0)));l.call(a,g,f,h,b)}return a}var c=/^(\d{4})-?(\d\d)-?(\d\d)(?:T(\d\d)(?::?(\d\d)(?::?(\d\d)(?:\.(\d+))?)?)?(Z|([+-])(\d\d):?(\d\d))?)?$/;return function(c,e){var g="",f=[],h,l;e=e||"mediumDate";e=b.DATETIME_FORMATS[e]||e;w(c)&&(c=Ge.test(c)?Y(c):a(c));vb(c)&&(c=new Date(c));if(!Na(c))return c;for(;e;)(l=He.exec(e))?(f=f.concat(ya.call(l,1)),e=f.pop()):(f.push(e),e=null);q(f,function(a){h= +Ie[a];g+=h?h(c,b.DATETIME_FORMATS):a.replace(/(^'|'$)/g,"").replace(/''/g,"'")});return g}}function Ce(){return function(b){return qa(b,!0)}}function De(){return function(b,a){if(!M(b)&&!w(b))return b;a=Y(a);if(w(b))return a?0<=a?b.slice(0,a):b.slice(a,b.length):"";var c=[],d,e;a>b.length?a=b.length:a<-b.length&&(a=-b.length);0a||37<=a&&40>=a)||m()});if(e.hasEvent("paste"))a.on("paste cut",m)}a.on("change",l);d.$render=function(){a.val(d.$isEmpty(d.$viewValue)? +"":d.$viewValue)};var n=c.ngPattern;n&&((e=n.match(/^\/(.*)\/([gim]*)$/))?(n=RegExp(e[1],e[2]),e=function(a){return pa(d,"pattern",d.$isEmpty(a)||n.test(a),a)}):e=function(c){var e=b.$eval(n);if(!e||!e.test)throw t("ngPattern")("noregexp",n,e,ha(a));return pa(d,"pattern",d.$isEmpty(c)||e.test(c),c)},d.$formatters.push(e),d.$parsers.push(e));if(c.ngMinlength){var p=Y(c.ngMinlength);e=function(a){return pa(d,"minlength",d.$isEmpty(a)||a.length>=p,a)};d.$parsers.push(e);d.$formatters.push(e)}if(c.ngMaxlength){var r= +Y(c.ngMaxlength);e=function(a){return pa(d,"maxlength",d.$isEmpty(a)||a.length<=r,a)};d.$parsers.push(e);d.$formatters.push(e)}}function Pb(b,a){b="ngClass"+b;return["$animate",function(c){function d(a,b){var c=[],d=0;a:for(;dS?function(b){b=b.nodeName?b:b[0];return b.scopeName&&"HTML"!=b.scopeName?Fa(b.scopeName+":"+b.nodeName):b.nodeName}:function(b){return b.nodeName?b.nodeName:b[0].nodeName};var Xc=/[A-Z]/g,$c={full:"1.2.16",major:1,minor:2,dot:16,codeName:"badger-enumeration"},Ua=N.cache={},gb=N.expando="ng-"+(new Date).getTime(), +me=1,Pc=O.document.addEventListener?function(b,a,c){b.addEventListener(a,c,!1)}:function(b,a,c){b.attachEvent("on"+a,c)},Fb=O.document.removeEventListener?function(b,a,c){b.removeEventListener(a,c,!1)}:function(b,a,c){b.detachEvent("on"+a,c)};N._data=function(b){return this.cache[b[this.expando]]||{}};var he=/([\:\-\_]+(.))/g,ie=/^moz([A-Z])/,Bb=t("jqLite"),je=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,Cb=/<|&#?\w+;/,ke=/<([\w:]+)/,le=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,ea= +{option:[1,'"],thead:[1,"","
    "],col:[2,"","
    "],tr:[2,"","
    "],td:[3,"","
    "],_default:[0,"",""]};ea.optgroup=ea.option;ea.tbody=ea.tfoot=ea.colgroup=ea.caption=ea.thead;ea.th=ea.td;var Ja=N.prototype={ready:function(b){function a(){c||(c=!0,b())}var c=!1;"complete"===U.readyState?setTimeout(a):(this.on("DOMContentLoaded",a),N(O).on("load",a))},toString:function(){var b= +[];q(this,function(a){b.push(""+a)});return"["+b.join(", ")+"]"},eq:function(b){return 0<=b?y(this[b]):y(this[this.length+b])},length:0,push:Ke,sort:[].sort,splice:[].splice},kb={};q("multiple selected checked disabled readOnly required open".split(" "),function(b){kb[K(b)]=b});var nc={};q("input select option textarea button form details".split(" "),function(b){nc[Fa(b)]=!0});q({data:jc,inheritedData:jb,scope:function(b){return y(b).data("$scope")||jb(b.parentNode||b,["$isolateScope","$scope"])}, +isolateScope:function(b){return y(b).data("$isolateScope")||y(b).data("$isolateScopeNoTemplate")},controller:kc,injector:function(b){return jb(b,"$injector")},removeAttr:function(b,a){b.removeAttribute(a)},hasClass:Gb,css:function(b,a,c){a=Ta(a);if(B(c))b.style[a]=c;else{var d;8>=S&&(d=b.currentStyle&&b.currentStyle[a],""===d&&(d="auto"));d=d||b.style[a];8>=S&&(d=""===d?s:d);return d}},attr:function(b,a,c){var d=K(a);if(kb[d])if(B(c))c?(b[a]=!0,b.setAttribute(a,d)):(b[a]=!1,b.removeAttribute(d)); +else return b[a]||(b.attributes.getNamedItem(a)||C).specified?d:s;else if(B(c))b.setAttribute(a,c);else if(b.getAttribute)return b=b.getAttribute(a,2),null===b?s:b},prop:function(b,a,c){if(B(c))b[a]=c;else return b[a]},text:function(){function b(b,d){var e=a[b.nodeType];if(E(d))return e?b[e]:"";b[e]=d}var a=[];9>S?(a[1]="innerText",a[3]="nodeValue"):a[1]=a[3]="textContent";b.$dv="";return b}(),val:function(b,a){if(E(a)){if("SELECT"===Ka(b)&&b.multiple){var c=[];q(b.options,function(a){a.selected&& +c.push(a.value||a.text)});return 0===c.length?null:c}return b.value}b.value=a},html:function(b,a){if(E(a))return b.innerHTML;for(var c=0,d=b.childNodes;c":function(a,c,d,e){return d(a,c)>e(a,c)},"<=":function(a,c,d,e){return d(a,c)<=e(a,c)},">=":function(a,c,d,e){return d(a,c)>=e(a,c)},"&&":function(a,c,d,e){return d(a,c)&&e(a,c)},"||":function(a,c,d,e){return d(a,c)||e(a,c)},"&":function(a,c,d,e){return d(a,c)&e(a,c)},"|":function(a,c,d,e){return e(a,c)(a,c,d(a,c))},"!":function(a,c,d){return!d(a,c)}},Ne={n:"\n",f:"\f",r:"\r",t:"\t",v:"\v","'":"'",'"':'"'}, +Nb=function(a){this.options=a};Nb.prototype={constructor:Nb,lex:function(a){this.text=a;this.index=0;this.ch=s;this.lastCh=":";this.tokens=[];var c;for(a=[];this.index=a},isWhitespace:function(a){return" "===a||"\r"===a||"\t"===a||"\n"===a||"\v"===a||"\u00a0"=== +a},isIdent:function(a){return"a"<=a&&"z">=a||"A"<=a&&"Z">=a||"_"===a||"$"===a},isExpOperator:function(a){return"-"===a||"+"===a||this.isNumber(a)},throwError:function(a,c,d){d=d||this.index;c=B(c)?"s "+c+"-"+this.index+" ["+this.text.substring(c,d)+"]":" "+d;throw Ba("lexerr",a,c,this.text);},readNumber:function(){for(var a="",c=this.index;this.index","<=",">="))a=this.binaryFn(a,c.fn,this.relational());return a},additive:function(){for(var a=this.multiplicative(),c;c=this.expect("+","-");)a=this.binaryFn(a,c.fn,this.multiplicative());return a},multiplicative:function(){for(var a=this.unary(),c;c=this.expect("*","/","%");)a=this.binaryFn(a,c.fn,this.unary());return a},unary:function(){var a;return this.expect("+")?this.primary():(a=this.expect("-"))?this.binaryFn($a.ZERO,a.fn, +this.unary()):(a=this.expect("!"))?this.unaryFn(a.fn,this.unary()):this.primary()},fieldAccess:function(a){var c=this,d=this.expect().text,e=Dc(d,this.options,this.text);return D(function(c,d,h){return e(h||a(c,d))},{assign:function(e,f,h){return ob(a(e,h),d,f,c.text,c.options)}})},objectIndex:function(a){var c=this,d=this.expression();this.consume("]");return D(function(e,g){var f=a(e,g),h=d(e,g),l;if(!f)return s;(f=Za(f[h],c.text))&&(f.then&&c.options.unwrapPromises)&&(l=f,"$$v"in f||(l.$$v=s,l.then(function(a){l.$$v= +a})),f=f.$$v);return f},{assign:function(e,g,f){var h=d(e,f);return Za(a(e,f),c.text)[h]=g}})},functionCall:function(a,c){var d=[];if(")"!==this.peekToken().text){do d.push(this.expression());while(this.expect(","))}this.consume(")");var e=this;return function(g,f){for(var h=[],l=c?c(g,f):g,k=0;ka.getHours()?c.AMPMS[0]:c.AMPMS[1]},Z:function(a){a=-1*a.getTimezoneOffset();return a=(0<=a?"+":"")+(Ob(Math[0=S&&(c.href||c.name||c.$set("href",""),a.append(U.createComment("IE fix")));if(!c.href&&!c.xlinkHref&&!c.name)return function(a,c){var g="[object SVGAnimatedString]"===wa.call(c.prop("href"))?"xlink:href":"href";c.on("click",function(a){c.attr(g)||a.preventDefault()})}}}),zb={};q(kb,function(a,c){if("multiple"!=a){var d=na("ng-"+c);zb[d]=function(){return{priority:100,link:function(a,g,f){a.$watch(f[d],function(a){f.$set(c,!!a)})}}}}});q(["src", +"srcset","href"],function(a){var c=na("ng-"+a);zb[c]=function(){return{priority:99,link:function(d,e,g){var f=a,h=a;"href"===a&&"[object SVGAnimatedString]"===wa.call(e.prop("href"))&&(h="xlinkHref",g.$attr[h]="xlink:href",f=null);g.$observe(c,function(a){a&&(g.$set(h,a),S&&f&&e.prop(f,g[h]))})}}}});var sb={$addControl:C,$removeControl:C,$setValidity:C,$setDirty:C,$setPristine:C};Oc.$inject=["$element","$attrs","$scope","$animate"];var Qc=function(a){return["$timeout",function(c){return{name:"form", +restrict:a?"EAC":"E",controller:Oc,compile:function(){return{pre:function(a,e,g,f){if(!g.action){var h=function(a){a.preventDefault?a.preventDefault():a.returnValue=!1};Pc(e[0],"submit",h);e.on("$destroy",function(){c(function(){Fb(e[0],"submit",h)},0,!1)})}var l=e.parent().controller("form"),k=g.name||g.ngForm;k&&ob(a,k,f,k);if(l)e.on("$destroy",function(){l.$removeControl(f);k&&ob(a,k,s,k);D(f,sb)})}}}}}]},dd=Qc(),qd=Qc(!0),Oe=/^(ftp|http|https):\/\/(\w+:{0,1}\w*@)?(\S+)(:[0-9]+)?(\/|\/([\w#!:.?+=&%@!\-\/]))?$/, +Pe=/^[a-z0-9!#$%&'*+/=?^_`{|}~.-]+@[a-z0-9-]+(\.[a-z0-9-]+)*$/i,Qe=/^\s*(\-|\+)?(\d+|(\d*(\.\d*)))\s*$/,Rc={text:ub,number:function(a,c,d,e,g,f){ub(a,c,d,e,g,f);e.$parsers.push(function(a){var c=e.$isEmpty(a);if(c||Qe.test(a))return e.$setValidity("number",!0),""===a?null:c?a:parseFloat(a);e.$setValidity("number",!1);return s});Je(e,"number",c);e.$formatters.push(function(a){return e.$isEmpty(a)?"":""+a});d.min&&(a=function(a){var c=parseFloat(d.min);return pa(e,"min",e.$isEmpty(a)||a>=c,a)},e.$parsers.push(a), +e.$formatters.push(a));d.max&&(a=function(a){var c=parseFloat(d.max);return pa(e,"max",e.$isEmpty(a)||a<=c,a)},e.$parsers.push(a),e.$formatters.push(a));e.$formatters.push(function(a){return pa(e,"number",e.$isEmpty(a)||vb(a),a)})},url:function(a,c,d,e,g,f){ub(a,c,d,e,g,f);a=function(a){return pa(e,"url",e.$isEmpty(a)||Oe.test(a),a)};e.$formatters.push(a);e.$parsers.push(a)},email:function(a,c,d,e,g,f){ub(a,c,d,e,g,f);a=function(a){return pa(e,"email",e.$isEmpty(a)||Pe.test(a),a)};e.$formatters.push(a); +e.$parsers.push(a)},radio:function(a,c,d,e){E(d.name)&&c.attr("name",bb());c.on("click",function(){c[0].checked&&a.$apply(function(){e.$setViewValue(d.value)})});e.$render=function(){c[0].checked=d.value==e.$viewValue};d.$observe("value",e.$render)},checkbox:function(a,c,d,e){var g=d.ngTrueValue,f=d.ngFalseValue;w(g)||(g=!0);w(f)||(f=!1);c.on("click",function(){a.$apply(function(){e.$setViewValue(c[0].checked)})});e.$render=function(){c[0].checked=e.$viewValue};e.$isEmpty=function(a){return a!==g}; +e.$formatters.push(function(a){return a===g});e.$parsers.push(function(a){return a?g:f})},hidden:C,button:C,submit:C,reset:C,file:C},dc=["$browser","$sniffer",function(a,c){return{restrict:"E",require:"?ngModel",link:function(d,e,g,f){f&&(Rc[K(g.type)]||Rc.text)(d,e,g,f,c,a)}}}],rb="ng-valid",qb="ng-invalid",La="ng-pristine",tb="ng-dirty",Re=["$scope","$exceptionHandler","$attrs","$element","$parse","$animate",function(a,c,d,e,g,f){function h(a,c){c=c?"-"+fb(c,"-"):"";f.removeClass(e,(a?qb:rb)+c); +f.addClass(e,(a?rb:qb)+c)}this.$modelValue=this.$viewValue=Number.NaN;this.$parsers=[];this.$formatters=[];this.$viewChangeListeners=[];this.$pristine=!0;this.$dirty=!1;this.$valid=!0;this.$invalid=!1;this.$name=d.name;var l=g(d.ngModel),k=l.assign;if(!k)throw t("ngModel")("nonassign",d.ngModel,ha(e));this.$render=C;this.$isEmpty=function(a){return E(a)||""===a||null===a||a!==a};var m=e.inheritedData("$formController")||sb,n=0,p=this.$error={};e.addClass(La);h(!0);this.$setValidity=function(a,c){p[a]!== +!c&&(c?(p[a]&&n--,n||(h(!0),this.$valid=!0,this.$invalid=!1)):(h(!1),this.$invalid=!0,this.$valid=!1,n++),p[a]=!c,h(c,a),m.$setValidity(a,c,this))};this.$setPristine=function(){this.$dirty=!1;this.$pristine=!0;f.removeClass(e,tb);f.addClass(e,La)};this.$setViewValue=function(d){this.$viewValue=d;this.$pristine&&(this.$dirty=!0,this.$pristine=!1,f.removeClass(e,La),f.addClass(e,tb),m.$setDirty());q(this.$parsers,function(a){d=a(d)});this.$modelValue!==d&&(this.$modelValue=d,k(a,d),q(this.$viewChangeListeners, +function(a){try{a()}catch(d){c(d)}}))};var r=this;a.$watch(function(){var c=l(a);if(r.$modelValue!==c){var d=r.$formatters,e=d.length;for(r.$modelValue=c;e--;)c=d[e](c);r.$viewValue!==c&&(r.$viewValue=c,r.$render())}return c})}],Fd=function(){return{require:["ngModel","^?form"],controller:Re,link:function(a,c,d,e){var g=e[0],f=e[1]||sb;f.$addControl(g);a.$on("$destroy",function(){f.$removeControl(g)})}}},Hd=aa({require:"ngModel",link:function(a,c,d,e){e.$viewChangeListeners.push(function(){a.$eval(d.ngChange)})}}), +ec=function(){return{require:"?ngModel",link:function(a,c,d,e){if(e){d.required=!0;var g=function(a){if(d.required&&e.$isEmpty(a))e.$setValidity("required",!1);else return e.$setValidity("required",!0),a};e.$formatters.push(g);e.$parsers.unshift(g);d.$observe("required",function(){g(e.$viewValue)})}}}},Gd=function(){return{require:"ngModel",link:function(a,c,d,e){var g=(a=/\/(.*)\//.exec(d.ngList))&&RegExp(a[1])||d.ngList||",";e.$parsers.push(function(a){if(!E(a)){var c=[];a&&q(a.split(g),function(a){a&& +c.push(ca(a))});return c}});e.$formatters.push(function(a){return M(a)?a.join(", "):s});e.$isEmpty=function(a){return!a||!a.length}}}},Se=/^(true|false|\d+)$/,Id=function(){return{priority:100,compile:function(a,c){return Se.test(c.ngValue)?function(a,c,g){g.$set("value",a.$eval(g.ngValue))}:function(a,c,g){a.$watch(g.ngValue,function(a){g.$set("value",a)})}}}},id=va(function(a,c,d){c.addClass("ng-binding").data("$binding",d.ngBind);a.$watch(d.ngBind,function(a){c.text(a==s?"":a)})}),kd=["$interpolate", +function(a){return function(c,d,e){c=a(d.attr(e.$attr.ngBindTemplate));d.addClass("ng-binding").data("$binding",c);e.$observe("ngBindTemplate",function(a){d.text(a)})}}],jd=["$sce","$parse",function(a,c){return function(d,e,g){e.addClass("ng-binding").data("$binding",g.ngBindHtml);var f=c(g.ngBindHtml);d.$watch(function(){return(f(d)||"").toString()},function(c){e.html(a.getTrustedHtml(f(d))||"")})}}],ld=Pb("",!0),nd=Pb("Odd",0),md=Pb("Even",1),od=va({compile:function(a,c){c.$set("ngCloak",s);a.removeClass("ng-cloak")}}), +pd=[function(){return{scope:!0,controller:"@",priority:500}}],fc={};q("click dblclick mousedown mouseup mouseover mouseout mousemove mouseenter mouseleave keydown keyup keypress submit focus blur copy cut paste".split(" "),function(a){var c=na("ng-"+a);fc[c]=["$parse",function(d){return{compile:function(e,g){var f=d(g[c]);return function(c,d,e){d.on(K(a),function(a){c.$apply(function(){f(c,{$event:a})})})}}}}]});var sd=["$animate",function(a){return{transclude:"element",priority:600,terminal:!0,restrict:"A", +$$tlb:!0,link:function(c,d,e,g,f){var h,l,k;c.$watch(e.ngIf,function(g){Qa(g)?l||(l=c.$new(),f(l,function(c){c[c.length++]=U.createComment(" end ngIf: "+e.ngIf+" ");h={clone:c};a.enter(c,d.parent(),d)})):(k&&(k.remove(),k=null),l&&(l.$destroy(),l=null),h&&(k=yb(h.clone),a.leave(k,function(){k=null}),h=null))})}}}],td=["$http","$templateCache","$anchorScroll","$animate","$sce",function(a,c,d,e,g){return{restrict:"ECA",priority:400,terminal:!0,transclude:"element",controller:Ea.noop,compile:function(f, +h){var l=h.ngInclude||h.src,k=h.onload||"",m=h.autoscroll;return function(f,h,q,s,u){var F=0,v,y,A,x=function(){y&&(y.remove(),y=null);v&&(v.$destroy(),v=null);A&&(e.leave(A,function(){y=null}),y=A,A=null)};f.$watch(g.parseAsResourceUrl(l),function(g){var l=function(){!B(m)||m&&!f.$eval(m)||d()},q=++F;g?(a.get(g,{cache:c}).success(function(a){if(q===F){var c=f.$new();s.template=a;a=u(c,function(a){x();e.enter(a,null,h,l)});v=c;A=a;v.$emit("$includeContentLoaded");f.$eval(k)}}).error(function(){q=== +F&&x()}),f.$emit("$includeContentRequested")):(x(),s.template=null)})}}}}],Jd=["$compile",function(a){return{restrict:"ECA",priority:-400,require:"ngInclude",link:function(c,d,e,g){d.html(g.template);a(d.contents())(c)}}}],ud=va({priority:450,compile:function(){return{pre:function(a,c,d){a.$eval(d.ngInit)}}}}),vd=va({terminal:!0,priority:1E3}),wd=["$locale","$interpolate",function(a,c){var d=/{}/g;return{restrict:"EA",link:function(e,g,f){var h=f.count,l=f.$attr.when&&g.attr(f.$attr.when),k=f.offset|| +0,m=e.$eval(l)||{},n={},p=c.startSymbol(),r=c.endSymbol(),s=/^when(Minus)?(.+)$/;q(f,function(a,c){s.test(c)&&(m[K(c.replace("when","").replace("Minus","-"))]=g.attr(f.$attr[c]))});q(m,function(a,e){n[e]=c(a.replace(d,p+h+"-"+k+r))});e.$watch(function(){var c=parseFloat(e.$eval(h));if(isNaN(c))return"";c in m||(c=a.pluralCat(c-k));return n[c](e,g,!0)},function(a){g.text(a)})}}}],xd=["$parse","$animate",function(a,c){var d=t("ngRepeat");return{transclude:"element",priority:1E3,terminal:!0,$$tlb:!0, +link:function(e,g,f,h,l){var k=f.ngRepeat,m=k.match(/^\s*([\s\S]+?)\s+in\s+([\s\S]+?)(?:\s+track\s+by\s+([\s\S]+?))?\s*$/),n,p,r,s,u,F,v={$id:Ia};if(!m)throw d("iexp",k);f=m[1];h=m[2];(m=m[3])?(n=a(m),p=function(a,c,d){F&&(v[F]=a);v[u]=c;v.$index=d;return n(e,v)}):(r=function(a,c){return Ia(c)},s=function(a){return a});m=f.match(/^(?:([\$\w]+)|\(([\$\w]+)\s*,\s*([\$\w]+)\))$/);if(!m)throw d("iidexp",f);u=m[3]||m[1];F=m[2];var B={};e.$watchCollection(h,function(a){var f,h,m=g[0],n,v={},H,R,w,C,T,t, +E=[];if(ab(a))T=a,n=p||r;else{n=p||s;T=[];for(w in a)a.hasOwnProperty(w)&&"$"!=w.charAt(0)&&T.push(w);T.sort()}H=T.length;h=E.length=T.length;for(f=0;fA;)z.pop().element.remove()}for(;x.length>I;)x.pop()[0].element.remove()}var k;if(!(k=t.match(d)))throw Te("iexp",t,ha(f));var l=c(k[2]||k[1]),m=k[4]||k[6],n=k[5],p=c(k[3]||""),q= +c(k[2]?k[1]:m),y=c(k[7]),w=k[8]?c(k[8]):null,x=[[{element:f,label:""}]];u&&(a(u)(e),u.removeClass("ng-scope"),u.remove());f.empty();f.on("change",function(){e.$apply(function(){var a,c=y(e)||[],d={},h,k,l,p,t,v,u;if(r)for(k=[],p=0,v=x.length;p@charset "UTF-8";[ng\\:cloak],[ng-cloak],[data-ng-cloak],[x-ng-cloak],.ng-cloak,.x-ng-cloak,.ng-hide{display:none !important;}ng\\:form{display:block;}.ng-animate-block-transitions{transition:0s all!important;-webkit-transition:0s all!important;}'); +//# sourceMappingURL=angular.min.js.map + +},{}],5:[function(require,module,exports){ +!function() { + var d3 = { + version: "3.4.5" + }; + if (!Date.now) Date.now = function() { + return +new Date(); + }; + var d3_arraySlice = [].slice, d3_array = function(list) { + return d3_arraySlice.call(list); + }; + var d3_document = document, d3_documentElement = d3_document.documentElement, d3_window = window; + try { + d3_array(d3_documentElement.childNodes)[0].nodeType; + } catch (e) { + d3_array = function(list) { + var i = list.length, array = new Array(i); + while (i--) array[i] = list[i]; + return array; + }; + } + try { + d3_document.createElement("div").style.setProperty("opacity", 0, ""); + } catch (error) { + var d3_element_prototype = d3_window.Element.prototype, d3_element_setAttribute = d3_element_prototype.setAttribute, d3_element_setAttributeNS = d3_element_prototype.setAttributeNS, d3_style_prototype = d3_window.CSSStyleDeclaration.prototype, d3_style_setProperty = d3_style_prototype.setProperty; + d3_element_prototype.setAttribute = function(name, value) { + d3_element_setAttribute.call(this, name, value + ""); + }; + d3_element_prototype.setAttributeNS = function(space, local, value) { + d3_element_setAttributeNS.call(this, space, local, value + ""); + }; + d3_style_prototype.setProperty = function(name, value, priority) { + d3_style_setProperty.call(this, name, value + "", priority); + }; + } + d3.ascending = d3_ascending; + function d3_ascending(a, b) { + return a < b ? -1 : a > b ? 1 : a >= b ? 0 : NaN; + } + d3.descending = function(a, b) { + return b < a ? -1 : b > a ? 1 : b >= a ? 0 : NaN; + }; + d3.min = function(array, f) { + var i = -1, n = array.length, a, b; + if (arguments.length === 1) { + while (++i < n && !((a = array[i]) != null && a <= a)) a = undefined; + while (++i < n) if ((b = array[i]) != null && a > b) a = b; + } else { + while (++i < n && !((a = f.call(array, array[i], i)) != null && a <= a)) a = undefined; + while (++i < n) if ((b = f.call(array, array[i], i)) != null && a > b) a = b; + } + return a; + }; + d3.max = function(array, f) { + var i = -1, n = array.length, a, b; + if (arguments.length === 1) { + while (++i < n && !((a = array[i]) != null && a <= a)) a = undefined; + while (++i < n) if ((b = array[i]) != null && b > a) a = b; + } else { + while (++i < n && !((a = f.call(array, array[i], i)) != null && a <= a)) a = undefined; + while (++i < n) if ((b = f.call(array, array[i], i)) != null && b > a) a = b; + } + return a; + }; + d3.extent = function(array, f) { + var i = -1, n = array.length, a, b, c; + if (arguments.length === 1) { + while (++i < n && !((a = c = array[i]) != null && a <= a)) a = c = undefined; + while (++i < n) if ((b = array[i]) != null) { + if (a > b) a = b; + if (c < b) c = b; + } + } else { + while (++i < n && !((a = c = f.call(array, array[i], i)) != null && a <= a)) a = undefined; + while (++i < n) if ((b = f.call(array, array[i], i)) != null) { + if (a > b) a = b; + if (c < b) c = b; + } + } + return [ a, c ]; + }; + d3.sum = function(array, f) { + var s = 0, n = array.length, a, i = -1; + if (arguments.length === 1) { + while (++i < n) if (!isNaN(a = +array[i])) s += a; + } else { + while (++i < n) if (!isNaN(a = +f.call(array, array[i], i))) s += a; + } + return s; + }; + function d3_number(x) { + return x != null && !isNaN(x); + } + d3.mean = function(array, f) { + var n = array.length, a, m = 0, i = -1, j = 0; + if (arguments.length === 1) { + while (++i < n) if (d3_number(a = array[i])) m += (a - m) / ++j; + } else { + while (++i < n) if (d3_number(a = f.call(array, array[i], i))) m += (a - m) / ++j; + } + return j ? m : undefined; + }; + d3.quantile = function(values, p) { + var H = (values.length - 1) * p + 1, h = Math.floor(H), v = +values[h - 1], e = H - h; + return e ? v + e * (values[h] - v) : v; + }; + d3.median = function(array, f) { + if (arguments.length > 1) array = array.map(f); + array = array.filter(d3_number); + return array.length ? d3.quantile(array.sort(d3_ascending), .5) : undefined; + }; + function d3_bisector(compare) { + return { + left: function(a, x, lo, hi) { + if (arguments.length < 3) lo = 0; + if (arguments.length < 4) hi = a.length; + while (lo < hi) { + var mid = lo + hi >>> 1; + if (compare(a[mid], x) < 0) lo = mid + 1; else hi = mid; + } + return lo; + }, + right: function(a, x, lo, hi) { + if (arguments.length < 3) lo = 0; + if (arguments.length < 4) hi = a.length; + while (lo < hi) { + var mid = lo + hi >>> 1; + if (compare(a[mid], x) > 0) hi = mid; else lo = mid + 1; + } + return lo; + } + }; + } + var d3_bisect = d3_bisector(d3_ascending); + d3.bisectLeft = d3_bisect.left; + d3.bisect = d3.bisectRight = d3_bisect.right; + d3.bisector = function(f) { + return d3_bisector(f.length === 1 ? function(d, x) { + return d3_ascending(f(d), x); + } : f); + }; + d3.shuffle = function(array) { + var m = array.length, t, i; + while (m) { + i = Math.random() * m-- | 0; + t = array[m], array[m] = array[i], array[i] = t; + } + return array; + }; + d3.permute = function(array, indexes) { + var i = indexes.length, permutes = new Array(i); + while (i--) permutes[i] = array[indexes[i]]; + return permutes; + }; + d3.pairs = function(array) { + var i = 0, n = array.length - 1, p0, p1 = array[0], pairs = new Array(n < 0 ? 0 : n); + while (i < n) pairs[i] = [ p0 = p1, p1 = array[++i] ]; + return pairs; + }; + d3.zip = function() { + if (!(n = arguments.length)) return []; + for (var i = -1, m = d3.min(arguments, d3_zipLength), zips = new Array(m); ++i < m; ) { + for (var j = -1, n, zip = zips[i] = new Array(n); ++j < n; ) { + zip[j] = arguments[j][i]; + } + } + return zips; + }; + function d3_zipLength(d) { + return d.length; + } + d3.transpose = function(matrix) { + return d3.zip.apply(d3, matrix); + }; + d3.keys = function(map) { + var keys = []; + for (var key in map) keys.push(key); + return keys; + }; + d3.values = function(map) { + var values = []; + for (var key in map) values.push(map[key]); + return values; + }; + d3.entries = function(map) { + var entries = []; + for (var key in map) entries.push({ + key: key, + value: map[key] + }); + return entries; + }; + d3.merge = function(arrays) { + var n = arrays.length, m, i = -1, j = 0, merged, array; + while (++i < n) j += arrays[i].length; + merged = new Array(j); + while (--n >= 0) { + array = arrays[n]; + m = array.length; + while (--m >= 0) { + merged[--j] = array[m]; + } + } + return merged; + }; + var abs = Math.abs; + d3.range = function(start, stop, step) { + if (arguments.length < 3) { + step = 1; + if (arguments.length < 2) { + stop = start; + start = 0; + } + } + if ((stop - start) / step === Infinity) throw new Error("infinite range"); + var range = [], k = d3_range_integerScale(abs(step)), i = -1, j; + start *= k, stop *= k, step *= k; + if (step < 0) while ((j = start + step * ++i) > stop) range.push(j / k); else while ((j = start + step * ++i) < stop) range.push(j / k); + return range; + }; + function d3_range_integerScale(x) { + var k = 1; + while (x * k % 1) k *= 10; + return k; + } + function d3_class(ctor, properties) { + try { + for (var key in properties) { + Object.defineProperty(ctor.prototype, key, { + value: properties[key], + enumerable: false + }); + } + } catch (e) { + ctor.prototype = properties; + } + } + d3.map = function(object) { + var map = new d3_Map(); + if (object instanceof d3_Map) object.forEach(function(key, value) { + map.set(key, value); + }); else for (var key in object) map.set(key, object[key]); + return map; + }; + function d3_Map() {} + d3_class(d3_Map, { + has: d3_map_has, + get: function(key) { + return this[d3_map_prefix + key]; + }, + set: function(key, value) { + return this[d3_map_prefix + key] = value; + }, + remove: d3_map_remove, + keys: d3_map_keys, + values: function() { + var values = []; + this.forEach(function(key, value) { + values.push(value); + }); + return values; + }, + entries: function() { + var entries = []; + this.forEach(function(key, value) { + entries.push({ + key: key, + value: value + }); + }); + return entries; + }, + size: d3_map_size, + empty: d3_map_empty, + forEach: function(f) { + for (var key in this) if (key.charCodeAt(0) === d3_map_prefixCode) f.call(this, key.substring(1), this[key]); + } + }); + var d3_map_prefix = "\x00", d3_map_prefixCode = d3_map_prefix.charCodeAt(0); + function d3_map_has(key) { + return d3_map_prefix + key in this; + } + function d3_map_remove(key) { + key = d3_map_prefix + key; + return key in this && delete this[key]; + } + function d3_map_keys() { + var keys = []; + this.forEach(function(key) { + keys.push(key); + }); + return keys; + } + function d3_map_size() { + var size = 0; + for (var key in this) if (key.charCodeAt(0) === d3_map_prefixCode) ++size; + return size; + } + function d3_map_empty() { + for (var key in this) if (key.charCodeAt(0) === d3_map_prefixCode) return false; + return true; + } + d3.nest = function() { + var nest = {}, keys = [], sortKeys = [], sortValues, rollup; + function map(mapType, array, depth) { + if (depth >= keys.length) return rollup ? rollup.call(nest, array) : sortValues ? array.sort(sortValues) : array; + var i = -1, n = array.length, key = keys[depth++], keyValue, object, setter, valuesByKey = new d3_Map(), values; + while (++i < n) { + if (values = valuesByKey.get(keyValue = key(object = array[i]))) { + values.push(object); + } else { + valuesByKey.set(keyValue, [ object ]); + } + } + if (mapType) { + object = mapType(); + setter = function(keyValue, values) { + object.set(keyValue, map(mapType, values, depth)); + }; + } else { + object = {}; + setter = function(keyValue, values) { + object[keyValue] = map(mapType, values, depth); + }; + } + valuesByKey.forEach(setter); + return object; + } + function entries(map, depth) { + if (depth >= keys.length) return map; + var array = [], sortKey = sortKeys[depth++]; + map.forEach(function(key, keyMap) { + array.push({ + key: key, + values: entries(keyMap, depth) + }); + }); + return sortKey ? array.sort(function(a, b) { + return sortKey(a.key, b.key); + }) : array; + } + nest.map = function(array, mapType) { + return map(mapType, array, 0); + }; + nest.entries = function(array) { + return entries(map(d3.map, array, 0), 0); + }; + nest.key = function(d) { + keys.push(d); + return nest; + }; + nest.sortKeys = function(order) { + sortKeys[keys.length - 1] = order; + return nest; + }; + nest.sortValues = function(order) { + sortValues = order; + return nest; + }; + nest.rollup = function(f) { + rollup = f; + return nest; + }; + return nest; + }; + d3.set = function(array) { + var set = new d3_Set(); + if (array) for (var i = 0, n = array.length; i < n; ++i) set.add(array[i]); + return set; + }; + function d3_Set() {} + d3_class(d3_Set, { + has: d3_map_has, + add: function(value) { + this[d3_map_prefix + value] = true; + return value; + }, + remove: function(value) { + value = d3_map_prefix + value; + return value in this && delete this[value]; + }, + values: d3_map_keys, + size: d3_map_size, + empty: d3_map_empty, + forEach: function(f) { + for (var value in this) if (value.charCodeAt(0) === d3_map_prefixCode) f.call(this, value.substring(1)); + } + }); + d3.behavior = {}; + d3.rebind = function(target, source) { + var i = 1, n = arguments.length, method; + while (++i < n) target[method = arguments[i]] = d3_rebind(target, source, source[method]); + return target; + }; + function d3_rebind(target, source, method) { + return function() { + var value = method.apply(source, arguments); + return value === source ? target : value; + }; + } + function d3_vendorSymbol(object, name) { + if (name in object) return name; + name = name.charAt(0).toUpperCase() + name.substring(1); + for (var i = 0, n = d3_vendorPrefixes.length; i < n; ++i) { + var prefixName = d3_vendorPrefixes[i] + name; + if (prefixName in object) return prefixName; + } + } + var d3_vendorPrefixes = [ "webkit", "ms", "moz", "Moz", "o", "O" ]; + function d3_noop() {} + d3.dispatch = function() { + var dispatch = new d3_dispatch(), i = -1, n = arguments.length; + while (++i < n) dispatch[arguments[i]] = d3_dispatch_event(dispatch); + return dispatch; + }; + function d3_dispatch() {} + d3_dispatch.prototype.on = function(type, listener) { + var i = type.indexOf("."), name = ""; + if (i >= 0) { + name = type.substring(i + 1); + type = type.substring(0, i); + } + if (type) return arguments.length < 2 ? this[type].on(name) : this[type].on(name, listener); + if (arguments.length === 2) { + if (listener == null) for (type in this) { + if (this.hasOwnProperty(type)) this[type].on(name, null); + } + return this; + } + }; + function d3_dispatch_event(dispatch) { + var listeners = [], listenerByName = new d3_Map(); + function event() { + var z = listeners, i = -1, n = z.length, l; + while (++i < n) if (l = z[i].on) l.apply(this, arguments); + return dispatch; + } + event.on = function(name, listener) { + var l = listenerByName.get(name), i; + if (arguments.length < 2) return l && l.on; + if (l) { + l.on = null; + listeners = listeners.slice(0, i = listeners.indexOf(l)).concat(listeners.slice(i + 1)); + listenerByName.remove(name); + } + if (listener) listeners.push(listenerByName.set(name, { + on: listener + })); + return dispatch; + }; + return event; + } + d3.event = null; + function d3_eventPreventDefault() { + d3.event.preventDefault(); + } + function d3_eventSource() { + var e = d3.event, s; + while (s = e.sourceEvent) e = s; + return e; + } + function d3_eventDispatch(target) { + var dispatch = new d3_dispatch(), i = 0, n = arguments.length; + while (++i < n) dispatch[arguments[i]] = d3_dispatch_event(dispatch); + dispatch.of = function(thiz, argumentz) { + return function(e1) { + try { + var e0 = e1.sourceEvent = d3.event; + e1.target = target; + d3.event = e1; + dispatch[e1.type].apply(thiz, argumentz); + } finally { + d3.event = e0; + } + }; + }; + return dispatch; + } + d3.requote = function(s) { + return s.replace(d3_requote_re, "\\$&"); + }; + var d3_requote_re = /[\\\^\$\*\+\?\|\[\]\(\)\.\{\}]/g; + var d3_subclass = {}.__proto__ ? function(object, prototype) { + object.__proto__ = prototype; + } : function(object, prototype) { + for (var property in prototype) object[property] = prototype[property]; + }; + function d3_selection(groups) { + d3_subclass(groups, d3_selectionPrototype); + return groups; + } + var d3_select = function(s, n) { + return n.querySelector(s); + }, d3_selectAll = function(s, n) { + return n.querySelectorAll(s); + }, d3_selectMatcher = d3_documentElement[d3_vendorSymbol(d3_documentElement, "matchesSelector")], d3_selectMatches = function(n, s) { + return d3_selectMatcher.call(n, s); + }; + if (typeof Sizzle === "function") { + d3_select = function(s, n) { + return Sizzle(s, n)[0] || null; + }; + d3_selectAll = Sizzle; + d3_selectMatches = Sizzle.matchesSelector; + } + d3.selection = function() { + return d3_selectionRoot; + }; + var d3_selectionPrototype = d3.selection.prototype = []; + d3_selectionPrototype.select = function(selector) { + var subgroups = [], subgroup, subnode, group, node; + selector = d3_selection_selector(selector); + for (var j = -1, m = this.length; ++j < m; ) { + subgroups.push(subgroup = []); + subgroup.parentNode = (group = this[j]).parentNode; + for (var i = -1, n = group.length; ++i < n; ) { + if (node = group[i]) { + subgroup.push(subnode = selector.call(node, node.__data__, i, j)); + if (subnode && "__data__" in node) subnode.__data__ = node.__data__; + } else { + subgroup.push(null); + } + } + } + return d3_selection(subgroups); + }; + function d3_selection_selector(selector) { + return typeof selector === "function" ? selector : function() { + return d3_select(selector, this); + }; + } + d3_selectionPrototype.selectAll = function(selector) { + var subgroups = [], subgroup, node; + selector = d3_selection_selectorAll(selector); + for (var j = -1, m = this.length; ++j < m; ) { + for (var group = this[j], i = -1, n = group.length; ++i < n; ) { + if (node = group[i]) { + subgroups.push(subgroup = d3_array(selector.call(node, node.__data__, i, j))); + subgroup.parentNode = node; + } + } + } + return d3_selection(subgroups); + }; + function d3_selection_selectorAll(selector) { + return typeof selector === "function" ? selector : function() { + return d3_selectAll(selector, this); + }; + } + var d3_nsPrefix = { + svg: "http://www.w3.org/2000/svg", + xhtml: "http://www.w3.org/1999/xhtml", + xlink: "http://www.w3.org/1999/xlink", + xml: "http://www.w3.org/XML/1998/namespace", + xmlns: "http://www.w3.org/2000/xmlns/" + }; + d3.ns = { + prefix: d3_nsPrefix, + qualify: function(name) { + var i = name.indexOf(":"), prefix = name; + if (i >= 0) { + prefix = name.substring(0, i); + name = name.substring(i + 1); + } + return d3_nsPrefix.hasOwnProperty(prefix) ? { + space: d3_nsPrefix[prefix], + local: name + } : name; + } + }; + d3_selectionPrototype.attr = function(name, value) { + if (arguments.length < 2) { + if (typeof name === "string") { + var node = this.node(); + name = d3.ns.qualify(name); + return name.local ? node.getAttributeNS(name.space, name.local) : node.getAttribute(name); + } + for (value in name) this.each(d3_selection_attr(value, name[value])); + return this; + } + return this.each(d3_selection_attr(name, value)); + }; + function d3_selection_attr(name, value) { + name = d3.ns.qualify(name); + function attrNull() { + this.removeAttribute(name); + } + function attrNullNS() { + this.removeAttributeNS(name.space, name.local); + } + function attrConstant() { + this.setAttribute(name, value); + } + function attrConstantNS() { + this.setAttributeNS(name.space, name.local, value); + } + function attrFunction() { + var x = value.apply(this, arguments); + if (x == null) this.removeAttribute(name); else this.setAttribute(name, x); + } + function attrFunctionNS() { + var x = value.apply(this, arguments); + if (x == null) this.removeAttributeNS(name.space, name.local); else this.setAttributeNS(name.space, name.local, x); + } + return value == null ? name.local ? attrNullNS : attrNull : typeof value === "function" ? name.local ? attrFunctionNS : attrFunction : name.local ? attrConstantNS : attrConstant; + } + function d3_collapse(s) { + return s.trim().replace(/\s+/g, " "); + } + d3_selectionPrototype.classed = function(name, value) { + if (arguments.length < 2) { + if (typeof name === "string") { + var node = this.node(), n = (name = d3_selection_classes(name)).length, i = -1; + if (value = node.classList) { + while (++i < n) if (!value.contains(name[i])) return false; + } else { + value = node.getAttribute("class"); + while (++i < n) if (!d3_selection_classedRe(name[i]).test(value)) return false; + } + return true; + } + for (value in name) this.each(d3_selection_classed(value, name[value])); + return this; + } + return this.each(d3_selection_classed(name, value)); + }; + function d3_selection_classedRe(name) { + return new RegExp("(?:^|\\s+)" + d3.requote(name) + "(?:\\s+|$)", "g"); + } + function d3_selection_classes(name) { + return name.trim().split(/^|\s+/); + } + function d3_selection_classed(name, value) { + name = d3_selection_classes(name).map(d3_selection_classedName); + var n = name.length; + function classedConstant() { + var i = -1; + while (++i < n) name[i](this, value); + } + function classedFunction() { + var i = -1, x = value.apply(this, arguments); + while (++i < n) name[i](this, x); + } + return typeof value === "function" ? classedFunction : classedConstant; + } + function d3_selection_classedName(name) { + var re = d3_selection_classedRe(name); + return function(node, value) { + if (c = node.classList) return value ? c.add(name) : c.remove(name); + var c = node.getAttribute("class") || ""; + if (value) { + re.lastIndex = 0; + if (!re.test(c)) node.setAttribute("class", d3_collapse(c + " " + name)); + } else { + node.setAttribute("class", d3_collapse(c.replace(re, " "))); + } + }; + } + d3_selectionPrototype.style = function(name, value, priority) { + var n = arguments.length; + if (n < 3) { + if (typeof name !== "string") { + if (n < 2) value = ""; + for (priority in name) this.each(d3_selection_style(priority, name[priority], value)); + return this; + } + if (n < 2) return d3_window.getComputedStyle(this.node(), null).getPropertyValue(name); + priority = ""; + } + return this.each(d3_selection_style(name, value, priority)); + }; + function d3_selection_style(name, value, priority) { + function styleNull() { + this.style.removeProperty(name); + } + function styleConstant() { + this.style.setProperty(name, value, priority); + } + function styleFunction() { + var x = value.apply(this, arguments); + if (x == null) this.style.removeProperty(name); else this.style.setProperty(name, x, priority); + } + return value == null ? styleNull : typeof value === "function" ? styleFunction : styleConstant; + } + d3_selectionPrototype.property = function(name, value) { + if (arguments.length < 2) { + if (typeof name === "string") return this.node()[name]; + for (value in name) this.each(d3_selection_property(value, name[value])); + return this; + } + return this.each(d3_selection_property(name, value)); + }; + function d3_selection_property(name, value) { + function propertyNull() { + delete this[name]; + } + function propertyConstant() { + this[name] = value; + } + function propertyFunction() { + var x = value.apply(this, arguments); + if (x == null) delete this[name]; else this[name] = x; + } + return value == null ? propertyNull : typeof value === "function" ? propertyFunction : propertyConstant; + } + d3_selectionPrototype.text = function(value) { + return arguments.length ? this.each(typeof value === "function" ? function() { + var v = value.apply(this, arguments); + this.textContent = v == null ? "" : v; + } : value == null ? function() { + this.textContent = ""; + } : function() { + this.textContent = value; + }) : this.node().textContent; + }; + d3_selectionPrototype.html = function(value) { + return arguments.length ? this.each(typeof value === "function" ? function() { + var v = value.apply(this, arguments); + this.innerHTML = v == null ? "" : v; + } : value == null ? function() { + this.innerHTML = ""; + } : function() { + this.innerHTML = value; + }) : this.node().innerHTML; + }; + d3_selectionPrototype.append = function(name) { + name = d3_selection_creator(name); + return this.select(function() { + return this.appendChild(name.apply(this, arguments)); + }); + }; + function d3_selection_creator(name) { + return typeof name === "function" ? name : (name = d3.ns.qualify(name)).local ? function() { + return this.ownerDocument.createElementNS(name.space, name.local); + } : function() { + return this.ownerDocument.createElementNS(this.namespaceURI, name); + }; + } + d3_selectionPrototype.insert = function(name, before) { + name = d3_selection_creator(name); + before = d3_selection_selector(before); + return this.select(function() { + return this.insertBefore(name.apply(this, arguments), before.apply(this, arguments) || null); + }); + }; + d3_selectionPrototype.remove = function() { + return this.each(function() { + var parent = this.parentNode; + if (parent) parent.removeChild(this); + }); + }; + d3_selectionPrototype.data = function(value, key) { + var i = -1, n = this.length, group, node; + if (!arguments.length) { + value = new Array(n = (group = this[0]).length); + while (++i < n) { + if (node = group[i]) { + value[i] = node.__data__; + } + } + return value; + } + function bind(group, groupData) { + var i, n = group.length, m = groupData.length, n0 = Math.min(n, m), updateNodes = new Array(m), enterNodes = new Array(m), exitNodes = new Array(n), node, nodeData; + if (key) { + var nodeByKeyValue = new d3_Map(), dataByKeyValue = new d3_Map(), keyValues = [], keyValue; + for (i = -1; ++i < n; ) { + keyValue = key.call(node = group[i], node.__data__, i); + if (nodeByKeyValue.has(keyValue)) { + exitNodes[i] = node; + } else { + nodeByKeyValue.set(keyValue, node); + } + keyValues.push(keyValue); + } + for (i = -1; ++i < m; ) { + keyValue = key.call(groupData, nodeData = groupData[i], i); + if (node = nodeByKeyValue.get(keyValue)) { + updateNodes[i] = node; + node.__data__ = nodeData; + } else if (!dataByKeyValue.has(keyValue)) { + enterNodes[i] = d3_selection_dataNode(nodeData); + } + dataByKeyValue.set(keyValue, nodeData); + nodeByKeyValue.remove(keyValue); + } + for (i = -1; ++i < n; ) { + if (nodeByKeyValue.has(keyValues[i])) { + exitNodes[i] = group[i]; + } + } + } else { + for (i = -1; ++i < n0; ) { + node = group[i]; + nodeData = groupData[i]; + if (node) { + node.__data__ = nodeData; + updateNodes[i] = node; + } else { + enterNodes[i] = d3_selection_dataNode(nodeData); + } + } + for (;i < m; ++i) { + enterNodes[i] = d3_selection_dataNode(groupData[i]); + } + for (;i < n; ++i) { + exitNodes[i] = group[i]; + } + } + enterNodes.update = updateNodes; + enterNodes.parentNode = updateNodes.parentNode = exitNodes.parentNode = group.parentNode; + enter.push(enterNodes); + update.push(updateNodes); + exit.push(exitNodes); + } + var enter = d3_selection_enter([]), update = d3_selection([]), exit = d3_selection([]); + if (typeof value === "function") { + while (++i < n) { + bind(group = this[i], value.call(group, group.parentNode.__data__, i)); + } + } else { + while (++i < n) { + bind(group = this[i], value); + } + } + update.enter = function() { + return enter; + }; + update.exit = function() { + return exit; + }; + return update; + }; + function d3_selection_dataNode(data) { + return { + __data__: data + }; + } + d3_selectionPrototype.datum = function(value) { + return arguments.length ? this.property("__data__", value) : this.property("__data__"); + }; + d3_selectionPrototype.filter = function(filter) { + var subgroups = [], subgroup, group, node; + if (typeof filter !== "function") filter = d3_selection_filter(filter); + for (var j = 0, m = this.length; j < m; j++) { + subgroups.push(subgroup = []); + subgroup.parentNode = (group = this[j]).parentNode; + for (var i = 0, n = group.length; i < n; i++) { + if ((node = group[i]) && filter.call(node, node.__data__, i, j)) { + subgroup.push(node); + } + } + } + return d3_selection(subgroups); + }; + function d3_selection_filter(selector) { + return function() { + return d3_selectMatches(this, selector); + }; + } + d3_selectionPrototype.order = function() { + for (var j = -1, m = this.length; ++j < m; ) { + for (var group = this[j], i = group.length - 1, next = group[i], node; --i >= 0; ) { + if (node = group[i]) { + if (next && next !== node.nextSibling) next.parentNode.insertBefore(node, next); + next = node; + } + } + } + return this; + }; + d3_selectionPrototype.sort = function(comparator) { + comparator = d3_selection_sortComparator.apply(this, arguments); + for (var j = -1, m = this.length; ++j < m; ) this[j].sort(comparator); + return this.order(); + }; + function d3_selection_sortComparator(comparator) { + if (!arguments.length) comparator = d3_ascending; + return function(a, b) { + return a && b ? comparator(a.__data__, b.__data__) : !a - !b; + }; + } + d3_selectionPrototype.each = function(callback) { + return d3_selection_each(this, function(node, i, j) { + callback.call(node, node.__data__, i, j); + }); + }; + function d3_selection_each(groups, callback) { + for (var j = 0, m = groups.length; j < m; j++) { + for (var group = groups[j], i = 0, n = group.length, node; i < n; i++) { + if (node = group[i]) callback(node, i, j); + } + } + return groups; + } + d3_selectionPrototype.call = function(callback) { + var args = d3_array(arguments); + callback.apply(args[0] = this, args); + return this; + }; + d3_selectionPrototype.empty = function() { + return !this.node(); + }; + d3_selectionPrototype.node = function() { + for (var j = 0, m = this.length; j < m; j++) { + for (var group = this[j], i = 0, n = group.length; i < n; i++) { + var node = group[i]; + if (node) return node; + } + } + return null; + }; + d3_selectionPrototype.size = function() { + var n = 0; + this.each(function() { + ++n; + }); + return n; + }; + function d3_selection_enter(selection) { + d3_subclass(selection, d3_selection_enterPrototype); + return selection; + } + var d3_selection_enterPrototype = []; + d3.selection.enter = d3_selection_enter; + d3.selection.enter.prototype = d3_selection_enterPrototype; + d3_selection_enterPrototype.append = d3_selectionPrototype.append; + d3_selection_enterPrototype.empty = d3_selectionPrototype.empty; + d3_selection_enterPrototype.node = d3_selectionPrototype.node; + d3_selection_enterPrototype.call = d3_selectionPrototype.call; + d3_selection_enterPrototype.size = d3_selectionPrototype.size; + d3_selection_enterPrototype.select = function(selector) { + var subgroups = [], subgroup, subnode, upgroup, group, node; + for (var j = -1, m = this.length; ++j < m; ) { + upgroup = (group = this[j]).update; + subgroups.push(subgroup = []); + subgroup.parentNode = group.parentNode; + for (var i = -1, n = group.length; ++i < n; ) { + if (node = group[i]) { + subgroup.push(upgroup[i] = subnode = selector.call(group.parentNode, node.__data__, i, j)); + subnode.__data__ = node.__data__; + } else { + subgroup.push(null); + } + } + } + return d3_selection(subgroups); + }; + d3_selection_enterPrototype.insert = function(name, before) { + if (arguments.length < 2) before = d3_selection_enterInsertBefore(this); + return d3_selectionPrototype.insert.call(this, name, before); + }; + function d3_selection_enterInsertBefore(enter) { + var i0, j0; + return function(d, i, j) { + var group = enter[j].update, n = group.length, node; + if (j != j0) j0 = j, i0 = 0; + if (i >= i0) i0 = i + 1; + while (!(node = group[i0]) && ++i0 < n) ; + return node; + }; + } + d3_selectionPrototype.transition = function() { + var id = d3_transitionInheritId || ++d3_transitionId, subgroups = [], subgroup, node, transition = d3_transitionInherit || { + time: Date.now(), + ease: d3_ease_cubicInOut, + delay: 0, + duration: 250 + }; + for (var j = -1, m = this.length; ++j < m; ) { + subgroups.push(subgroup = []); + for (var group = this[j], i = -1, n = group.length; ++i < n; ) { + if (node = group[i]) d3_transitionNode(node, i, id, transition); + subgroup.push(node); + } + } + return d3_transition(subgroups, id); + }; + d3_selectionPrototype.interrupt = function() { + return this.each(d3_selection_interrupt); + }; + function d3_selection_interrupt() { + var lock = this.__transition__; + if (lock) ++lock.active; + } + d3.select = function(node) { + var group = [ typeof node === "string" ? d3_select(node, d3_document) : node ]; + group.parentNode = d3_documentElement; + return d3_selection([ group ]); + }; + d3.selectAll = function(nodes) { + var group = d3_array(typeof nodes === "string" ? d3_selectAll(nodes, d3_document) : nodes); + group.parentNode = d3_documentElement; + return d3_selection([ group ]); + }; + var d3_selectionRoot = d3.select(d3_documentElement); + d3_selectionPrototype.on = function(type, listener, capture) { + var n = arguments.length; + if (n < 3) { + if (typeof type !== "string") { + if (n < 2) listener = false; + for (capture in type) this.each(d3_selection_on(capture, type[capture], listener)); + return this; + } + if (n < 2) return (n = this.node()["__on" + type]) && n._; + capture = false; + } + return this.each(d3_selection_on(type, listener, capture)); + }; + function d3_selection_on(type, listener, capture) { + var name = "__on" + type, i = type.indexOf("."), wrap = d3_selection_onListener; + if (i > 0) type = type.substring(0, i); + var filter = d3_selection_onFilters.get(type); + if (filter) type = filter, wrap = d3_selection_onFilter; + function onRemove() { + var l = this[name]; + if (l) { + this.removeEventListener(type, l, l.$); + delete this[name]; + } + } + function onAdd() { + var l = wrap(listener, d3_array(arguments)); + onRemove.call(this); + this.addEventListener(type, this[name] = l, l.$ = capture); + l._ = listener; + } + function removeAll() { + var re = new RegExp("^__on([^.]+)" + d3.requote(type) + "$"), match; + for (var name in this) { + if (match = name.match(re)) { + var l = this[name]; + this.removeEventListener(match[1], l, l.$); + delete this[name]; + } + } + } + return i ? listener ? onAdd : onRemove : listener ? d3_noop : removeAll; + } + var d3_selection_onFilters = d3.map({ + mouseenter: "mouseover", + mouseleave: "mouseout" + }); + d3_selection_onFilters.forEach(function(k) { + if ("on" + k in d3_document) d3_selection_onFilters.remove(k); + }); + function d3_selection_onListener(listener, argumentz) { + return function(e) { + var o = d3.event; + d3.event = e; + argumentz[0] = this.__data__; + try { + listener.apply(this, argumentz); + } finally { + d3.event = o; + } + }; + } + function d3_selection_onFilter(listener, argumentz) { + var l = d3_selection_onListener(listener, argumentz); + return function(e) { + var target = this, related = e.relatedTarget; + if (!related || related !== target && !(related.compareDocumentPosition(target) & 8)) { + l.call(target, e); + } + }; + } + var d3_event_dragSelect = "onselectstart" in d3_document ? null : d3_vendorSymbol(d3_documentElement.style, "userSelect"), d3_event_dragId = 0; + function d3_event_dragSuppress() { + var name = ".dragsuppress-" + ++d3_event_dragId, click = "click" + name, w = d3.select(d3_window).on("touchmove" + name, d3_eventPreventDefault).on("dragstart" + name, d3_eventPreventDefault).on("selectstart" + name, d3_eventPreventDefault); + if (d3_event_dragSelect) { + var style = d3_documentElement.style, select = style[d3_event_dragSelect]; + style[d3_event_dragSelect] = "none"; + } + return function(suppressClick) { + w.on(name, null); + if (d3_event_dragSelect) style[d3_event_dragSelect] = select; + if (suppressClick) { + function off() { + w.on(click, null); + } + w.on(click, function() { + d3_eventPreventDefault(); + off(); + }, true); + setTimeout(off, 0); + } + }; + } + d3.mouse = function(container) { + return d3_mousePoint(container, d3_eventSource()); + }; + function d3_mousePoint(container, e) { + if (e.changedTouches) e = e.changedTouches[0]; + var svg = container.ownerSVGElement || container; + if (svg.createSVGPoint) { + var point = svg.createSVGPoint(); + point.x = e.clientX, point.y = e.clientY; + point = point.matrixTransform(container.getScreenCTM().inverse()); + return [ point.x, point.y ]; + } + var rect = container.getBoundingClientRect(); + return [ e.clientX - rect.left - container.clientLeft, e.clientY - rect.top - container.clientTop ]; + } + d3.touches = function(container, touches) { + if (arguments.length < 2) touches = d3_eventSource().touches; + return touches ? d3_array(touches).map(function(touch) { + var point = d3_mousePoint(container, touch); + point.identifier = touch.identifier; + return point; + }) : []; + }; + d3.behavior.drag = function() { + var event = d3_eventDispatch(drag, "drag", "dragstart", "dragend"), origin = null, mousedown = dragstart(d3_noop, d3.mouse, d3_behavior_dragMouseSubject, "mousemove", "mouseup"), touchstart = dragstart(d3_behavior_dragTouchId, d3.touch, d3_behavior_dragTouchSubject, "touchmove", "touchend"); + function drag() { + this.on("mousedown.drag", mousedown).on("touchstart.drag", touchstart); + } + function dragstart(id, position, subject, move, end) { + return function() { + var that = this, target = d3.event.target, parent = that.parentNode, dispatch = event.of(that, arguments), dragged = 0, dragId = id(), dragName = ".drag" + (dragId == null ? "" : "-" + dragId), dragOffset, dragSubject = d3.select(subject()).on(move + dragName, moved).on(end + dragName, ended), dragRestore = d3_event_dragSuppress(), position0 = position(parent, dragId); + if (origin) { + dragOffset = origin.apply(that, arguments); + dragOffset = [ dragOffset.x - position0[0], dragOffset.y - position0[1] ]; + } else { + dragOffset = [ 0, 0 ]; + } + dispatch({ + type: "dragstart" + }); + function moved() { + var position1 = position(parent, dragId), dx, dy; + if (!position1) return; + dx = position1[0] - position0[0]; + dy = position1[1] - position0[1]; + dragged |= dx | dy; + position0 = position1; + dispatch({ + type: "drag", + x: position1[0] + dragOffset[0], + y: position1[1] + dragOffset[1], + dx: dx, + dy: dy + }); + } + function ended() { + if (!position(parent, dragId)) return; + dragSubject.on(move + dragName, null).on(end + dragName, null); + dragRestore(dragged && d3.event.target === target); + dispatch({ + type: "dragend" + }); + } + }; + } + drag.origin = function(x) { + if (!arguments.length) return origin; + origin = x; + return drag; + }; + return d3.rebind(drag, event, "on"); + }; + function d3_behavior_dragTouchId() { + return d3.event.changedTouches[0].identifier; + } + function d3_behavior_dragTouchSubject() { + return d3.event.target; + } + function d3_behavior_dragMouseSubject() { + return d3_window; + } + var π = Math.PI, τ = 2 * π, halfπ = π / 2, ε = 1e-6, ε2 = ε * ε, d3_radians = π / 180, d3_degrees = 180 / π; + function d3_sgn(x) { + return x > 0 ? 1 : x < 0 ? -1 : 0; + } + function d3_cross2d(a, b, c) { + return (b[0] - a[0]) * (c[1] - a[1]) - (b[1] - a[1]) * (c[0] - a[0]); + } + function d3_acos(x) { + return x > 1 ? 0 : x < -1 ? π : Math.acos(x); + } + function d3_asin(x) { + return x > 1 ? halfπ : x < -1 ? -halfπ : Math.asin(x); + } + function d3_sinh(x) { + return ((x = Math.exp(x)) - 1 / x) / 2; + } + function d3_cosh(x) { + return ((x = Math.exp(x)) + 1 / x) / 2; + } + function d3_tanh(x) { + return ((x = Math.exp(2 * x)) - 1) / (x + 1); + } + function d3_haversin(x) { + return (x = Math.sin(x / 2)) * x; + } + var ρ = Math.SQRT2, ρ2 = 2, ρ4 = 4; + d3.interpolateZoom = function(p0, p1) { + var ux0 = p0[0], uy0 = p0[1], w0 = p0[2], ux1 = p1[0], uy1 = p1[1], w1 = p1[2]; + var dx = ux1 - ux0, dy = uy1 - uy0, d2 = dx * dx + dy * dy, d1 = Math.sqrt(d2), b0 = (w1 * w1 - w0 * w0 + ρ4 * d2) / (2 * w0 * ρ2 * d1), b1 = (w1 * w1 - w0 * w0 - ρ4 * d2) / (2 * w1 * ρ2 * d1), r0 = Math.log(Math.sqrt(b0 * b0 + 1) - b0), r1 = Math.log(Math.sqrt(b1 * b1 + 1) - b1), dr = r1 - r0, S = (dr || Math.log(w1 / w0)) / ρ; + function interpolate(t) { + var s = t * S; + if (dr) { + var coshr0 = d3_cosh(r0), u = w0 / (ρ2 * d1) * (coshr0 * d3_tanh(ρ * s + r0) - d3_sinh(r0)); + return [ ux0 + u * dx, uy0 + u * dy, w0 * coshr0 / d3_cosh(ρ * s + r0) ]; + } + return [ ux0 + t * dx, uy0 + t * dy, w0 * Math.exp(ρ * s) ]; + } + interpolate.duration = S * 1e3; + return interpolate; + }; + d3.behavior.zoom = function() { + var view = { + x: 0, + y: 0, + k: 1 + }, translate0, center, size = [ 960, 500 ], scaleExtent = d3_behavior_zoomInfinity, mousedown = "mousedown.zoom", mousemove = "mousemove.zoom", mouseup = "mouseup.zoom", mousewheelTimer, touchstart = "touchstart.zoom", touchtime, event = d3_eventDispatch(zoom, "zoomstart", "zoom", "zoomend"), x0, x1, y0, y1; + function zoom(g) { + g.on(mousedown, mousedowned).on(d3_behavior_zoomWheel + ".zoom", mousewheeled).on(mousemove, mousewheelreset).on("dblclick.zoom", dblclicked).on(touchstart, touchstarted); + } + zoom.event = function(g) { + g.each(function() { + var dispatch = event.of(this, arguments), view1 = view; + if (d3_transitionInheritId) { + d3.select(this).transition().each("start.zoom", function() { + view = this.__chart__ || { + x: 0, + y: 0, + k: 1 + }; + zoomstarted(dispatch); + }).tween("zoom:zoom", function() { + var dx = size[0], dy = size[1], cx = dx / 2, cy = dy / 2, i = d3.interpolateZoom([ (cx - view.x) / view.k, (cy - view.y) / view.k, dx / view.k ], [ (cx - view1.x) / view1.k, (cy - view1.y) / view1.k, dx / view1.k ]); + return function(t) { + var l = i(t), k = dx / l[2]; + this.__chart__ = view = { + x: cx - l[0] * k, + y: cy - l[1] * k, + k: k + }; + zoomed(dispatch); + }; + }).each("end.zoom", function() { + zoomended(dispatch); + }); + } else { + this.__chart__ = view; + zoomstarted(dispatch); + zoomed(dispatch); + zoomended(dispatch); + } + }); + }; + zoom.translate = function(_) { + if (!arguments.length) return [ view.x, view.y ]; + view = { + x: +_[0], + y: +_[1], + k: view.k + }; + rescale(); + return zoom; + }; + zoom.scale = function(_) { + if (!arguments.length) return view.k; + view = { + x: view.x, + y: view.y, + k: +_ + }; + rescale(); + return zoom; + }; + zoom.scaleExtent = function(_) { + if (!arguments.length) return scaleExtent; + scaleExtent = _ == null ? d3_behavior_zoomInfinity : [ +_[0], +_[1] ]; + return zoom; + }; + zoom.center = function(_) { + if (!arguments.length) return center; + center = _ && [ +_[0], +_[1] ]; + return zoom; + }; + zoom.size = function(_) { + if (!arguments.length) return size; + size = _ && [ +_[0], +_[1] ]; + return zoom; + }; + zoom.x = function(z) { + if (!arguments.length) return x1; + x1 = z; + x0 = z.copy(); + view = { + x: 0, + y: 0, + k: 1 + }; + return zoom; + }; + zoom.y = function(z) { + if (!arguments.length) return y1; + y1 = z; + y0 = z.copy(); + view = { + x: 0, + y: 0, + k: 1 + }; + return zoom; + }; + function location(p) { + return [ (p[0] - view.x) / view.k, (p[1] - view.y) / view.k ]; + } + function point(l) { + return [ l[0] * view.k + view.x, l[1] * view.k + view.y ]; + } + function scaleTo(s) { + view.k = Math.max(scaleExtent[0], Math.min(scaleExtent[1], s)); + } + function translateTo(p, l) { + l = point(l); + view.x += p[0] - l[0]; + view.y += p[1] - l[1]; + } + function rescale() { + if (x1) x1.domain(x0.range().map(function(x) { + return (x - view.x) / view.k; + }).map(x0.invert)); + if (y1) y1.domain(y0.range().map(function(y) { + return (y - view.y) / view.k; + }).map(y0.invert)); + } + function zoomstarted(dispatch) { + dispatch({ + type: "zoomstart" + }); + } + function zoomed(dispatch) { + rescale(); + dispatch({ + type: "zoom", + scale: view.k, + translate: [ view.x, view.y ] + }); + } + function zoomended(dispatch) { + dispatch({ + type: "zoomend" + }); + } + function mousedowned() { + var that = this, target = d3.event.target, dispatch = event.of(that, arguments), dragged = 0, subject = d3.select(d3_window).on(mousemove, moved).on(mouseup, ended), location0 = location(d3.mouse(that)), dragRestore = d3_event_dragSuppress(); + d3_selection_interrupt.call(that); + zoomstarted(dispatch); + function moved() { + dragged = 1; + translateTo(d3.mouse(that), location0); + zoomed(dispatch); + } + function ended() { + subject.on(mousemove, d3_window === that ? mousewheelreset : null).on(mouseup, null); + dragRestore(dragged && d3.event.target === target); + zoomended(dispatch); + } + } + function touchstarted() { + var that = this, dispatch = event.of(that, arguments), locations0 = {}, distance0 = 0, scale0, zoomName = ".zoom-" + d3.event.changedTouches[0].identifier, touchmove = "touchmove" + zoomName, touchend = "touchend" + zoomName, target = d3.select(d3.event.target).on(touchmove, moved).on(touchend, ended), subject = d3.select(that).on(mousedown, null).on(touchstart, started), dragRestore = d3_event_dragSuppress(); + d3_selection_interrupt.call(that); + started(); + zoomstarted(dispatch); + function relocate() { + var touches = d3.touches(that); + scale0 = view.k; + touches.forEach(function(t) { + if (t.identifier in locations0) locations0[t.identifier] = location(t); + }); + return touches; + } + function started() { + var changed = d3.event.changedTouches; + for (var i = 0, n = changed.length; i < n; ++i) { + locations0[changed[i].identifier] = null; + } + var touches = relocate(), now = Date.now(); + if (touches.length === 1) { + if (now - touchtime < 500) { + var p = touches[0], l = locations0[p.identifier]; + scaleTo(view.k * 2); + translateTo(p, l); + d3_eventPreventDefault(); + zoomed(dispatch); + } + touchtime = now; + } else if (touches.length > 1) { + var p = touches[0], q = touches[1], dx = p[0] - q[0], dy = p[1] - q[1]; + distance0 = dx * dx + dy * dy; + } + } + function moved() { + var touches = d3.touches(that), p0, l0, p1, l1; + for (var i = 0, n = touches.length; i < n; ++i, l1 = null) { + p1 = touches[i]; + if (l1 = locations0[p1.identifier]) { + if (l0) break; + p0 = p1, l0 = l1; + } + } + if (l1) { + var distance1 = (distance1 = p1[0] - p0[0]) * distance1 + (distance1 = p1[1] - p0[1]) * distance1, scale1 = distance0 && Math.sqrt(distance1 / distance0); + p0 = [ (p0[0] + p1[0]) / 2, (p0[1] + p1[1]) / 2 ]; + l0 = [ (l0[0] + l1[0]) / 2, (l0[1] + l1[1]) / 2 ]; + scaleTo(scale1 * scale0); + } + touchtime = null; + translateTo(p0, l0); + zoomed(dispatch); + } + function ended() { + if (d3.event.touches.length) { + var changed = d3.event.changedTouches; + for (var i = 0, n = changed.length; i < n; ++i) { + delete locations0[changed[i].identifier]; + } + for (var identifier in locations0) { + return void relocate(); + } + } + target.on(zoomName, null); + subject.on(mousedown, mousedowned).on(touchstart, touchstarted); + dragRestore(); + zoomended(dispatch); + } + } + function mousewheeled() { + var dispatch = event.of(this, arguments); + if (mousewheelTimer) clearTimeout(mousewheelTimer); else d3_selection_interrupt.call(this), + zoomstarted(dispatch); + mousewheelTimer = setTimeout(function() { + mousewheelTimer = null; + zoomended(dispatch); + }, 50); + d3_eventPreventDefault(); + var point = center || d3.mouse(this); + if (!translate0) translate0 = location(point); + scaleTo(Math.pow(2, d3_behavior_zoomDelta() * .002) * view.k); + translateTo(point, translate0); + zoomed(dispatch); + } + function mousewheelreset() { + translate0 = null; + } + function dblclicked() { + var dispatch = event.of(this, arguments), p = d3.mouse(this), l = location(p), k = Math.log(view.k) / Math.LN2; + zoomstarted(dispatch); + scaleTo(Math.pow(2, d3.event.shiftKey ? Math.ceil(k) - 1 : Math.floor(k) + 1)); + translateTo(p, l); + zoomed(dispatch); + zoomended(dispatch); + } + return d3.rebind(zoom, event, "on"); + }; + var d3_behavior_zoomInfinity = [ 0, Infinity ]; + var d3_behavior_zoomDelta, d3_behavior_zoomWheel = "onwheel" in d3_document ? (d3_behavior_zoomDelta = function() { + return -d3.event.deltaY * (d3.event.deltaMode ? 120 : 1); + }, "wheel") : "onmousewheel" in d3_document ? (d3_behavior_zoomDelta = function() { + return d3.event.wheelDelta; + }, "mousewheel") : (d3_behavior_zoomDelta = function() { + return -d3.event.detail; + }, "MozMousePixelScroll"); + function d3_Color() {} + d3_Color.prototype.toString = function() { + return this.rgb() + ""; + }; + d3.hsl = function(h, s, l) { + return arguments.length === 1 ? h instanceof d3_Hsl ? d3_hsl(h.h, h.s, h.l) : d3_rgb_parse("" + h, d3_rgb_hsl, d3_hsl) : d3_hsl(+h, +s, +l); + }; + function d3_hsl(h, s, l) { + return new d3_Hsl(h, s, l); + } + function d3_Hsl(h, s, l) { + this.h = h; + this.s = s; + this.l = l; + } + var d3_hslPrototype = d3_Hsl.prototype = new d3_Color(); + d3_hslPrototype.brighter = function(k) { + k = Math.pow(.7, arguments.length ? k : 1); + return d3_hsl(this.h, this.s, this.l / k); + }; + d3_hslPrototype.darker = function(k) { + k = Math.pow(.7, arguments.length ? k : 1); + return d3_hsl(this.h, this.s, k * this.l); + }; + d3_hslPrototype.rgb = function() { + return d3_hsl_rgb(this.h, this.s, this.l); + }; + function d3_hsl_rgb(h, s, l) { + var m1, m2; + h = isNaN(h) ? 0 : (h %= 360) < 0 ? h + 360 : h; + s = isNaN(s) ? 0 : s < 0 ? 0 : s > 1 ? 1 : s; + l = l < 0 ? 0 : l > 1 ? 1 : l; + m2 = l <= .5 ? l * (1 + s) : l + s - l * s; + m1 = 2 * l - m2; + function v(h) { + if (h > 360) h -= 360; else if (h < 0) h += 360; + if (h < 60) return m1 + (m2 - m1) * h / 60; + if (h < 180) return m2; + if (h < 240) return m1 + (m2 - m1) * (240 - h) / 60; + return m1; + } + function vv(h) { + return Math.round(v(h) * 255); + } + return d3_rgb(vv(h + 120), vv(h), vv(h - 120)); + } + d3.hcl = function(h, c, l) { + return arguments.length === 1 ? h instanceof d3_Hcl ? d3_hcl(h.h, h.c, h.l) : h instanceof d3_Lab ? d3_lab_hcl(h.l, h.a, h.b) : d3_lab_hcl((h = d3_rgb_lab((h = d3.rgb(h)).r, h.g, h.b)).l, h.a, h.b) : d3_hcl(+h, +c, +l); + }; + function d3_hcl(h, c, l) { + return new d3_Hcl(h, c, l); + } + function d3_Hcl(h, c, l) { + this.h = h; + this.c = c; + this.l = l; + } + var d3_hclPrototype = d3_Hcl.prototype = new d3_Color(); + d3_hclPrototype.brighter = function(k) { + return d3_hcl(this.h, this.c, Math.min(100, this.l + d3_lab_K * (arguments.length ? k : 1))); + }; + d3_hclPrototype.darker = function(k) { + return d3_hcl(this.h, this.c, Math.max(0, this.l - d3_lab_K * (arguments.length ? k : 1))); + }; + d3_hclPrototype.rgb = function() { + return d3_hcl_lab(this.h, this.c, this.l).rgb(); + }; + function d3_hcl_lab(h, c, l) { + if (isNaN(h)) h = 0; + if (isNaN(c)) c = 0; + return d3_lab(l, Math.cos(h *= d3_radians) * c, Math.sin(h) * c); + } + d3.lab = function(l, a, b) { + return arguments.length === 1 ? l instanceof d3_Lab ? d3_lab(l.l, l.a, l.b) : l instanceof d3_Hcl ? d3_hcl_lab(l.l, l.c, l.h) : d3_rgb_lab((l = d3.rgb(l)).r, l.g, l.b) : d3_lab(+l, +a, +b); + }; + function d3_lab(l, a, b) { + return new d3_Lab(l, a, b); + } + function d3_Lab(l, a, b) { + this.l = l; + this.a = a; + this.b = b; + } + var d3_lab_K = 18; + var d3_lab_X = .95047, d3_lab_Y = 1, d3_lab_Z = 1.08883; + var d3_labPrototype = d3_Lab.prototype = new d3_Color(); + d3_labPrototype.brighter = function(k) { + return d3_lab(Math.min(100, this.l + d3_lab_K * (arguments.length ? k : 1)), this.a, this.b); + }; + d3_labPrototype.darker = function(k) { + return d3_lab(Math.max(0, this.l - d3_lab_K * (arguments.length ? k : 1)), this.a, this.b); + }; + d3_labPrototype.rgb = function() { + return d3_lab_rgb(this.l, this.a, this.b); + }; + function d3_lab_rgb(l, a, b) { + var y = (l + 16) / 116, x = y + a / 500, z = y - b / 200; + x = d3_lab_xyz(x) * d3_lab_X; + y = d3_lab_xyz(y) * d3_lab_Y; + z = d3_lab_xyz(z) * d3_lab_Z; + return d3_rgb(d3_xyz_rgb(3.2404542 * x - 1.5371385 * y - .4985314 * z), d3_xyz_rgb(-.969266 * x + 1.8760108 * y + .041556 * z), d3_xyz_rgb(.0556434 * x - .2040259 * y + 1.0572252 * z)); + } + function d3_lab_hcl(l, a, b) { + return l > 0 ? d3_hcl(Math.atan2(b, a) * d3_degrees, Math.sqrt(a * a + b * b), l) : d3_hcl(NaN, NaN, l); + } + function d3_lab_xyz(x) { + return x > .206893034 ? x * x * x : (x - 4 / 29) / 7.787037; + } + function d3_xyz_lab(x) { + return x > .008856 ? Math.pow(x, 1 / 3) : 7.787037 * x + 4 / 29; + } + function d3_xyz_rgb(r) { + return Math.round(255 * (r <= .00304 ? 12.92 * r : 1.055 * Math.pow(r, 1 / 2.4) - .055)); + } + d3.rgb = function(r, g, b) { + return arguments.length === 1 ? r instanceof d3_Rgb ? d3_rgb(r.r, r.g, r.b) : d3_rgb_parse("" + r, d3_rgb, d3_hsl_rgb) : d3_rgb(~~r, ~~g, ~~b); + }; + function d3_rgbNumber(value) { + return d3_rgb(value >> 16, value >> 8 & 255, value & 255); + } + function d3_rgbString(value) { + return d3_rgbNumber(value) + ""; + } + function d3_rgb(r, g, b) { + return new d3_Rgb(r, g, b); + } + function d3_Rgb(r, g, b) { + this.r = r; + this.g = g; + this.b = b; + } + var d3_rgbPrototype = d3_Rgb.prototype = new d3_Color(); + d3_rgbPrototype.brighter = function(k) { + k = Math.pow(.7, arguments.length ? k : 1); + var r = this.r, g = this.g, b = this.b, i = 30; + if (!r && !g && !b) return d3_rgb(i, i, i); + if (r && r < i) r = i; + if (g && g < i) g = i; + if (b && b < i) b = i; + return d3_rgb(Math.min(255, ~~(r / k)), Math.min(255, ~~(g / k)), Math.min(255, ~~(b / k))); + }; + d3_rgbPrototype.darker = function(k) { + k = Math.pow(.7, arguments.length ? k : 1); + return d3_rgb(~~(k * this.r), ~~(k * this.g), ~~(k * this.b)); + }; + d3_rgbPrototype.hsl = function() { + return d3_rgb_hsl(this.r, this.g, this.b); + }; + d3_rgbPrototype.toString = function() { + return "#" + d3_rgb_hex(this.r) + d3_rgb_hex(this.g) + d3_rgb_hex(this.b); + }; + function d3_rgb_hex(v) { + return v < 16 ? "0" + Math.max(0, v).toString(16) : Math.min(255, v).toString(16); + } + function d3_rgb_parse(format, rgb, hsl) { + var r = 0, g = 0, b = 0, m1, m2, color; + m1 = /([a-z]+)\((.*)\)/i.exec(format); + if (m1) { + m2 = m1[2].split(","); + switch (m1[1]) { + case "hsl": + { + return hsl(parseFloat(m2[0]), parseFloat(m2[1]) / 100, parseFloat(m2[2]) / 100); + } + + case "rgb": + { + return rgb(d3_rgb_parseNumber(m2[0]), d3_rgb_parseNumber(m2[1]), d3_rgb_parseNumber(m2[2])); + } + } + } + if (color = d3_rgb_names.get(format)) return rgb(color.r, color.g, color.b); + if (format != null && format.charAt(0) === "#" && !isNaN(color = parseInt(format.substring(1), 16))) { + if (format.length === 4) { + r = (color & 3840) >> 4; + r = r >> 4 | r; + g = color & 240; + g = g >> 4 | g; + b = color & 15; + b = b << 4 | b; + } else if (format.length === 7) { + r = (color & 16711680) >> 16; + g = (color & 65280) >> 8; + b = color & 255; + } + } + return rgb(r, g, b); + } + function d3_rgb_hsl(r, g, b) { + var min = Math.min(r /= 255, g /= 255, b /= 255), max = Math.max(r, g, b), d = max - min, h, s, l = (max + min) / 2; + if (d) { + s = l < .5 ? d / (max + min) : d / (2 - max - min); + if (r == max) h = (g - b) / d + (g < b ? 6 : 0); else if (g == max) h = (b - r) / d + 2; else h = (r - g) / d + 4; + h *= 60; + } else { + h = NaN; + s = l > 0 && l < 1 ? 0 : h; + } + return d3_hsl(h, s, l); + } + function d3_rgb_lab(r, g, b) { + r = d3_rgb_xyz(r); + g = d3_rgb_xyz(g); + b = d3_rgb_xyz(b); + var x = d3_xyz_lab((.4124564 * r + .3575761 * g + .1804375 * b) / d3_lab_X), y = d3_xyz_lab((.2126729 * r + .7151522 * g + .072175 * b) / d3_lab_Y), z = d3_xyz_lab((.0193339 * r + .119192 * g + .9503041 * b) / d3_lab_Z); + return d3_lab(116 * y - 16, 500 * (x - y), 200 * (y - z)); + } + function d3_rgb_xyz(r) { + return (r /= 255) <= .04045 ? r / 12.92 : Math.pow((r + .055) / 1.055, 2.4); + } + function d3_rgb_parseNumber(c) { + var f = parseFloat(c); + return c.charAt(c.length - 1) === "%" ? Math.round(f * 2.55) : f; + } + var d3_rgb_names = d3.map({ + aliceblue: 15792383, + antiquewhite: 16444375, + aqua: 65535, + aquamarine: 8388564, + azure: 15794175, + beige: 16119260, + bisque: 16770244, + black: 0, + blanchedalmond: 16772045, + blue: 255, + blueviolet: 9055202, + brown: 10824234, + burlywood: 14596231, + cadetblue: 6266528, + chartreuse: 8388352, + chocolate: 13789470, + coral: 16744272, + cornflowerblue: 6591981, + cornsilk: 16775388, + crimson: 14423100, + cyan: 65535, + darkblue: 139, + darkcyan: 35723, + darkgoldenrod: 12092939, + darkgray: 11119017, + darkgreen: 25600, + darkgrey: 11119017, + darkkhaki: 12433259, + darkmagenta: 9109643, + darkolivegreen: 5597999, + darkorange: 16747520, + darkorchid: 10040012, + darkred: 9109504, + darksalmon: 15308410, + darkseagreen: 9419919, + darkslateblue: 4734347, + darkslategray: 3100495, + darkslategrey: 3100495, + darkturquoise: 52945, + darkviolet: 9699539, + deeppink: 16716947, + deepskyblue: 49151, + dimgray: 6908265, + dimgrey: 6908265, + dodgerblue: 2003199, + firebrick: 11674146, + floralwhite: 16775920, + forestgreen: 2263842, + fuchsia: 16711935, + gainsboro: 14474460, + ghostwhite: 16316671, + gold: 16766720, + goldenrod: 14329120, + gray: 8421504, + green: 32768, + greenyellow: 11403055, + grey: 8421504, + honeydew: 15794160, + hotpink: 16738740, + indianred: 13458524, + indigo: 4915330, + ivory: 16777200, + khaki: 15787660, + lavender: 15132410, + lavenderblush: 16773365, + lawngreen: 8190976, + lemonchiffon: 16775885, + lightblue: 11393254, + lightcoral: 15761536, + lightcyan: 14745599, + lightgoldenrodyellow: 16448210, + lightgray: 13882323, + lightgreen: 9498256, + lightgrey: 13882323, + lightpink: 16758465, + lightsalmon: 16752762, + lightseagreen: 2142890, + lightskyblue: 8900346, + lightslategray: 7833753, + lightslategrey: 7833753, + lightsteelblue: 11584734, + lightyellow: 16777184, + lime: 65280, + limegreen: 3329330, + linen: 16445670, + magenta: 16711935, + maroon: 8388608, + mediumaquamarine: 6737322, + mediumblue: 205, + mediumorchid: 12211667, + mediumpurple: 9662683, + mediumseagreen: 3978097, + mediumslateblue: 8087790, + mediumspringgreen: 64154, + mediumturquoise: 4772300, + mediumvioletred: 13047173, + midnightblue: 1644912, + mintcream: 16121850, + mistyrose: 16770273, + moccasin: 16770229, + navajowhite: 16768685, + navy: 128, + oldlace: 16643558, + olive: 8421376, + olivedrab: 7048739, + orange: 16753920, + orangered: 16729344, + orchid: 14315734, + palegoldenrod: 15657130, + palegreen: 10025880, + paleturquoise: 11529966, + palevioletred: 14381203, + papayawhip: 16773077, + peachpuff: 16767673, + peru: 13468991, + pink: 16761035, + plum: 14524637, + powderblue: 11591910, + purple: 8388736, + red: 16711680, + rosybrown: 12357519, + royalblue: 4286945, + saddlebrown: 9127187, + salmon: 16416882, + sandybrown: 16032864, + seagreen: 3050327, + seashell: 16774638, + sienna: 10506797, + silver: 12632256, + skyblue: 8900331, + slateblue: 6970061, + slategray: 7372944, + slategrey: 7372944, + snow: 16775930, + springgreen: 65407, + steelblue: 4620980, + tan: 13808780, + teal: 32896, + thistle: 14204888, + tomato: 16737095, + turquoise: 4251856, + violet: 15631086, + wheat: 16113331, + white: 16777215, + whitesmoke: 16119285, + yellow: 16776960, + yellowgreen: 10145074 + }); + d3_rgb_names.forEach(function(key, value) { + d3_rgb_names.set(key, d3_rgbNumber(value)); + }); + function d3_functor(v) { + return typeof v === "function" ? v : function() { + return v; + }; + } + d3.functor = d3_functor; + function d3_identity(d) { + return d; + } + d3.xhr = d3_xhrType(d3_identity); + function d3_xhrType(response) { + return function(url, mimeType, callback) { + if (arguments.length === 2 && typeof mimeType === "function") callback = mimeType, + mimeType = null; + return d3_xhr(url, mimeType, response, callback); + }; + } + function d3_xhr(url, mimeType, response, callback) { + var xhr = {}, dispatch = d3.dispatch("beforesend", "progress", "load", "error"), headers = {}, request = new XMLHttpRequest(), responseType = null; + if (d3_window.XDomainRequest && !("withCredentials" in request) && /^(http(s)?:)?\/\//.test(url)) request = new XDomainRequest(); + "onload" in request ? request.onload = request.onerror = respond : request.onreadystatechange = function() { + request.readyState > 3 && respond(); + }; + function respond() { + var status = request.status, result; + if (!status && request.responseText || status >= 200 && status < 300 || status === 304) { + try { + result = response.call(xhr, request); + } catch (e) { + dispatch.error.call(xhr, e); + return; + } + dispatch.load.call(xhr, result); + } else { + dispatch.error.call(xhr, request); + } + } + request.onprogress = function(event) { + var o = d3.event; + d3.event = event; + try { + dispatch.progress.call(xhr, request); + } finally { + d3.event = o; + } + }; + xhr.header = function(name, value) { + name = (name + "").toLowerCase(); + if (arguments.length < 2) return headers[name]; + if (value == null) delete headers[name]; else headers[name] = value + ""; + return xhr; + }; + xhr.mimeType = function(value) { + if (!arguments.length) return mimeType; + mimeType = value == null ? null : value + ""; + return xhr; + }; + xhr.responseType = function(value) { + if (!arguments.length) return responseType; + responseType = value; + return xhr; + }; + xhr.response = function(value) { + response = value; + return xhr; + }; + [ "get", "post" ].forEach(function(method) { + xhr[method] = function() { + return xhr.send.apply(xhr, [ method ].concat(d3_array(arguments))); + }; + }); + xhr.send = function(method, data, callback) { + if (arguments.length === 2 && typeof data === "function") callback = data, data = null; + request.open(method, url, true); + if (mimeType != null && !("accept" in headers)) headers["accept"] = mimeType + ",*/*"; + if (request.setRequestHeader) for (var name in headers) request.setRequestHeader(name, headers[name]); + if (mimeType != null && request.overrideMimeType) request.overrideMimeType(mimeType); + if (responseType != null) request.responseType = responseType; + if (callback != null) xhr.on("error", callback).on("load", function(request) { + callback(null, request); + }); + dispatch.beforesend.call(xhr, request); + request.send(data == null ? null : data); + return xhr; + }; + xhr.abort = function() { + request.abort(); + return xhr; + }; + d3.rebind(xhr, dispatch, "on"); + return callback == null ? xhr : xhr.get(d3_xhr_fixCallback(callback)); + } + function d3_xhr_fixCallback(callback) { + return callback.length === 1 ? function(error, request) { + callback(error == null ? request : null); + } : callback; + } + d3.dsv = function(delimiter, mimeType) { + var reFormat = new RegExp('["' + delimiter + "\n]"), delimiterCode = delimiter.charCodeAt(0); + function dsv(url, row, callback) { + if (arguments.length < 3) callback = row, row = null; + var xhr = d3_xhr(url, mimeType, row == null ? response : typedResponse(row), callback); + xhr.row = function(_) { + return arguments.length ? xhr.response((row = _) == null ? response : typedResponse(_)) : row; + }; + return xhr; + } + function response(request) { + return dsv.parse(request.responseText); + } + function typedResponse(f) { + return function(request) { + return dsv.parse(request.responseText, f); + }; + } + dsv.parse = function(text, f) { + var o; + return dsv.parseRows(text, function(row, i) { + if (o) return o(row, i - 1); + var a = new Function("d", "return {" + row.map(function(name, i) { + return JSON.stringify(name) + ": d[" + i + "]"; + }).join(",") + "}"); + o = f ? function(row, i) { + return f(a(row), i); + } : a; + }); + }; + dsv.parseRows = function(text, f) { + var EOL = {}, EOF = {}, rows = [], N = text.length, I = 0, n = 0, t, eol; + function token() { + if (I >= N) return EOF; + if (eol) return eol = false, EOL; + var j = I; + if (text.charCodeAt(j) === 34) { + var i = j; + while (i++ < N) { + if (text.charCodeAt(i) === 34) { + if (text.charCodeAt(i + 1) !== 34) break; + ++i; + } + } + I = i + 2; + var c = text.charCodeAt(i + 1); + if (c === 13) { + eol = true; + if (text.charCodeAt(i + 2) === 10) ++I; + } else if (c === 10) { + eol = true; + } + return text.substring(j + 1, i).replace(/""/g, '"'); + } + while (I < N) { + var c = text.charCodeAt(I++), k = 1; + if (c === 10) eol = true; else if (c === 13) { + eol = true; + if (text.charCodeAt(I) === 10) ++I, ++k; + } else if (c !== delimiterCode) continue; + return text.substring(j, I - k); + } + return text.substring(j); + } + while ((t = token()) !== EOF) { + var a = []; + while (t !== EOL && t !== EOF) { + a.push(t); + t = token(); + } + if (f && !(a = f(a, n++))) continue; + rows.push(a); + } + return rows; + }; + dsv.format = function(rows) { + if (Array.isArray(rows[0])) return dsv.formatRows(rows); + var fieldSet = new d3_Set(), fields = []; + rows.forEach(function(row) { + for (var field in row) { + if (!fieldSet.has(field)) { + fields.push(fieldSet.add(field)); + } + } + }); + return [ fields.map(formatValue).join(delimiter) ].concat(rows.map(function(row) { + return fields.map(function(field) { + return formatValue(row[field]); + }).join(delimiter); + })).join("\n"); + }; + dsv.formatRows = function(rows) { + return rows.map(formatRow).join("\n"); + }; + function formatRow(row) { + return row.map(formatValue).join(delimiter); + } + function formatValue(text) { + return reFormat.test(text) ? '"' + text.replace(/\"/g, '""') + '"' : text; + } + return dsv; + }; + d3.csv = d3.dsv(",", "text/csv"); + d3.tsv = d3.dsv(" ", "text/tab-separated-values"); + d3.touch = function(container, touches, identifier) { + if (arguments.length < 3) identifier = touches, touches = d3_eventSource().changedTouches; + if (touches) for (var i = 0, n = touches.length, touch; i < n; ++i) { + if ((touch = touches[i]).identifier === identifier) { + return d3_mousePoint(container, touch); + } + } + }; + var d3_timer_queueHead, d3_timer_queueTail, d3_timer_interval, d3_timer_timeout, d3_timer_active, d3_timer_frame = d3_window[d3_vendorSymbol(d3_window, "requestAnimationFrame")] || function(callback) { + setTimeout(callback, 17); + }; + d3.timer = function(callback, delay, then) { + var n = arguments.length; + if (n < 2) delay = 0; + if (n < 3) then = Date.now(); + var time = then + delay, timer = { + c: callback, + t: time, + f: false, + n: null + }; + if (d3_timer_queueTail) d3_timer_queueTail.n = timer; else d3_timer_queueHead = timer; + d3_timer_queueTail = timer; + if (!d3_timer_interval) { + d3_timer_timeout = clearTimeout(d3_timer_timeout); + d3_timer_interval = 1; + d3_timer_frame(d3_timer_step); + } + }; + function d3_timer_step() { + var now = d3_timer_mark(), delay = d3_timer_sweep() - now; + if (delay > 24) { + if (isFinite(delay)) { + clearTimeout(d3_timer_timeout); + d3_timer_timeout = setTimeout(d3_timer_step, delay); + } + d3_timer_interval = 0; + } else { + d3_timer_interval = 1; + d3_timer_frame(d3_timer_step); + } + } + d3.timer.flush = function() { + d3_timer_mark(); + d3_timer_sweep(); + }; + function d3_timer_mark() { + var now = Date.now(); + d3_timer_active = d3_timer_queueHead; + while (d3_timer_active) { + if (now >= d3_timer_active.t) d3_timer_active.f = d3_timer_active.c(now - d3_timer_active.t); + d3_timer_active = d3_timer_active.n; + } + return now; + } + function d3_timer_sweep() { + var t0, t1 = d3_timer_queueHead, time = Infinity; + while (t1) { + if (t1.f) { + t1 = t0 ? t0.n = t1.n : d3_timer_queueHead = t1.n; + } else { + if (t1.t < time) time = t1.t; + t1 = (t0 = t1).n; + } + } + d3_timer_queueTail = t0; + return time; + } + function d3_format_precision(x, p) { + return p - (x ? Math.ceil(Math.log(x) / Math.LN10) : 1); + } + d3.round = function(x, n) { + return n ? Math.round(x * (n = Math.pow(10, n))) / n : Math.round(x); + }; + var d3_formatPrefixes = [ "y", "z", "a", "f", "p", "n", "µ", "m", "", "k", "M", "G", "T", "P", "E", "Z", "Y" ].map(d3_formatPrefix); + d3.formatPrefix = function(value, precision) { + var i = 0; + if (value) { + if (value < 0) value *= -1; + if (precision) value = d3.round(value, d3_format_precision(value, precision)); + i = 1 + Math.floor(1e-12 + Math.log(value) / Math.LN10); + i = Math.max(-24, Math.min(24, Math.floor((i - 1) / 3) * 3)); + } + return d3_formatPrefixes[8 + i / 3]; + }; + function d3_formatPrefix(d, i) { + var k = Math.pow(10, abs(8 - i) * 3); + return { + scale: i > 8 ? function(d) { + return d / k; + } : function(d) { + return d * k; + }, + symbol: d + }; + } + function d3_locale_numberFormat(locale) { + var locale_decimal = locale.decimal, locale_thousands = locale.thousands, locale_grouping = locale.grouping, locale_currency = locale.currency, formatGroup = locale_grouping ? function(value) { + var i = value.length, t = [], j = 0, g = locale_grouping[0]; + while (i > 0 && g > 0) { + t.push(value.substring(i -= g, i + g)); + g = locale_grouping[j = (j + 1) % locale_grouping.length]; + } + return t.reverse().join(locale_thousands); + } : d3_identity; + return function(specifier) { + var match = d3_format_re.exec(specifier), fill = match[1] || " ", align = match[2] || ">", sign = match[3] || "", symbol = match[4] || "", zfill = match[5], width = +match[6], comma = match[7], precision = match[8], type = match[9], scale = 1, prefix = "", suffix = "", integer = false; + if (precision) precision = +precision.substring(1); + if (zfill || fill === "0" && align === "=") { + zfill = fill = "0"; + align = "="; + if (comma) width -= Math.floor((width - 1) / 4); + } + switch (type) { + case "n": + comma = true; + type = "g"; + break; + + case "%": + scale = 100; + suffix = "%"; + type = "f"; + break; + + case "p": + scale = 100; + suffix = "%"; + type = "r"; + break; + + case "b": + case "o": + case "x": + case "X": + if (symbol === "#") prefix = "0" + type.toLowerCase(); + + case "c": + case "d": + integer = true; + precision = 0; + break; + + case "s": + scale = -1; + type = "r"; + break; + } + if (symbol === "$") prefix = locale_currency[0], suffix = locale_currency[1]; + if (type == "r" && !precision) type = "g"; + if (precision != null) { + if (type == "g") precision = Math.max(1, Math.min(21, precision)); else if (type == "e" || type == "f") precision = Math.max(0, Math.min(20, precision)); + } + type = d3_format_types.get(type) || d3_format_typeDefault; + var zcomma = zfill && comma; + return function(value) { + var fullSuffix = suffix; + if (integer && value % 1) return ""; + var negative = value < 0 || value === 0 && 1 / value < 0 ? (value = -value, "-") : sign; + if (scale < 0) { + var unit = d3.formatPrefix(value, precision); + value = unit.scale(value); + fullSuffix = unit.symbol + suffix; + } else { + value *= scale; + } + value = type(value, precision); + var i = value.lastIndexOf("."), before = i < 0 ? value : value.substring(0, i), after = i < 0 ? "" : locale_decimal + value.substring(i + 1); + if (!zfill && comma) before = formatGroup(before); + var length = prefix.length + before.length + after.length + (zcomma ? 0 : negative.length), padding = length < width ? new Array(length = width - length + 1).join(fill) : ""; + if (zcomma) before = formatGroup(padding + before); + negative += prefix; + value = before + after; + return (align === "<" ? negative + value + padding : align === ">" ? padding + negative + value : align === "^" ? padding.substring(0, length >>= 1) + negative + value + padding.substring(length) : negative + (zcomma ? value : padding + value)) + fullSuffix; + }; + }; + } + var d3_format_re = /(?:([^{])?([<>=^]))?([+\- ])?([$#])?(0)?(\d+)?(,)?(\.-?\d+)?([a-z%])?/i; + var d3_format_types = d3.map({ + b: function(x) { + return x.toString(2); + }, + c: function(x) { + return String.fromCharCode(x); + }, + o: function(x) { + return x.toString(8); + }, + x: function(x) { + return x.toString(16); + }, + X: function(x) { + return x.toString(16).toUpperCase(); + }, + g: function(x, p) { + return x.toPrecision(p); + }, + e: function(x, p) { + return x.toExponential(p); + }, + f: function(x, p) { + return x.toFixed(p); + }, + r: function(x, p) { + return (x = d3.round(x, d3_format_precision(x, p))).toFixed(Math.max(0, Math.min(20, d3_format_precision(x * (1 + 1e-15), p)))); + } + }); + function d3_format_typeDefault(x) { + return x + ""; + } + var d3_time = d3.time = {}, d3_date = Date; + function d3_date_utc() { + this._ = new Date(arguments.length > 1 ? Date.UTC.apply(this, arguments) : arguments[0]); + } + d3_date_utc.prototype = { + getDate: function() { + return this._.getUTCDate(); + }, + getDay: function() { + return this._.getUTCDay(); + }, + getFullYear: function() { + return this._.getUTCFullYear(); + }, + getHours: function() { + return this._.getUTCHours(); + }, + getMilliseconds: function() { + return this._.getUTCMilliseconds(); + }, + getMinutes: function() { + return this._.getUTCMinutes(); + }, + getMonth: function() { + return this._.getUTCMonth(); + }, + getSeconds: function() { + return this._.getUTCSeconds(); + }, + getTime: function() { + return this._.getTime(); + }, + getTimezoneOffset: function() { + return 0; + }, + valueOf: function() { + return this._.valueOf(); + }, + setDate: function() { + d3_time_prototype.setUTCDate.apply(this._, arguments); + }, + setDay: function() { + d3_time_prototype.setUTCDay.apply(this._, arguments); + }, + setFullYear: function() { + d3_time_prototype.setUTCFullYear.apply(this._, arguments); + }, + setHours: function() { + d3_time_prototype.setUTCHours.apply(this._, arguments); + }, + setMilliseconds: function() { + d3_time_prototype.setUTCMilliseconds.apply(this._, arguments); + }, + setMinutes: function() { + d3_time_prototype.setUTCMinutes.apply(this._, arguments); + }, + setMonth: function() { + d3_time_prototype.setUTCMonth.apply(this._, arguments); + }, + setSeconds: function() { + d3_time_prototype.setUTCSeconds.apply(this._, arguments); + }, + setTime: function() { + d3_time_prototype.setTime.apply(this._, arguments); + } + }; + var d3_time_prototype = Date.prototype; + function d3_time_interval(local, step, number) { + function round(date) { + var d0 = local(date), d1 = offset(d0, 1); + return date - d0 < d1 - date ? d0 : d1; + } + function ceil(date) { + step(date = local(new d3_date(date - 1)), 1); + return date; + } + function offset(date, k) { + step(date = new d3_date(+date), k); + return date; + } + function range(t0, t1, dt) { + var time = ceil(t0), times = []; + if (dt > 1) { + while (time < t1) { + if (!(number(time) % dt)) times.push(new Date(+time)); + step(time, 1); + } + } else { + while (time < t1) times.push(new Date(+time)), step(time, 1); + } + return times; + } + function range_utc(t0, t1, dt) { + try { + d3_date = d3_date_utc; + var utc = new d3_date_utc(); + utc._ = t0; + return range(utc, t1, dt); + } finally { + d3_date = Date; + } + } + local.floor = local; + local.round = round; + local.ceil = ceil; + local.offset = offset; + local.range = range; + var utc = local.utc = d3_time_interval_utc(local); + utc.floor = utc; + utc.round = d3_time_interval_utc(round); + utc.ceil = d3_time_interval_utc(ceil); + utc.offset = d3_time_interval_utc(offset); + utc.range = range_utc; + return local; + } + function d3_time_interval_utc(method) { + return function(date, k) { + try { + d3_date = d3_date_utc; + var utc = new d3_date_utc(); + utc._ = date; + return method(utc, k)._; + } finally { + d3_date = Date; + } + }; + } + d3_time.year = d3_time_interval(function(date) { + date = d3_time.day(date); + date.setMonth(0, 1); + return date; + }, function(date, offset) { + date.setFullYear(date.getFullYear() + offset); + }, function(date) { + return date.getFullYear(); + }); + d3_time.years = d3_time.year.range; + d3_time.years.utc = d3_time.year.utc.range; + d3_time.day = d3_time_interval(function(date) { + var day = new d3_date(2e3, 0); + day.setFullYear(date.getFullYear(), date.getMonth(), date.getDate()); + return day; + }, function(date, offset) { + date.setDate(date.getDate() + offset); + }, function(date) { + return date.getDate() - 1; + }); + d3_time.days = d3_time.day.range; + d3_time.days.utc = d3_time.day.utc.range; + d3_time.dayOfYear = function(date) { + var year = d3_time.year(date); + return Math.floor((date - year - (date.getTimezoneOffset() - year.getTimezoneOffset()) * 6e4) / 864e5); + }; + [ "sunday", "monday", "tuesday", "wednesday", "thursday", "friday", "saturday" ].forEach(function(day, i) { + i = 7 - i; + var interval = d3_time[day] = d3_time_interval(function(date) { + (date = d3_time.day(date)).setDate(date.getDate() - (date.getDay() + i) % 7); + return date; + }, function(date, offset) { + date.setDate(date.getDate() + Math.floor(offset) * 7); + }, function(date) { + var day = d3_time.year(date).getDay(); + return Math.floor((d3_time.dayOfYear(date) + (day + i) % 7) / 7) - (day !== i); + }); + d3_time[day + "s"] = interval.range; + d3_time[day + "s"].utc = interval.utc.range; + d3_time[day + "OfYear"] = function(date) { + var day = d3_time.year(date).getDay(); + return Math.floor((d3_time.dayOfYear(date) + (day + i) % 7) / 7); + }; + }); + d3_time.week = d3_time.sunday; + d3_time.weeks = d3_time.sunday.range; + d3_time.weeks.utc = d3_time.sunday.utc.range; + d3_time.weekOfYear = d3_time.sundayOfYear; + function d3_locale_timeFormat(locale) { + var locale_dateTime = locale.dateTime, locale_date = locale.date, locale_time = locale.time, locale_periods = locale.periods, locale_days = locale.days, locale_shortDays = locale.shortDays, locale_months = locale.months, locale_shortMonths = locale.shortMonths; + function d3_time_format(template) { + var n = template.length; + function format(date) { + var string = [], i = -1, j = 0, c, p, f; + while (++i < n) { + if (template.charCodeAt(i) === 37) { + string.push(template.substring(j, i)); + if ((p = d3_time_formatPads[c = template.charAt(++i)]) != null) c = template.charAt(++i); + if (f = d3_time_formats[c]) c = f(date, p == null ? c === "e" ? " " : "0" : p); + string.push(c); + j = i + 1; + } + } + string.push(template.substring(j, i)); + return string.join(""); + } + format.parse = function(string) { + var d = { + y: 1900, + m: 0, + d: 1, + H: 0, + M: 0, + S: 0, + L: 0, + Z: null + }, i = d3_time_parse(d, template, string, 0); + if (i != string.length) return null; + if ("p" in d) d.H = d.H % 12 + d.p * 12; + var localZ = d.Z != null && d3_date !== d3_date_utc, date = new (localZ ? d3_date_utc : d3_date)(); + if ("j" in d) date.setFullYear(d.y, 0, d.j); else if ("w" in d && ("W" in d || "U" in d)) { + date.setFullYear(d.y, 0, 1); + date.setFullYear(d.y, 0, "W" in d ? (d.w + 6) % 7 + d.W * 7 - (date.getDay() + 5) % 7 : d.w + d.U * 7 - (date.getDay() + 6) % 7); + } else date.setFullYear(d.y, d.m, d.d); + date.setHours(d.H + Math.floor(d.Z / 100), d.M + d.Z % 100, d.S, d.L); + return localZ ? date._ : date; + }; + format.toString = function() { + return template; + }; + return format; + } + function d3_time_parse(date, template, string, j) { + var c, p, t, i = 0, n = template.length, m = string.length; + while (i < n) { + if (j >= m) return -1; + c = template.charCodeAt(i++); + if (c === 37) { + t = template.charAt(i++); + p = d3_time_parsers[t in d3_time_formatPads ? template.charAt(i++) : t]; + if (!p || (j = p(date, string, j)) < 0) return -1; + } else if (c != string.charCodeAt(j++)) { + return -1; + } + } + return j; + } + d3_time_format.utc = function(template) { + var local = d3_time_format(template); + function format(date) { + try { + d3_date = d3_date_utc; + var utc = new d3_date(); + utc._ = date; + return local(utc); + } finally { + d3_date = Date; + } + } + format.parse = function(string) { + try { + d3_date = d3_date_utc; + var date = local.parse(string); + return date && date._; + } finally { + d3_date = Date; + } + }; + format.toString = local.toString; + return format; + }; + d3_time_format.multi = d3_time_format.utc.multi = d3_time_formatMulti; + var d3_time_periodLookup = d3.map(), d3_time_dayRe = d3_time_formatRe(locale_days), d3_time_dayLookup = d3_time_formatLookup(locale_days), d3_time_dayAbbrevRe = d3_time_formatRe(locale_shortDays), d3_time_dayAbbrevLookup = d3_time_formatLookup(locale_shortDays), d3_time_monthRe = d3_time_formatRe(locale_months), d3_time_monthLookup = d3_time_formatLookup(locale_months), d3_time_monthAbbrevRe = d3_time_formatRe(locale_shortMonths), d3_time_monthAbbrevLookup = d3_time_formatLookup(locale_shortMonths); + locale_periods.forEach(function(p, i) { + d3_time_periodLookup.set(p.toLowerCase(), i); + }); + var d3_time_formats = { + a: function(d) { + return locale_shortDays[d.getDay()]; + }, + A: function(d) { + return locale_days[d.getDay()]; + }, + b: function(d) { + return locale_shortMonths[d.getMonth()]; + }, + B: function(d) { + return locale_months[d.getMonth()]; + }, + c: d3_time_format(locale_dateTime), + d: function(d, p) { + return d3_time_formatPad(d.getDate(), p, 2); + }, + e: function(d, p) { + return d3_time_formatPad(d.getDate(), p, 2); + }, + H: function(d, p) { + return d3_time_formatPad(d.getHours(), p, 2); + }, + I: function(d, p) { + return d3_time_formatPad(d.getHours() % 12 || 12, p, 2); + }, + j: function(d, p) { + return d3_time_formatPad(1 + d3_time.dayOfYear(d), p, 3); + }, + L: function(d, p) { + return d3_time_formatPad(d.getMilliseconds(), p, 3); + }, + m: function(d, p) { + return d3_time_formatPad(d.getMonth() + 1, p, 2); + }, + M: function(d, p) { + return d3_time_formatPad(d.getMinutes(), p, 2); + }, + p: function(d) { + return locale_periods[+(d.getHours() >= 12)]; + }, + S: function(d, p) { + return d3_time_formatPad(d.getSeconds(), p, 2); + }, + U: function(d, p) { + return d3_time_formatPad(d3_time.sundayOfYear(d), p, 2); + }, + w: function(d) { + return d.getDay(); + }, + W: function(d, p) { + return d3_time_formatPad(d3_time.mondayOfYear(d), p, 2); + }, + x: d3_time_format(locale_date), + X: d3_time_format(locale_time), + y: function(d, p) { + return d3_time_formatPad(d.getFullYear() % 100, p, 2); + }, + Y: function(d, p) { + return d3_time_formatPad(d.getFullYear() % 1e4, p, 4); + }, + Z: d3_time_zone, + "%": function() { + return "%"; + } + }; + var d3_time_parsers = { + a: d3_time_parseWeekdayAbbrev, + A: d3_time_parseWeekday, + b: d3_time_parseMonthAbbrev, + B: d3_time_parseMonth, + c: d3_time_parseLocaleFull, + d: d3_time_parseDay, + e: d3_time_parseDay, + H: d3_time_parseHour24, + I: d3_time_parseHour24, + j: d3_time_parseDayOfYear, + L: d3_time_parseMilliseconds, + m: d3_time_parseMonthNumber, + M: d3_time_parseMinutes, + p: d3_time_parseAmPm, + S: d3_time_parseSeconds, + U: d3_time_parseWeekNumberSunday, + w: d3_time_parseWeekdayNumber, + W: d3_time_parseWeekNumberMonday, + x: d3_time_parseLocaleDate, + X: d3_time_parseLocaleTime, + y: d3_time_parseYear, + Y: d3_time_parseFullYear, + Z: d3_time_parseZone, + "%": d3_time_parseLiteralPercent + }; + function d3_time_parseWeekdayAbbrev(date, string, i) { + d3_time_dayAbbrevRe.lastIndex = 0; + var n = d3_time_dayAbbrevRe.exec(string.substring(i)); + return n ? (date.w = d3_time_dayAbbrevLookup.get(n[0].toLowerCase()), i + n[0].length) : -1; + } + function d3_time_parseWeekday(date, string, i) { + d3_time_dayRe.lastIndex = 0; + var n = d3_time_dayRe.exec(string.substring(i)); + return n ? (date.w = d3_time_dayLookup.get(n[0].toLowerCase()), i + n[0].length) : -1; + } + function d3_time_parseMonthAbbrev(date, string, i) { + d3_time_monthAbbrevRe.lastIndex = 0; + var n = d3_time_monthAbbrevRe.exec(string.substring(i)); + return n ? (date.m = d3_time_monthAbbrevLookup.get(n[0].toLowerCase()), i + n[0].length) : -1; + } + function d3_time_parseMonth(date, string, i) { + d3_time_monthRe.lastIndex = 0; + var n = d3_time_monthRe.exec(string.substring(i)); + return n ? (date.m = d3_time_monthLookup.get(n[0].toLowerCase()), i + n[0].length) : -1; + } + function d3_time_parseLocaleFull(date, string, i) { + return d3_time_parse(date, d3_time_formats.c.toString(), string, i); + } + function d3_time_parseLocaleDate(date, string, i) { + return d3_time_parse(date, d3_time_formats.x.toString(), string, i); + } + function d3_time_parseLocaleTime(date, string, i) { + return d3_time_parse(date, d3_time_formats.X.toString(), string, i); + } + function d3_time_parseAmPm(date, string, i) { + var n = d3_time_periodLookup.get(string.substring(i, i += 2).toLowerCase()); + return n == null ? -1 : (date.p = n, i); + } + return d3_time_format; + } + var d3_time_formatPads = { + "-": "", + _: " ", + "0": "0" + }, d3_time_numberRe = /^\s*\d+/, d3_time_percentRe = /^%/; + function d3_time_formatPad(value, fill, width) { + var sign = value < 0 ? "-" : "", string = (sign ? -value : value) + "", length = string.length; + return sign + (length < width ? new Array(width - length + 1).join(fill) + string : string); + } + function d3_time_formatRe(names) { + return new RegExp("^(?:" + names.map(d3.requote).join("|") + ")", "i"); + } + function d3_time_formatLookup(names) { + var map = new d3_Map(), i = -1, n = names.length; + while (++i < n) map.set(names[i].toLowerCase(), i); + return map; + } + function d3_time_parseWeekdayNumber(date, string, i) { + d3_time_numberRe.lastIndex = 0; + var n = d3_time_numberRe.exec(string.substring(i, i + 1)); + return n ? (date.w = +n[0], i + n[0].length) : -1; + } + function d3_time_parseWeekNumberSunday(date, string, i) { + d3_time_numberRe.lastIndex = 0; + var n = d3_time_numberRe.exec(string.substring(i)); + return n ? (date.U = +n[0], i + n[0].length) : -1; + } + function d3_time_parseWeekNumberMonday(date, string, i) { + d3_time_numberRe.lastIndex = 0; + var n = d3_time_numberRe.exec(string.substring(i)); + return n ? (date.W = +n[0], i + n[0].length) : -1; + } + function d3_time_parseFullYear(date, string, i) { + d3_time_numberRe.lastIndex = 0; + var n = d3_time_numberRe.exec(string.substring(i, i + 4)); + return n ? (date.y = +n[0], i + n[0].length) : -1; + } + function d3_time_parseYear(date, string, i) { + d3_time_numberRe.lastIndex = 0; + var n = d3_time_numberRe.exec(string.substring(i, i + 2)); + return n ? (date.y = d3_time_expandYear(+n[0]), i + n[0].length) : -1; + } + function d3_time_parseZone(date, string, i) { + return /^[+-]\d{4}$/.test(string = string.substring(i, i + 5)) ? (date.Z = +string, + i + 5) : -1; + } + function d3_time_expandYear(d) { + return d + (d > 68 ? 1900 : 2e3); + } + function d3_time_parseMonthNumber(date, string, i) { + d3_time_numberRe.lastIndex = 0; + var n = d3_time_numberRe.exec(string.substring(i, i + 2)); + return n ? (date.m = n[0] - 1, i + n[0].length) : -1; + } + function d3_time_parseDay(date, string, i) { + d3_time_numberRe.lastIndex = 0; + var n = d3_time_numberRe.exec(string.substring(i, i + 2)); + return n ? (date.d = +n[0], i + n[0].length) : -1; + } + function d3_time_parseDayOfYear(date, string, i) { + d3_time_numberRe.lastIndex = 0; + var n = d3_time_numberRe.exec(string.substring(i, i + 3)); + return n ? (date.j = +n[0], i + n[0].length) : -1; + } + function d3_time_parseHour24(date, string, i) { + d3_time_numberRe.lastIndex = 0; + var n = d3_time_numberRe.exec(string.substring(i, i + 2)); + return n ? (date.H = +n[0], i + n[0].length) : -1; + } + function d3_time_parseMinutes(date, string, i) { + d3_time_numberRe.lastIndex = 0; + var n = d3_time_numberRe.exec(string.substring(i, i + 2)); + return n ? (date.M = +n[0], i + n[0].length) : -1; + } + function d3_time_parseSeconds(date, string, i) { + d3_time_numberRe.lastIndex = 0; + var n = d3_time_numberRe.exec(string.substring(i, i + 2)); + return n ? (date.S = +n[0], i + n[0].length) : -1; + } + function d3_time_parseMilliseconds(date, string, i) { + d3_time_numberRe.lastIndex = 0; + var n = d3_time_numberRe.exec(string.substring(i, i + 3)); + return n ? (date.L = +n[0], i + n[0].length) : -1; + } + function d3_time_zone(d) { + var z = d.getTimezoneOffset(), zs = z > 0 ? "-" : "+", zh = ~~(abs(z) / 60), zm = abs(z) % 60; + return zs + d3_time_formatPad(zh, "0", 2) + d3_time_formatPad(zm, "0", 2); + } + function d3_time_parseLiteralPercent(date, string, i) { + d3_time_percentRe.lastIndex = 0; + var n = d3_time_percentRe.exec(string.substring(i, i + 1)); + return n ? i + n[0].length : -1; + } + function d3_time_formatMulti(formats) { + var n = formats.length, i = -1; + while (++i < n) formats[i][0] = this(formats[i][0]); + return function(date) { + var i = 0, f = formats[i]; + while (!f[1](date)) f = formats[++i]; + return f[0](date); + }; + } + d3.locale = function(locale) { + return { + numberFormat: d3_locale_numberFormat(locale), + timeFormat: d3_locale_timeFormat(locale) + }; + }; + var d3_locale_enUS = d3.locale({ + decimal: ".", + thousands: ",", + grouping: [ 3 ], + currency: [ "$", "" ], + dateTime: "%a %b %e %X %Y", + date: "%m/%d/%Y", + time: "%H:%M:%S", + periods: [ "AM", "PM" ], + days: [ "Sunday", "Monday", "Tuesday", "Wednesday", "Thursday", "Friday", "Saturday" ], + shortDays: [ "Sun", "Mon", "Tue", "Wed", "Thu", "Fri", "Sat" ], + months: [ "January", "February", "March", "April", "May", "June", "July", "August", "September", "October", "November", "December" ], + shortMonths: [ "Jan", "Feb", "Mar", "Apr", "May", "Jun", "Jul", "Aug", "Sep", "Oct", "Nov", "Dec" ] + }); + d3.format = d3_locale_enUS.numberFormat; + d3.geo = {}; + function d3_adder() {} + d3_adder.prototype = { + s: 0, + t: 0, + add: function(y) { + d3_adderSum(y, this.t, d3_adderTemp); + d3_adderSum(d3_adderTemp.s, this.s, this); + if (this.s) this.t += d3_adderTemp.t; else this.s = d3_adderTemp.t; + }, + reset: function() { + this.s = this.t = 0; + }, + valueOf: function() { + return this.s; + } + }; + var d3_adderTemp = new d3_adder(); + function d3_adderSum(a, b, o) { + var x = o.s = a + b, bv = x - a, av = x - bv; + o.t = a - av + (b - bv); + } + d3.geo.stream = function(object, listener) { + if (object && d3_geo_streamObjectType.hasOwnProperty(object.type)) { + d3_geo_streamObjectType[object.type](object, listener); + } else { + d3_geo_streamGeometry(object, listener); + } + }; + function d3_geo_streamGeometry(geometry, listener) { + if (geometry && d3_geo_streamGeometryType.hasOwnProperty(geometry.type)) { + d3_geo_streamGeometryType[geometry.type](geometry, listener); + } + } + var d3_geo_streamObjectType = { + Feature: function(feature, listener) { + d3_geo_streamGeometry(feature.geometry, listener); + }, + FeatureCollection: function(object, listener) { + var features = object.features, i = -1, n = features.length; + while (++i < n) d3_geo_streamGeometry(features[i].geometry, listener); + } + }; + var d3_geo_streamGeometryType = { + Sphere: function(object, listener) { + listener.sphere(); + }, + Point: function(object, listener) { + object = object.coordinates; + listener.point(object[0], object[1], object[2]); + }, + MultiPoint: function(object, listener) { + var coordinates = object.coordinates, i = -1, n = coordinates.length; + while (++i < n) object = coordinates[i], listener.point(object[0], object[1], object[2]); + }, + LineString: function(object, listener) { + d3_geo_streamLine(object.coordinates, listener, 0); + }, + MultiLineString: function(object, listener) { + var coordinates = object.coordinates, i = -1, n = coordinates.length; + while (++i < n) d3_geo_streamLine(coordinates[i], listener, 0); + }, + Polygon: function(object, listener) { + d3_geo_streamPolygon(object.coordinates, listener); + }, + MultiPolygon: function(object, listener) { + var coordinates = object.coordinates, i = -1, n = coordinates.length; + while (++i < n) d3_geo_streamPolygon(coordinates[i], listener); + }, + GeometryCollection: function(object, listener) { + var geometries = object.geometries, i = -1, n = geometries.length; + while (++i < n) d3_geo_streamGeometry(geometries[i], listener); + } + }; + function d3_geo_streamLine(coordinates, listener, closed) { + var i = -1, n = coordinates.length - closed, coordinate; + listener.lineStart(); + while (++i < n) coordinate = coordinates[i], listener.point(coordinate[0], coordinate[1], coordinate[2]); + listener.lineEnd(); + } + function d3_geo_streamPolygon(coordinates, listener) { + var i = -1, n = coordinates.length; + listener.polygonStart(); + while (++i < n) d3_geo_streamLine(coordinates[i], listener, 1); + listener.polygonEnd(); + } + d3.geo.area = function(object) { + d3_geo_areaSum = 0; + d3.geo.stream(object, d3_geo_area); + return d3_geo_areaSum; + }; + var d3_geo_areaSum, d3_geo_areaRingSum = new d3_adder(); + var d3_geo_area = { + sphere: function() { + d3_geo_areaSum += 4 * π; + }, + point: d3_noop, + lineStart: d3_noop, + lineEnd: d3_noop, + polygonStart: function() { + d3_geo_areaRingSum.reset(); + d3_geo_area.lineStart = d3_geo_areaRingStart; + }, + polygonEnd: function() { + var area = 2 * d3_geo_areaRingSum; + d3_geo_areaSum += area < 0 ? 4 * π + area : area; + d3_geo_area.lineStart = d3_geo_area.lineEnd = d3_geo_area.point = d3_noop; + } + }; + function d3_geo_areaRingStart() { + var λ00, φ00, λ0, cosφ0, sinφ0; + d3_geo_area.point = function(λ, φ) { + d3_geo_area.point = nextPoint; + λ0 = (λ00 = λ) * d3_radians, cosφ0 = Math.cos(φ = (φ00 = φ) * d3_radians / 2 + π / 4), + sinφ0 = Math.sin(φ); + }; + function nextPoint(λ, φ) { + λ *= d3_radians; + φ = φ * d3_radians / 2 + π / 4; + var dλ = λ - λ0, sdλ = dλ >= 0 ? 1 : -1, adλ = sdλ * dλ, cosφ = Math.cos(φ), sinφ = Math.sin(φ), k = sinφ0 * sinφ, u = cosφ0 * cosφ + k * Math.cos(adλ), v = k * sdλ * Math.sin(adλ); + d3_geo_areaRingSum.add(Math.atan2(v, u)); + λ0 = λ, cosφ0 = cosφ, sinφ0 = sinφ; + } + d3_geo_area.lineEnd = function() { + nextPoint(λ00, φ00); + }; + } + function d3_geo_cartesian(spherical) { + var λ = spherical[0], φ = spherical[1], cosφ = Math.cos(φ); + return [ cosφ * Math.cos(λ), cosφ * Math.sin(λ), Math.sin(φ) ]; + } + function d3_geo_cartesianDot(a, b) { + return a[0] * b[0] + a[1] * b[1] + a[2] * b[2]; + } + function d3_geo_cartesianCross(a, b) { + return [ a[1] * b[2] - a[2] * b[1], a[2] * b[0] - a[0] * b[2], a[0] * b[1] - a[1] * b[0] ]; + } + function d3_geo_cartesianAdd(a, b) { + a[0] += b[0]; + a[1] += b[1]; + a[2] += b[2]; + } + function d3_geo_cartesianScale(vector, k) { + return [ vector[0] * k, vector[1] * k, vector[2] * k ]; + } + function d3_geo_cartesianNormalize(d) { + var l = Math.sqrt(d[0] * d[0] + d[1] * d[1] + d[2] * d[2]); + d[0] /= l; + d[1] /= l; + d[2] /= l; + } + function d3_geo_spherical(cartesian) { + return [ Math.atan2(cartesian[1], cartesian[0]), d3_asin(cartesian[2]) ]; + } + function d3_geo_sphericalEqual(a, b) { + return abs(a[0] - b[0]) < ε && abs(a[1] - b[1]) < ε; + } + d3.geo.bounds = function() { + var λ0, φ0, λ1, φ1, λ_, λ__, φ__, p0, dλSum, ranges, range; + var bound = { + point: point, + lineStart: lineStart, + lineEnd: lineEnd, + polygonStart: function() { + bound.point = ringPoint; + bound.lineStart = ringStart; + bound.lineEnd = ringEnd; + dλSum = 0; + d3_geo_area.polygonStart(); + }, + polygonEnd: function() { + d3_geo_area.polygonEnd(); + bound.point = point; + bound.lineStart = lineStart; + bound.lineEnd = lineEnd; + if (d3_geo_areaRingSum < 0) λ0 = -(λ1 = 180), φ0 = -(φ1 = 90); else if (dλSum > ε) φ1 = 90; else if (dλSum < -ε) φ0 = -90; + range[0] = λ0, range[1] = λ1; + } + }; + function point(λ, φ) { + ranges.push(range = [ λ0 = λ, λ1 = λ ]); + if (φ < φ0) φ0 = φ; + if (φ > φ1) φ1 = φ; + } + function linePoint(λ, φ) { + var p = d3_geo_cartesian([ λ * d3_radians, φ * d3_radians ]); + if (p0) { + var normal = d3_geo_cartesianCross(p0, p), equatorial = [ normal[1], -normal[0], 0 ], inflection = d3_geo_cartesianCross(equatorial, normal); + d3_geo_cartesianNormalize(inflection); + inflection = d3_geo_spherical(inflection); + var dλ = λ - λ_, s = dλ > 0 ? 1 : -1, λi = inflection[0] * d3_degrees * s, antimeridian = abs(dλ) > 180; + if (antimeridian ^ (s * λ_ < λi && λi < s * λ)) { + var φi = inflection[1] * d3_degrees; + if (φi > φ1) φ1 = φi; + } else if (λi = (λi + 360) % 360 - 180, antimeridian ^ (s * λ_ < λi && λi < s * λ)) { + var φi = -inflection[1] * d3_degrees; + if (φi < φ0) φ0 = φi; + } else { + if (φ < φ0) φ0 = φ; + if (φ > φ1) φ1 = φ; + } + if (antimeridian) { + if (λ < λ_) { + if (angle(λ0, λ) > angle(λ0, λ1)) λ1 = λ; + } else { + if (angle(λ, λ1) > angle(λ0, λ1)) λ0 = λ; + } + } else { + if (λ1 >= λ0) { + if (λ < λ0) λ0 = λ; + if (λ > λ1) λ1 = λ; + } else { + if (λ > λ_) { + if (angle(λ0, λ) > angle(λ0, λ1)) λ1 = λ; + } else { + if (angle(λ, λ1) > angle(λ0, λ1)) λ0 = λ; + } + } + } + } else { + point(λ, φ); + } + p0 = p, λ_ = λ; + } + function lineStart() { + bound.point = linePoint; + } + function lineEnd() { + range[0] = λ0, range[1] = λ1; + bound.point = point; + p0 = null; + } + function ringPoint(λ, φ) { + if (p0) { + var dλ = λ - λ_; + dλSum += abs(dλ) > 180 ? dλ + (dλ > 0 ? 360 : -360) : dλ; + } else λ__ = λ, φ__ = φ; + d3_geo_area.point(λ, φ); + linePoint(λ, φ); + } + function ringStart() { + d3_geo_area.lineStart(); + } + function ringEnd() { + ringPoint(λ__, φ__); + d3_geo_area.lineEnd(); + if (abs(dλSum) > ε) λ0 = -(λ1 = 180); + range[0] = λ0, range[1] = λ1; + p0 = null; + } + function angle(λ0, λ1) { + return (λ1 -= λ0) < 0 ? λ1 + 360 : λ1; + } + function compareRanges(a, b) { + return a[0] - b[0]; + } + function withinRange(x, range) { + return range[0] <= range[1] ? range[0] <= x && x <= range[1] : x < range[0] || range[1] < x; + } + return function(feature) { + φ1 = λ1 = -(λ0 = φ0 = Infinity); + ranges = []; + d3.geo.stream(feature, bound); + var n = ranges.length; + if (n) { + ranges.sort(compareRanges); + for (var i = 1, a = ranges[0], b, merged = [ a ]; i < n; ++i) { + b = ranges[i]; + if (withinRange(b[0], a) || withinRange(b[1], a)) { + if (angle(a[0], b[1]) > angle(a[0], a[1])) a[1] = b[1]; + if (angle(b[0], a[1]) > angle(a[0], a[1])) a[0] = b[0]; + } else { + merged.push(a = b); + } + } + var best = -Infinity, dλ; + for (var n = merged.length - 1, i = 0, a = merged[n], b; i <= n; a = b, ++i) { + b = merged[i]; + if ((dλ = angle(a[1], b[0])) > best) best = dλ, λ0 = b[0], λ1 = a[1]; + } + } + ranges = range = null; + return λ0 === Infinity || φ0 === Infinity ? [ [ NaN, NaN ], [ NaN, NaN ] ] : [ [ λ0, φ0 ], [ λ1, φ1 ] ]; + }; + }(); + d3.geo.centroid = function(object) { + d3_geo_centroidW0 = d3_geo_centroidW1 = d3_geo_centroidX0 = d3_geo_centroidY0 = d3_geo_centroidZ0 = d3_geo_centroidX1 = d3_geo_centroidY1 = d3_geo_centroidZ1 = d3_geo_centroidX2 = d3_geo_centroidY2 = d3_geo_centroidZ2 = 0; + d3.geo.stream(object, d3_geo_centroid); + var x = d3_geo_centroidX2, y = d3_geo_centroidY2, z = d3_geo_centroidZ2, m = x * x + y * y + z * z; + if (m < ε2) { + x = d3_geo_centroidX1, y = d3_geo_centroidY1, z = d3_geo_centroidZ1; + if (d3_geo_centroidW1 < ε) x = d3_geo_centroidX0, y = d3_geo_centroidY0, z = d3_geo_centroidZ0; + m = x * x + y * y + z * z; + if (m < ε2) return [ NaN, NaN ]; + } + return [ Math.atan2(y, x) * d3_degrees, d3_asin(z / Math.sqrt(m)) * d3_degrees ]; + }; + var d3_geo_centroidW0, d3_geo_centroidW1, d3_geo_centroidX0, d3_geo_centroidY0, d3_geo_centroidZ0, d3_geo_centroidX1, d3_geo_centroidY1, d3_geo_centroidZ1, d3_geo_centroidX2, d3_geo_centroidY2, d3_geo_centroidZ2; + var d3_geo_centroid = { + sphere: d3_noop, + point: d3_geo_centroidPoint, + lineStart: d3_geo_centroidLineStart, + lineEnd: d3_geo_centroidLineEnd, + polygonStart: function() { + d3_geo_centroid.lineStart = d3_geo_centroidRingStart; + }, + polygonEnd: function() { + d3_geo_centroid.lineStart = d3_geo_centroidLineStart; + } + }; + function d3_geo_centroidPoint(λ, φ) { + λ *= d3_radians; + var cosφ = Math.cos(φ *= d3_radians); + d3_geo_centroidPointXYZ(cosφ * Math.cos(λ), cosφ * Math.sin(λ), Math.sin(φ)); + } + function d3_geo_centroidPointXYZ(x, y, z) { + ++d3_geo_centroidW0; + d3_geo_centroidX0 += (x - d3_geo_centroidX0) / d3_geo_centroidW0; + d3_geo_centroidY0 += (y - d3_geo_centroidY0) / d3_geo_centroidW0; + d3_geo_centroidZ0 += (z - d3_geo_centroidZ0) / d3_geo_centroidW0; + } + function d3_geo_centroidLineStart() { + var x0, y0, z0; + d3_geo_centroid.point = function(λ, φ) { + λ *= d3_radians; + var cosφ = Math.cos(φ *= d3_radians); + x0 = cosφ * Math.cos(λ); + y0 = cosφ * Math.sin(λ); + z0 = Math.sin(φ); + d3_geo_centroid.point = nextPoint; + d3_geo_centroidPointXYZ(x0, y0, z0); + }; + function nextPoint(λ, φ) { + λ *= d3_radians; + var cosφ = Math.cos(φ *= d3_radians), x = cosφ * Math.cos(λ), y = cosφ * Math.sin(λ), z = Math.sin(φ), w = Math.atan2(Math.sqrt((w = y0 * z - z0 * y) * w + (w = z0 * x - x0 * z) * w + (w = x0 * y - y0 * x) * w), x0 * x + y0 * y + z0 * z); + d3_geo_centroidW1 += w; + d3_geo_centroidX1 += w * (x0 + (x0 = x)); + d3_geo_centroidY1 += w * (y0 + (y0 = y)); + d3_geo_centroidZ1 += w * (z0 + (z0 = z)); + d3_geo_centroidPointXYZ(x0, y0, z0); + } + } + function d3_geo_centroidLineEnd() { + d3_geo_centroid.point = d3_geo_centroidPoint; + } + function d3_geo_centroidRingStart() { + var λ00, φ00, x0, y0, z0; + d3_geo_centroid.point = function(λ, φ) { + λ00 = λ, φ00 = φ; + d3_geo_centroid.point = nextPoint; + λ *= d3_radians; + var cosφ = Math.cos(φ *= d3_radians); + x0 = cosφ * Math.cos(λ); + y0 = cosφ * Math.sin(λ); + z0 = Math.sin(φ); + d3_geo_centroidPointXYZ(x0, y0, z0); + }; + d3_geo_centroid.lineEnd = function() { + nextPoint(λ00, φ00); + d3_geo_centroid.lineEnd = d3_geo_centroidLineEnd; + d3_geo_centroid.point = d3_geo_centroidPoint; + }; + function nextPoint(λ, φ) { + λ *= d3_radians; + var cosφ = Math.cos(φ *= d3_radians), x = cosφ * Math.cos(λ), y = cosφ * Math.sin(λ), z = Math.sin(φ), cx = y0 * z - z0 * y, cy = z0 * x - x0 * z, cz = x0 * y - y0 * x, m = Math.sqrt(cx * cx + cy * cy + cz * cz), u = x0 * x + y0 * y + z0 * z, v = m && -d3_acos(u) / m, w = Math.atan2(m, u); + d3_geo_centroidX2 += v * cx; + d3_geo_centroidY2 += v * cy; + d3_geo_centroidZ2 += v * cz; + d3_geo_centroidW1 += w; + d3_geo_centroidX1 += w * (x0 + (x0 = x)); + d3_geo_centroidY1 += w * (y0 + (y0 = y)); + d3_geo_centroidZ1 += w * (z0 + (z0 = z)); + d3_geo_centroidPointXYZ(x0, y0, z0); + } + } + function d3_true() { + return true; + } + function d3_geo_clipPolygon(segments, compare, clipStartInside, interpolate, listener) { + var subject = [], clip = []; + segments.forEach(function(segment) { + if ((n = segment.length - 1) <= 0) return; + var n, p0 = segment[0], p1 = segment[n]; + if (d3_geo_sphericalEqual(p0, p1)) { + listener.lineStart(); + for (var i = 0; i < n; ++i) listener.point((p0 = segment[i])[0], p0[1]); + listener.lineEnd(); + return; + } + var a = new d3_geo_clipPolygonIntersection(p0, segment, null, true), b = new d3_geo_clipPolygonIntersection(p0, null, a, false); + a.o = b; + subject.push(a); + clip.push(b); + a = new d3_geo_clipPolygonIntersection(p1, segment, null, false); + b = new d3_geo_clipPolygonIntersection(p1, null, a, true); + a.o = b; + subject.push(a); + clip.push(b); + }); + clip.sort(compare); + d3_geo_clipPolygonLinkCircular(subject); + d3_geo_clipPolygonLinkCircular(clip); + if (!subject.length) return; + for (var i = 0, entry = clipStartInside, n = clip.length; i < n; ++i) { + clip[i].e = entry = !entry; + } + var start = subject[0], points, point; + while (1) { + var current = start, isSubject = true; + while (current.v) if ((current = current.n) === start) return; + points = current.z; + listener.lineStart(); + do { + current.v = current.o.v = true; + if (current.e) { + if (isSubject) { + for (var i = 0, n = points.length; i < n; ++i) listener.point((point = points[i])[0], point[1]); + } else { + interpolate(current.x, current.n.x, 1, listener); + } + current = current.n; + } else { + if (isSubject) { + points = current.p.z; + for (var i = points.length - 1; i >= 0; --i) listener.point((point = points[i])[0], point[1]); + } else { + interpolate(current.x, current.p.x, -1, listener); + } + current = current.p; + } + current = current.o; + points = current.z; + isSubject = !isSubject; + } while (!current.v); + listener.lineEnd(); + } + } + function d3_geo_clipPolygonLinkCircular(array) { + if (!(n = array.length)) return; + var n, i = 0, a = array[0], b; + while (++i < n) { + a.n = b = array[i]; + b.p = a; + a = b; + } + a.n = b = array[0]; + b.p = a; + } + function d3_geo_clipPolygonIntersection(point, points, other, entry) { + this.x = point; + this.z = points; + this.o = other; + this.e = entry; + this.v = false; + this.n = this.p = null; + } + function d3_geo_clip(pointVisible, clipLine, interpolate, clipStart) { + return function(rotate, listener) { + var line = clipLine(listener), rotatedClipStart = rotate.invert(clipStart[0], clipStart[1]); + var clip = { + point: point, + lineStart: lineStart, + lineEnd: lineEnd, + polygonStart: function() { + clip.point = pointRing; + clip.lineStart = ringStart; + clip.lineEnd = ringEnd; + segments = []; + polygon = []; + }, + polygonEnd: function() { + clip.point = point; + clip.lineStart = lineStart; + clip.lineEnd = lineEnd; + segments = d3.merge(segments); + var clipStartInside = d3_geo_pointInPolygon(rotatedClipStart, polygon); + if (segments.length) { + if (!polygonStarted) listener.polygonStart(), polygonStarted = true; + d3_geo_clipPolygon(segments, d3_geo_clipSort, clipStartInside, interpolate, listener); + } else if (clipStartInside) { + if (!polygonStarted) listener.polygonStart(), polygonStarted = true; + listener.lineStart(); + interpolate(null, null, 1, listener); + listener.lineEnd(); + } + if (polygonStarted) listener.polygonEnd(), polygonStarted = false; + segments = polygon = null; + }, + sphere: function() { + listener.polygonStart(); + listener.lineStart(); + interpolate(null, null, 1, listener); + listener.lineEnd(); + listener.polygonEnd(); + } + }; + function point(λ, φ) { + var point = rotate(λ, φ); + if (pointVisible(λ = point[0], φ = point[1])) listener.point(λ, φ); + } + function pointLine(λ, φ) { + var point = rotate(λ, φ); + line.point(point[0], point[1]); + } + function lineStart() { + clip.point = pointLine; + line.lineStart(); + } + function lineEnd() { + clip.point = point; + line.lineEnd(); + } + var segments; + var buffer = d3_geo_clipBufferListener(), ringListener = clipLine(buffer), polygonStarted = false, polygon, ring; + function pointRing(λ, φ) { + ring.push([ λ, φ ]); + var point = rotate(λ, φ); + ringListener.point(point[0], point[1]); + } + function ringStart() { + ringListener.lineStart(); + ring = []; + } + function ringEnd() { + pointRing(ring[0][0], ring[0][1]); + ringListener.lineEnd(); + var clean = ringListener.clean(), ringSegments = buffer.buffer(), segment, n = ringSegments.length; + ring.pop(); + polygon.push(ring); + ring = null; + if (!n) return; + if (clean & 1) { + segment = ringSegments[0]; + var n = segment.length - 1, i = -1, point; + if (n > 0) { + if (!polygonStarted) listener.polygonStart(), polygonStarted = true; + listener.lineStart(); + while (++i < n) listener.point((point = segment[i])[0], point[1]); + listener.lineEnd(); + } + return; + } + if (n > 1 && clean & 2) ringSegments.push(ringSegments.pop().concat(ringSegments.shift())); + segments.push(ringSegments.filter(d3_geo_clipSegmentLength1)); + } + return clip; + }; + } + function d3_geo_clipSegmentLength1(segment) { + return segment.length > 1; + } + function d3_geo_clipBufferListener() { + var lines = [], line; + return { + lineStart: function() { + lines.push(line = []); + }, + point: function(λ, φ) { + line.push([ λ, φ ]); + }, + lineEnd: d3_noop, + buffer: function() { + var buffer = lines; + lines = []; + line = null; + return buffer; + }, + rejoin: function() { + if (lines.length > 1) lines.push(lines.pop().concat(lines.shift())); + } + }; + } + function d3_geo_clipSort(a, b) { + return ((a = a.x)[0] < 0 ? a[1] - halfπ - ε : halfπ - a[1]) - ((b = b.x)[0] < 0 ? b[1] - halfπ - ε : halfπ - b[1]); + } + function d3_geo_pointInPolygon(point, polygon) { + var meridian = point[0], parallel = point[1], meridianNormal = [ Math.sin(meridian), -Math.cos(meridian), 0 ], polarAngle = 0, winding = 0; + d3_geo_areaRingSum.reset(); + for (var i = 0, n = polygon.length; i < n; ++i) { + var ring = polygon[i], m = ring.length; + if (!m) continue; + var point0 = ring[0], λ0 = point0[0], φ0 = point0[1] / 2 + π / 4, sinφ0 = Math.sin(φ0), cosφ0 = Math.cos(φ0), j = 1; + while (true) { + if (j === m) j = 0; + point = ring[j]; + var λ = point[0], φ = point[1] / 2 + π / 4, sinφ = Math.sin(φ), cosφ = Math.cos(φ), dλ = λ - λ0, sdλ = dλ >= 0 ? 1 : -1, adλ = sdλ * dλ, antimeridian = adλ > π, k = sinφ0 * sinφ; + d3_geo_areaRingSum.add(Math.atan2(k * sdλ * Math.sin(adλ), cosφ0 * cosφ + k * Math.cos(adλ))); + polarAngle += antimeridian ? dλ + sdλ * τ : dλ; + if (antimeridian ^ λ0 >= meridian ^ λ >= meridian) { + var arc = d3_geo_cartesianCross(d3_geo_cartesian(point0), d3_geo_cartesian(point)); + d3_geo_cartesianNormalize(arc); + var intersection = d3_geo_cartesianCross(meridianNormal, arc); + d3_geo_cartesianNormalize(intersection); + var φarc = (antimeridian ^ dλ >= 0 ? -1 : 1) * d3_asin(intersection[2]); + if (parallel > φarc || parallel === φarc && (arc[0] || arc[1])) { + winding += antimeridian ^ dλ >= 0 ? 1 : -1; + } + } + if (!j++) break; + λ0 = λ, sinφ0 = sinφ, cosφ0 = cosφ, point0 = point; + } + } + return (polarAngle < -ε || polarAngle < ε && d3_geo_areaRingSum < 0) ^ winding & 1; + } + var d3_geo_clipAntimeridian = d3_geo_clip(d3_true, d3_geo_clipAntimeridianLine, d3_geo_clipAntimeridianInterpolate, [ -π, -π / 2 ]); + function d3_geo_clipAntimeridianLine(listener) { + var λ0 = NaN, φ0 = NaN, sλ0 = NaN, clean; + return { + lineStart: function() { + listener.lineStart(); + clean = 1; + }, + point: function(λ1, φ1) { + var sλ1 = λ1 > 0 ? π : -π, dλ = abs(λ1 - λ0); + if (abs(dλ - π) < ε) { + listener.point(λ0, φ0 = (φ0 + φ1) / 2 > 0 ? halfπ : -halfπ); + listener.point(sλ0, φ0); + listener.lineEnd(); + listener.lineStart(); + listener.point(sλ1, φ0); + listener.point(λ1, φ0); + clean = 0; + } else if (sλ0 !== sλ1 && dλ >= π) { + if (abs(λ0 - sλ0) < ε) λ0 -= sλ0 * ε; + if (abs(λ1 - sλ1) < ε) λ1 -= sλ1 * ε; + φ0 = d3_geo_clipAntimeridianIntersect(λ0, φ0, λ1, φ1); + listener.point(sλ0, φ0); + listener.lineEnd(); + listener.lineStart(); + listener.point(sλ1, φ0); + clean = 0; + } + listener.point(λ0 = λ1, φ0 = φ1); + sλ0 = sλ1; + }, + lineEnd: function() { + listener.lineEnd(); + λ0 = φ0 = NaN; + }, + clean: function() { + return 2 - clean; + } + }; + } + function d3_geo_clipAntimeridianIntersect(λ0, φ0, λ1, φ1) { + var cosφ0, cosφ1, sinλ0_λ1 = Math.sin(λ0 - λ1); + return abs(sinλ0_λ1) > ε ? Math.atan((Math.sin(φ0) * (cosφ1 = Math.cos(φ1)) * Math.sin(λ1) - Math.sin(φ1) * (cosφ0 = Math.cos(φ0)) * Math.sin(λ0)) / (cosφ0 * cosφ1 * sinλ0_λ1)) : (φ0 + φ1) / 2; + } + function d3_geo_clipAntimeridianInterpolate(from, to, direction, listener) { + var φ; + if (from == null) { + φ = direction * halfπ; + listener.point(-π, φ); + listener.point(0, φ); + listener.point(π, φ); + listener.point(π, 0); + listener.point(π, -φ); + listener.point(0, -φ); + listener.point(-π, -φ); + listener.point(-π, 0); + listener.point(-π, φ); + } else if (abs(from[0] - to[0]) > ε) { + var s = from[0] < to[0] ? π : -π; + φ = direction * s / 2; + listener.point(-s, φ); + listener.point(0, φ); + listener.point(s, φ); + } else { + listener.point(to[0], to[1]); + } + } + function d3_geo_clipCircle(radius) { + var cr = Math.cos(radius), smallRadius = cr > 0, notHemisphere = abs(cr) > ε, interpolate = d3_geo_circleInterpolate(radius, 6 * d3_radians); + return d3_geo_clip(visible, clipLine, interpolate, smallRadius ? [ 0, -radius ] : [ -π, radius - π ]); + function visible(λ, φ) { + return Math.cos(λ) * Math.cos(φ) > cr; + } + function clipLine(listener) { + var point0, c0, v0, v00, clean; + return { + lineStart: function() { + v00 = v0 = false; + clean = 1; + }, + point: function(λ, φ) { + var point1 = [ λ, φ ], point2, v = visible(λ, φ), c = smallRadius ? v ? 0 : code(λ, φ) : v ? code(λ + (λ < 0 ? π : -π), φ) : 0; + if (!point0 && (v00 = v0 = v)) listener.lineStart(); + if (v !== v0) { + point2 = intersect(point0, point1); + if (d3_geo_sphericalEqual(point0, point2) || d3_geo_sphericalEqual(point1, point2)) { + point1[0] += ε; + point1[1] += ε; + v = visible(point1[0], point1[1]); + } + } + if (v !== v0) { + clean = 0; + if (v) { + listener.lineStart(); + point2 = intersect(point1, point0); + listener.point(point2[0], point2[1]); + } else { + point2 = intersect(point0, point1); + listener.point(point2[0], point2[1]); + listener.lineEnd(); + } + point0 = point2; + } else if (notHemisphere && point0 && smallRadius ^ v) { + var t; + if (!(c & c0) && (t = intersect(point1, point0, true))) { + clean = 0; + if (smallRadius) { + listener.lineStart(); + listener.point(t[0][0], t[0][1]); + listener.point(t[1][0], t[1][1]); + listener.lineEnd(); + } else { + listener.point(t[1][0], t[1][1]); + listener.lineEnd(); + listener.lineStart(); + listener.point(t[0][0], t[0][1]); + } + } + } + if (v && (!point0 || !d3_geo_sphericalEqual(point0, point1))) { + listener.point(point1[0], point1[1]); + } + point0 = point1, v0 = v, c0 = c; + }, + lineEnd: function() { + if (v0) listener.lineEnd(); + point0 = null; + }, + clean: function() { + return clean | (v00 && v0) << 1; + } + }; + } + function intersect(a, b, two) { + var pa = d3_geo_cartesian(a), pb = d3_geo_cartesian(b); + var n1 = [ 1, 0, 0 ], n2 = d3_geo_cartesianCross(pa, pb), n2n2 = d3_geo_cartesianDot(n2, n2), n1n2 = n2[0], determinant = n2n2 - n1n2 * n1n2; + if (!determinant) return !two && a; + var c1 = cr * n2n2 / determinant, c2 = -cr * n1n2 / determinant, n1xn2 = d3_geo_cartesianCross(n1, n2), A = d3_geo_cartesianScale(n1, c1), B = d3_geo_cartesianScale(n2, c2); + d3_geo_cartesianAdd(A, B); + var u = n1xn2, w = d3_geo_cartesianDot(A, u), uu = d3_geo_cartesianDot(u, u), t2 = w * w - uu * (d3_geo_cartesianDot(A, A) - 1); + if (t2 < 0) return; + var t = Math.sqrt(t2), q = d3_geo_cartesianScale(u, (-w - t) / uu); + d3_geo_cartesianAdd(q, A); + q = d3_geo_spherical(q); + if (!two) return q; + var λ0 = a[0], λ1 = b[0], φ0 = a[1], φ1 = b[1], z; + if (λ1 < λ0) z = λ0, λ0 = λ1, λ1 = z; + var δλ = λ1 - λ0, polar = abs(δλ - π) < ε, meridian = polar || δλ < ε; + if (!polar && φ1 < φ0) z = φ0, φ0 = φ1, φ1 = z; + if (meridian ? polar ? φ0 + φ1 > 0 ^ q[1] < (abs(q[0] - λ0) < ε ? φ0 : φ1) : φ0 <= q[1] && q[1] <= φ1 : δλ > π ^ (λ0 <= q[0] && q[0] <= λ1)) { + var q1 = d3_geo_cartesianScale(u, (-w + t) / uu); + d3_geo_cartesianAdd(q1, A); + return [ q, d3_geo_spherical(q1) ]; + } + } + function code(λ, φ) { + var r = smallRadius ? radius : π - radius, code = 0; + if (λ < -r) code |= 1; else if (λ > r) code |= 2; + if (φ < -r) code |= 4; else if (φ > r) code |= 8; + return code; + } + } + function d3_geom_clipLine(x0, y0, x1, y1) { + return function(line) { + var a = line.a, b = line.b, ax = a.x, ay = a.y, bx = b.x, by = b.y, t0 = 0, t1 = 1, dx = bx - ax, dy = by - ay, r; + r = x0 - ax; + if (!dx && r > 0) return; + r /= dx; + if (dx < 0) { + if (r < t0) return; + if (r < t1) t1 = r; + } else if (dx > 0) { + if (r > t1) return; + if (r > t0) t0 = r; + } + r = x1 - ax; + if (!dx && r < 0) return; + r /= dx; + if (dx < 0) { + if (r > t1) return; + if (r > t0) t0 = r; + } else if (dx > 0) { + if (r < t0) return; + if (r < t1) t1 = r; + } + r = y0 - ay; + if (!dy && r > 0) return; + r /= dy; + if (dy < 0) { + if (r < t0) return; + if (r < t1) t1 = r; + } else if (dy > 0) { + if (r > t1) return; + if (r > t0) t0 = r; + } + r = y1 - ay; + if (!dy && r < 0) return; + r /= dy; + if (dy < 0) { + if (r > t1) return; + if (r > t0) t0 = r; + } else if (dy > 0) { + if (r < t0) return; + if (r < t1) t1 = r; + } + if (t0 > 0) line.a = { + x: ax + t0 * dx, + y: ay + t0 * dy + }; + if (t1 < 1) line.b = { + x: ax + t1 * dx, + y: ay + t1 * dy + }; + return line; + }; + } + var d3_geo_clipExtentMAX = 1e9; + d3.geo.clipExtent = function() { + var x0, y0, x1, y1, stream, clip, clipExtent = { + stream: function(output) { + if (stream) stream.valid = false; + stream = clip(output); + stream.valid = true; + return stream; + }, + extent: function(_) { + if (!arguments.length) return [ [ x0, y0 ], [ x1, y1 ] ]; + clip = d3_geo_clipExtent(x0 = +_[0][0], y0 = +_[0][1], x1 = +_[1][0], y1 = +_[1][1]); + if (stream) stream.valid = false, stream = null; + return clipExtent; + } + }; + return clipExtent.extent([ [ 0, 0 ], [ 960, 500 ] ]); + }; + function d3_geo_clipExtent(x0, y0, x1, y1) { + return function(listener) { + var listener_ = listener, bufferListener = d3_geo_clipBufferListener(), clipLine = d3_geom_clipLine(x0, y0, x1, y1), segments, polygon, ring; + var clip = { + point: point, + lineStart: lineStart, + lineEnd: lineEnd, + polygonStart: function() { + listener = bufferListener; + segments = []; + polygon = []; + clean = true; + }, + polygonEnd: function() { + listener = listener_; + segments = d3.merge(segments); + var clipStartInside = insidePolygon([ x0, y1 ]), inside = clean && clipStartInside, visible = segments.length; + if (inside || visible) { + listener.polygonStart(); + if (inside) { + listener.lineStart(); + interpolate(null, null, 1, listener); + listener.lineEnd(); + } + if (visible) { + d3_geo_clipPolygon(segments, compare, clipStartInside, interpolate, listener); + } + listener.polygonEnd(); + } + segments = polygon = ring = null; + } + }; + function insidePolygon(p) { + var wn = 0, n = polygon.length, y = p[1]; + for (var i = 0; i < n; ++i) { + for (var j = 1, v = polygon[i], m = v.length, a = v[0], b; j < m; ++j) { + b = v[j]; + if (a[1] <= y) { + if (b[1] > y && d3_cross2d(a, b, p) > 0) ++wn; + } else { + if (b[1] <= y && d3_cross2d(a, b, p) < 0) --wn; + } + a = b; + } + } + return wn !== 0; + } + function interpolate(from, to, direction, listener) { + var a = 0, a1 = 0; + if (from == null || (a = corner(from, direction)) !== (a1 = corner(to, direction)) || comparePoints(from, to) < 0 ^ direction > 0) { + do { + listener.point(a === 0 || a === 3 ? x0 : x1, a > 1 ? y1 : y0); + } while ((a = (a + direction + 4) % 4) !== a1); + } else { + listener.point(to[0], to[1]); + } + } + function pointVisible(x, y) { + return x0 <= x && x <= x1 && y0 <= y && y <= y1; + } + function point(x, y) { + if (pointVisible(x, y)) listener.point(x, y); + } + var x__, y__, v__, x_, y_, v_, first, clean; + function lineStart() { + clip.point = linePoint; + if (polygon) polygon.push(ring = []); + first = true; + v_ = false; + x_ = y_ = NaN; + } + function lineEnd() { + if (segments) { + linePoint(x__, y__); + if (v__ && v_) bufferListener.rejoin(); + segments.push(bufferListener.buffer()); + } + clip.point = point; + if (v_) listener.lineEnd(); + } + function linePoint(x, y) { + x = Math.max(-d3_geo_clipExtentMAX, Math.min(d3_geo_clipExtentMAX, x)); + y = Math.max(-d3_geo_clipExtentMAX, Math.min(d3_geo_clipExtentMAX, y)); + var v = pointVisible(x, y); + if (polygon) ring.push([ x, y ]); + if (first) { + x__ = x, y__ = y, v__ = v; + first = false; + if (v) { + listener.lineStart(); + listener.point(x, y); + } + } else { + if (v && v_) listener.point(x, y); else { + var l = { + a: { + x: x_, + y: y_ + }, + b: { + x: x, + y: y + } + }; + if (clipLine(l)) { + if (!v_) { + listener.lineStart(); + listener.point(l.a.x, l.a.y); + } + listener.point(l.b.x, l.b.y); + if (!v) listener.lineEnd(); + clean = false; + } else if (v) { + listener.lineStart(); + listener.point(x, y); + clean = false; + } + } + } + x_ = x, y_ = y, v_ = v; + } + return clip; + }; + function corner(p, direction) { + return abs(p[0] - x0) < ε ? direction > 0 ? 0 : 3 : abs(p[0] - x1) < ε ? direction > 0 ? 2 : 1 : abs(p[1] - y0) < ε ? direction > 0 ? 1 : 0 : direction > 0 ? 3 : 2; + } + function compare(a, b) { + return comparePoints(a.x, b.x); + } + function comparePoints(a, b) { + var ca = corner(a, 1), cb = corner(b, 1); + return ca !== cb ? ca - cb : ca === 0 ? b[1] - a[1] : ca === 1 ? a[0] - b[0] : ca === 2 ? a[1] - b[1] : b[0] - a[0]; + } + } + function d3_geo_compose(a, b) { + function compose(x, y) { + return x = a(x, y), b(x[0], x[1]); + } + if (a.invert && b.invert) compose.invert = function(x, y) { + return x = b.invert(x, y), x && a.invert(x[0], x[1]); + }; + return compose; + } + function d3_geo_conic(projectAt) { + var φ0 = 0, φ1 = π / 3, m = d3_geo_projectionMutator(projectAt), p = m(φ0, φ1); + p.parallels = function(_) { + if (!arguments.length) return [ φ0 / π * 180, φ1 / π * 180 ]; + return m(φ0 = _[0] * π / 180, φ1 = _[1] * π / 180); + }; + return p; + } + function d3_geo_conicEqualArea(φ0, φ1) { + var sinφ0 = Math.sin(φ0), n = (sinφ0 + Math.sin(φ1)) / 2, C = 1 + sinφ0 * (2 * n - sinφ0), ρ0 = Math.sqrt(C) / n; + function forward(λ, φ) { + var ρ = Math.sqrt(C - 2 * n * Math.sin(φ)) / n; + return [ ρ * Math.sin(λ *= n), ρ0 - ρ * Math.cos(λ) ]; + } + forward.invert = function(x, y) { + var ρ0_y = ρ0 - y; + return [ Math.atan2(x, ρ0_y) / n, d3_asin((C - (x * x + ρ0_y * ρ0_y) * n * n) / (2 * n)) ]; + }; + return forward; + } + (d3.geo.conicEqualArea = function() { + return d3_geo_conic(d3_geo_conicEqualArea); + }).raw = d3_geo_conicEqualArea; + d3.geo.albers = function() { + return d3.geo.conicEqualArea().rotate([ 96, 0 ]).center([ -.6, 38.7 ]).parallels([ 29.5, 45.5 ]).scale(1070); + }; + d3.geo.albersUsa = function() { + var lower48 = d3.geo.albers(); + var alaska = d3.geo.conicEqualArea().rotate([ 154, 0 ]).center([ -2, 58.5 ]).parallels([ 55, 65 ]); + var hawaii = d3.geo.conicEqualArea().rotate([ 157, 0 ]).center([ -3, 19.9 ]).parallels([ 8, 18 ]); + var point, pointStream = { + point: function(x, y) { + point = [ x, y ]; + } + }, lower48Point, alaskaPoint, hawaiiPoint; + function albersUsa(coordinates) { + var x = coordinates[0], y = coordinates[1]; + point = null; + (lower48Point(x, y), point) || (alaskaPoint(x, y), point) || hawaiiPoint(x, y); + return point; + } + albersUsa.invert = function(coordinates) { + var k = lower48.scale(), t = lower48.translate(), x = (coordinates[0] - t[0]) / k, y = (coordinates[1] - t[1]) / k; + return (y >= .12 && y < .234 && x >= -.425 && x < -.214 ? alaska : y >= .166 && y < .234 && x >= -.214 && x < -.115 ? hawaii : lower48).invert(coordinates); + }; + albersUsa.stream = function(stream) { + var lower48Stream = lower48.stream(stream), alaskaStream = alaska.stream(stream), hawaiiStream = hawaii.stream(stream); + return { + point: function(x, y) { + lower48Stream.point(x, y); + alaskaStream.point(x, y); + hawaiiStream.point(x, y); + }, + sphere: function() { + lower48Stream.sphere(); + alaskaStream.sphere(); + hawaiiStream.sphere(); + }, + lineStart: function() { + lower48Stream.lineStart(); + alaskaStream.lineStart(); + hawaiiStream.lineStart(); + }, + lineEnd: function() { + lower48Stream.lineEnd(); + alaskaStream.lineEnd(); + hawaiiStream.lineEnd(); + }, + polygonStart: function() { + lower48Stream.polygonStart(); + alaskaStream.polygonStart(); + hawaiiStream.polygonStart(); + }, + polygonEnd: function() { + lower48Stream.polygonEnd(); + alaskaStream.polygonEnd(); + hawaiiStream.polygonEnd(); + } + }; + }; + albersUsa.precision = function(_) { + if (!arguments.length) return lower48.precision(); + lower48.precision(_); + alaska.precision(_); + hawaii.precision(_); + return albersUsa; + }; + albersUsa.scale = function(_) { + if (!arguments.length) return lower48.scale(); + lower48.scale(_); + alaska.scale(_ * .35); + hawaii.scale(_); + return albersUsa.translate(lower48.translate()); + }; + albersUsa.translate = function(_) { + if (!arguments.length) return lower48.translate(); + var k = lower48.scale(), x = +_[0], y = +_[1]; + lower48Point = lower48.translate(_).clipExtent([ [ x - .455 * k, y - .238 * k ], [ x + .455 * k, y + .238 * k ] ]).stream(pointStream).point; + alaskaPoint = alaska.translate([ x - .307 * k, y + .201 * k ]).clipExtent([ [ x - .425 * k + ε, y + .12 * k + ε ], [ x - .214 * k - ε, y + .234 * k - ε ] ]).stream(pointStream).point; + hawaiiPoint = hawaii.translate([ x - .205 * k, y + .212 * k ]).clipExtent([ [ x - .214 * k + ε, y + .166 * k + ε ], [ x - .115 * k - ε, y + .234 * k - ε ] ]).stream(pointStream).point; + return albersUsa; + }; + return albersUsa.scale(1070); + }; + var d3_geo_pathAreaSum, d3_geo_pathAreaPolygon, d3_geo_pathArea = { + point: d3_noop, + lineStart: d3_noop, + lineEnd: d3_noop, + polygonStart: function() { + d3_geo_pathAreaPolygon = 0; + d3_geo_pathArea.lineStart = d3_geo_pathAreaRingStart; + }, + polygonEnd: function() { + d3_geo_pathArea.lineStart = d3_geo_pathArea.lineEnd = d3_geo_pathArea.point = d3_noop; + d3_geo_pathAreaSum += abs(d3_geo_pathAreaPolygon / 2); + } + }; + function d3_geo_pathAreaRingStart() { + var x00, y00, x0, y0; + d3_geo_pathArea.point = function(x, y) { + d3_geo_pathArea.point = nextPoint; + x00 = x0 = x, y00 = y0 = y; + }; + function nextPoint(x, y) { + d3_geo_pathAreaPolygon += y0 * x - x0 * y; + x0 = x, y0 = y; + } + d3_geo_pathArea.lineEnd = function() { + nextPoint(x00, y00); + }; + } + var d3_geo_pathBoundsX0, d3_geo_pathBoundsY0, d3_geo_pathBoundsX1, d3_geo_pathBoundsY1; + var d3_geo_pathBounds = { + point: d3_geo_pathBoundsPoint, + lineStart: d3_noop, + lineEnd: d3_noop, + polygonStart: d3_noop, + polygonEnd: d3_noop + }; + function d3_geo_pathBoundsPoint(x, y) { + if (x < d3_geo_pathBoundsX0) d3_geo_pathBoundsX0 = x; + if (x > d3_geo_pathBoundsX1) d3_geo_pathBoundsX1 = x; + if (y < d3_geo_pathBoundsY0) d3_geo_pathBoundsY0 = y; + if (y > d3_geo_pathBoundsY1) d3_geo_pathBoundsY1 = y; + } + function d3_geo_pathBuffer() { + var pointCircle = d3_geo_pathBufferCircle(4.5), buffer = []; + var stream = { + point: point, + lineStart: function() { + stream.point = pointLineStart; + }, + lineEnd: lineEnd, + polygonStart: function() { + stream.lineEnd = lineEndPolygon; + }, + polygonEnd: function() { + stream.lineEnd = lineEnd; + stream.point = point; + }, + pointRadius: function(_) { + pointCircle = d3_geo_pathBufferCircle(_); + return stream; + }, + result: function() { + if (buffer.length) { + var result = buffer.join(""); + buffer = []; + return result; + } + } + }; + function point(x, y) { + buffer.push("M", x, ",", y, pointCircle); + } + function pointLineStart(x, y) { + buffer.push("M", x, ",", y); + stream.point = pointLine; + } + function pointLine(x, y) { + buffer.push("L", x, ",", y); + } + function lineEnd() { + stream.point = point; + } + function lineEndPolygon() { + buffer.push("Z"); + } + return stream; + } + function d3_geo_pathBufferCircle(radius) { + return "m0," + radius + "a" + radius + "," + radius + " 0 1,1 0," + -2 * radius + "a" + radius + "," + radius + " 0 1,1 0," + 2 * radius + "z"; + } + var d3_geo_pathCentroid = { + point: d3_geo_pathCentroidPoint, + lineStart: d3_geo_pathCentroidLineStart, + lineEnd: d3_geo_pathCentroidLineEnd, + polygonStart: function() { + d3_geo_pathCentroid.lineStart = d3_geo_pathCentroidRingStart; + }, + polygonEnd: function() { + d3_geo_pathCentroid.point = d3_geo_pathCentroidPoint; + d3_geo_pathCentroid.lineStart = d3_geo_pathCentroidLineStart; + d3_geo_pathCentroid.lineEnd = d3_geo_pathCentroidLineEnd; + } + }; + function d3_geo_pathCentroidPoint(x, y) { + d3_geo_centroidX0 += x; + d3_geo_centroidY0 += y; + ++d3_geo_centroidZ0; + } + function d3_geo_pathCentroidLineStart() { + var x0, y0; + d3_geo_pathCentroid.point = function(x, y) { + d3_geo_pathCentroid.point = nextPoint; + d3_geo_pathCentroidPoint(x0 = x, y0 = y); + }; + function nextPoint(x, y) { + var dx = x - x0, dy = y - y0, z = Math.sqrt(dx * dx + dy * dy); + d3_geo_centroidX1 += z * (x0 + x) / 2; + d3_geo_centroidY1 += z * (y0 + y) / 2; + d3_geo_centroidZ1 += z; + d3_geo_pathCentroidPoint(x0 = x, y0 = y); + } + } + function d3_geo_pathCentroidLineEnd() { + d3_geo_pathCentroid.point = d3_geo_pathCentroidPoint; + } + function d3_geo_pathCentroidRingStart() { + var x00, y00, x0, y0; + d3_geo_pathCentroid.point = function(x, y) { + d3_geo_pathCentroid.point = nextPoint; + d3_geo_pathCentroidPoint(x00 = x0 = x, y00 = y0 = y); + }; + function nextPoint(x, y) { + var dx = x - x0, dy = y - y0, z = Math.sqrt(dx * dx + dy * dy); + d3_geo_centroidX1 += z * (x0 + x) / 2; + d3_geo_centroidY1 += z * (y0 + y) / 2; + d3_geo_centroidZ1 += z; + z = y0 * x - x0 * y; + d3_geo_centroidX2 += z * (x0 + x); + d3_geo_centroidY2 += z * (y0 + y); + d3_geo_centroidZ2 += z * 3; + d3_geo_pathCentroidPoint(x0 = x, y0 = y); + } + d3_geo_pathCentroid.lineEnd = function() { + nextPoint(x00, y00); + }; + } + function d3_geo_pathContext(context) { + var pointRadius = 4.5; + var stream = { + point: point, + lineStart: function() { + stream.point = pointLineStart; + }, + lineEnd: lineEnd, + polygonStart: function() { + stream.lineEnd = lineEndPolygon; + }, + polygonEnd: function() { + stream.lineEnd = lineEnd; + stream.point = point; + }, + pointRadius: function(_) { + pointRadius = _; + return stream; + }, + result: d3_noop + }; + function point(x, y) { + context.moveTo(x, y); + context.arc(x, y, pointRadius, 0, τ); + } + function pointLineStart(x, y) { + context.moveTo(x, y); + stream.point = pointLine; + } + function pointLine(x, y) { + context.lineTo(x, y); + } + function lineEnd() { + stream.point = point; + } + function lineEndPolygon() { + context.closePath(); + } + return stream; + } + function d3_geo_resample(project) { + var δ2 = .5, cosMinDistance = Math.cos(30 * d3_radians), maxDepth = 16; + function resample(stream) { + return (maxDepth ? resampleRecursive : resampleNone)(stream); + } + function resampleNone(stream) { + return d3_geo_transformPoint(stream, function(x, y) { + x = project(x, y); + stream.point(x[0], x[1]); + }); + } + function resampleRecursive(stream) { + var λ00, φ00, x00, y00, a00, b00, c00, λ0, x0, y0, a0, b0, c0; + var resample = { + point: point, + lineStart: lineStart, + lineEnd: lineEnd, + polygonStart: function() { + stream.polygonStart(); + resample.lineStart = ringStart; + }, + polygonEnd: function() { + stream.polygonEnd(); + resample.lineStart = lineStart; + } + }; + function point(x, y) { + x = project(x, y); + stream.point(x[0], x[1]); + } + function lineStart() { + x0 = NaN; + resample.point = linePoint; + stream.lineStart(); + } + function linePoint(λ, φ) { + var c = d3_geo_cartesian([ λ, φ ]), p = project(λ, φ); + resampleLineTo(x0, y0, λ0, a0, b0, c0, x0 = p[0], y0 = p[1], λ0 = λ, a0 = c[0], b0 = c[1], c0 = c[2], maxDepth, stream); + stream.point(x0, y0); + } + function lineEnd() { + resample.point = point; + stream.lineEnd(); + } + function ringStart() { + lineStart(); + resample.point = ringPoint; + resample.lineEnd = ringEnd; + } + function ringPoint(λ, φ) { + linePoint(λ00 = λ, φ00 = φ), x00 = x0, y00 = y0, a00 = a0, b00 = b0, c00 = c0; + resample.point = linePoint; + } + function ringEnd() { + resampleLineTo(x0, y0, λ0, a0, b0, c0, x00, y00, λ00, a00, b00, c00, maxDepth, stream); + resample.lineEnd = lineEnd; + lineEnd(); + } + return resample; + } + function resampleLineTo(x0, y0, λ0, a0, b0, c0, x1, y1, λ1, a1, b1, c1, depth, stream) { + var dx = x1 - x0, dy = y1 - y0, d2 = dx * dx + dy * dy; + if (d2 > 4 * δ2 && depth--) { + var a = a0 + a1, b = b0 + b1, c = c0 + c1, m = Math.sqrt(a * a + b * b + c * c), φ2 = Math.asin(c /= m), λ2 = abs(abs(c) - 1) < ε || abs(λ0 - λ1) < ε ? (λ0 + λ1) / 2 : Math.atan2(b, a), p = project(λ2, φ2), x2 = p[0], y2 = p[1], dx2 = x2 - x0, dy2 = y2 - y0, dz = dy * dx2 - dx * dy2; + if (dz * dz / d2 > δ2 || abs((dx * dx2 + dy * dy2) / d2 - .5) > .3 || a0 * a1 + b0 * b1 + c0 * c1 < cosMinDistance) { + resampleLineTo(x0, y0, λ0, a0, b0, c0, x2, y2, λ2, a /= m, b /= m, c, depth, stream); + stream.point(x2, y2); + resampleLineTo(x2, y2, λ2, a, b, c, x1, y1, λ1, a1, b1, c1, depth, stream); + } + } + } + resample.precision = function(_) { + if (!arguments.length) return Math.sqrt(δ2); + maxDepth = (δ2 = _ * _) > 0 && 16; + return resample; + }; + return resample; + } + d3.geo.path = function() { + var pointRadius = 4.5, projection, context, projectStream, contextStream, cacheStream; + function path(object) { + if (object) { + if (typeof pointRadius === "function") contextStream.pointRadius(+pointRadius.apply(this, arguments)); + if (!cacheStream || !cacheStream.valid) cacheStream = projectStream(contextStream); + d3.geo.stream(object, cacheStream); + } + return contextStream.result(); + } + path.area = function(object) { + d3_geo_pathAreaSum = 0; + d3.geo.stream(object, projectStream(d3_geo_pathArea)); + return d3_geo_pathAreaSum; + }; + path.centroid = function(object) { + d3_geo_centroidX0 = d3_geo_centroidY0 = d3_geo_centroidZ0 = d3_geo_centroidX1 = d3_geo_centroidY1 = d3_geo_centroidZ1 = d3_geo_centroidX2 = d3_geo_centroidY2 = d3_geo_centroidZ2 = 0; + d3.geo.stream(object, projectStream(d3_geo_pathCentroid)); + return d3_geo_centroidZ2 ? [ d3_geo_centroidX2 / d3_geo_centroidZ2, d3_geo_centroidY2 / d3_geo_centroidZ2 ] : d3_geo_centroidZ1 ? [ d3_geo_centroidX1 / d3_geo_centroidZ1, d3_geo_centroidY1 / d3_geo_centroidZ1 ] : d3_geo_centroidZ0 ? [ d3_geo_centroidX0 / d3_geo_centroidZ0, d3_geo_centroidY0 / d3_geo_centroidZ0 ] : [ NaN, NaN ]; + }; + path.bounds = function(object) { + d3_geo_pathBoundsX1 = d3_geo_pathBoundsY1 = -(d3_geo_pathBoundsX0 = d3_geo_pathBoundsY0 = Infinity); + d3.geo.stream(object, projectStream(d3_geo_pathBounds)); + return [ [ d3_geo_pathBoundsX0, d3_geo_pathBoundsY0 ], [ d3_geo_pathBoundsX1, d3_geo_pathBoundsY1 ] ]; + }; + path.projection = function(_) { + if (!arguments.length) return projection; + projectStream = (projection = _) ? _.stream || d3_geo_pathProjectStream(_) : d3_identity; + return reset(); + }; + path.context = function(_) { + if (!arguments.length) return context; + contextStream = (context = _) == null ? new d3_geo_pathBuffer() : new d3_geo_pathContext(_); + if (typeof pointRadius !== "function") contextStream.pointRadius(pointRadius); + return reset(); + }; + path.pointRadius = function(_) { + if (!arguments.length) return pointRadius; + pointRadius = typeof _ === "function" ? _ : (contextStream.pointRadius(+_), +_); + return path; + }; + function reset() { + cacheStream = null; + return path; + } + return path.projection(d3.geo.albersUsa()).context(null); + }; + function d3_geo_pathProjectStream(project) { + var resample = d3_geo_resample(function(x, y) { + return project([ x * d3_degrees, y * d3_degrees ]); + }); + return function(stream) { + return d3_geo_projectionRadians(resample(stream)); + }; + } + d3.geo.transform = function(methods) { + return { + stream: function(stream) { + var transform = new d3_geo_transform(stream); + for (var k in methods) transform[k] = methods[k]; + return transform; + } + }; + }; + function d3_geo_transform(stream) { + this.stream = stream; + } + d3_geo_transform.prototype = { + point: function(x, y) { + this.stream.point(x, y); + }, + sphere: function() { + this.stream.sphere(); + }, + lineStart: function() { + this.stream.lineStart(); + }, + lineEnd: function() { + this.stream.lineEnd(); + }, + polygonStart: function() { + this.stream.polygonStart(); + }, + polygonEnd: function() { + this.stream.polygonEnd(); + } + }; + function d3_geo_transformPoint(stream, point) { + return { + point: point, + sphere: function() { + stream.sphere(); + }, + lineStart: function() { + stream.lineStart(); + }, + lineEnd: function() { + stream.lineEnd(); + }, + polygonStart: function() { + stream.polygonStart(); + }, + polygonEnd: function() { + stream.polygonEnd(); + } + }; + } + d3.geo.projection = d3_geo_projection; + d3.geo.projectionMutator = d3_geo_projectionMutator; + function d3_geo_projection(project) { + return d3_geo_projectionMutator(function() { + return project; + })(); + } + function d3_geo_projectionMutator(projectAt) { + var project, rotate, projectRotate, projectResample = d3_geo_resample(function(x, y) { + x = project(x, y); + return [ x[0] * k + δx, δy - x[1] * k ]; + }), k = 150, x = 480, y = 250, λ = 0, φ = 0, δλ = 0, δφ = 0, δγ = 0, δx, δy, preclip = d3_geo_clipAntimeridian, postclip = d3_identity, clipAngle = null, clipExtent = null, stream; + function projection(point) { + point = projectRotate(point[0] * d3_radians, point[1] * d3_radians); + return [ point[0] * k + δx, δy - point[1] * k ]; + } + function invert(point) { + point = projectRotate.invert((point[0] - δx) / k, (δy - point[1]) / k); + return point && [ point[0] * d3_degrees, point[1] * d3_degrees ]; + } + projection.stream = function(output) { + if (stream) stream.valid = false; + stream = d3_geo_projectionRadians(preclip(rotate, projectResample(postclip(output)))); + stream.valid = true; + return stream; + }; + projection.clipAngle = function(_) { + if (!arguments.length) return clipAngle; + preclip = _ == null ? (clipAngle = _, d3_geo_clipAntimeridian) : d3_geo_clipCircle((clipAngle = +_) * d3_radians); + return invalidate(); + }; + projection.clipExtent = function(_) { + if (!arguments.length) return clipExtent; + clipExtent = _; + postclip = _ ? d3_geo_clipExtent(_[0][0], _[0][1], _[1][0], _[1][1]) : d3_identity; + return invalidate(); + }; + projection.scale = function(_) { + if (!arguments.length) return k; + k = +_; + return reset(); + }; + projection.translate = function(_) { + if (!arguments.length) return [ x, y ]; + x = +_[0]; + y = +_[1]; + return reset(); + }; + projection.center = function(_) { + if (!arguments.length) return [ λ * d3_degrees, φ * d3_degrees ]; + λ = _[0] % 360 * d3_radians; + φ = _[1] % 360 * d3_radians; + return reset(); + }; + projection.rotate = function(_) { + if (!arguments.length) return [ δλ * d3_degrees, δφ * d3_degrees, δγ * d3_degrees ]; + δλ = _[0] % 360 * d3_radians; + δφ = _[1] % 360 * d3_radians; + δγ = _.length > 2 ? _[2] % 360 * d3_radians : 0; + return reset(); + }; + d3.rebind(projection, projectResample, "precision"); + function reset() { + projectRotate = d3_geo_compose(rotate = d3_geo_rotation(δλ, δφ, δγ), project); + var center = project(λ, φ); + δx = x - center[0] * k; + δy = y + center[1] * k; + return invalidate(); + } + function invalidate() { + if (stream) stream.valid = false, stream = null; + return projection; + } + return function() { + project = projectAt.apply(this, arguments); + projection.invert = project.invert && invert; + return reset(); + }; + } + function d3_geo_projectionRadians(stream) { + return d3_geo_transformPoint(stream, function(x, y) { + stream.point(x * d3_radians, y * d3_radians); + }); + } + function d3_geo_equirectangular(λ, φ) { + return [ λ, φ ]; + } + (d3.geo.equirectangular = function() { + return d3_geo_projection(d3_geo_equirectangular); + }).raw = d3_geo_equirectangular.invert = d3_geo_equirectangular; + d3.geo.rotation = function(rotate) { + rotate = d3_geo_rotation(rotate[0] % 360 * d3_radians, rotate[1] * d3_radians, rotate.length > 2 ? rotate[2] * d3_radians : 0); + function forward(coordinates) { + coordinates = rotate(coordinates[0] * d3_radians, coordinates[1] * d3_radians); + return coordinates[0] *= d3_degrees, coordinates[1] *= d3_degrees, coordinates; + } + forward.invert = function(coordinates) { + coordinates = rotate.invert(coordinates[0] * d3_radians, coordinates[1] * d3_radians); + return coordinates[0] *= d3_degrees, coordinates[1] *= d3_degrees, coordinates; + }; + return forward; + }; + function d3_geo_identityRotation(λ, φ) { + return [ λ > π ? λ - τ : λ < -π ? λ + τ : λ, φ ]; + } + d3_geo_identityRotation.invert = d3_geo_equirectangular; + function d3_geo_rotation(δλ, δφ, δγ) { + return δλ ? δφ || δγ ? d3_geo_compose(d3_geo_rotationλ(δλ), d3_geo_rotationφγ(δφ, δγ)) : d3_geo_rotationλ(δλ) : δφ || δγ ? d3_geo_rotationφγ(δφ, δγ) : d3_geo_identityRotation; + } + function d3_geo_forwardRotationλ(δλ) { + return function(λ, φ) { + return λ += δλ, [ λ > π ? λ - τ : λ < -π ? λ + τ : λ, φ ]; + }; + } + function d3_geo_rotationλ(δλ) { + var rotation = d3_geo_forwardRotationλ(δλ); + rotation.invert = d3_geo_forwardRotationλ(-δλ); + return rotation; + } + function d3_geo_rotationφγ(δφ, δγ) { + var cosδφ = Math.cos(δφ), sinδφ = Math.sin(δφ), cosδγ = Math.cos(δγ), sinδγ = Math.sin(δγ); + function rotation(λ, φ) { + var cosφ = Math.cos(φ), x = Math.cos(λ) * cosφ, y = Math.sin(λ) * cosφ, z = Math.sin(φ), k = z * cosδφ + x * sinδφ; + return [ Math.atan2(y * cosδγ - k * sinδγ, x * cosδφ - z * sinδφ), d3_asin(k * cosδγ + y * sinδγ) ]; + } + rotation.invert = function(λ, φ) { + var cosφ = Math.cos(φ), x = Math.cos(λ) * cosφ, y = Math.sin(λ) * cosφ, z = Math.sin(φ), k = z * cosδγ - y * sinδγ; + return [ Math.atan2(y * cosδγ + z * sinδγ, x * cosδφ + k * sinδφ), d3_asin(k * cosδφ - x * sinδφ) ]; + }; + return rotation; + } + d3.geo.circle = function() { + var origin = [ 0, 0 ], angle, precision = 6, interpolate; + function circle() { + var center = typeof origin === "function" ? origin.apply(this, arguments) : origin, rotate = d3_geo_rotation(-center[0] * d3_radians, -center[1] * d3_radians, 0).invert, ring = []; + interpolate(null, null, 1, { + point: function(x, y) { + ring.push(x = rotate(x, y)); + x[0] *= d3_degrees, x[1] *= d3_degrees; + } + }); + return { + type: "Polygon", + coordinates: [ ring ] + }; + } + circle.origin = function(x) { + if (!arguments.length) return origin; + origin = x; + return circle; + }; + circle.angle = function(x) { + if (!arguments.length) return angle; + interpolate = d3_geo_circleInterpolate((angle = +x) * d3_radians, precision * d3_radians); + return circle; + }; + circle.precision = function(_) { + if (!arguments.length) return precision; + interpolate = d3_geo_circleInterpolate(angle * d3_radians, (precision = +_) * d3_radians); + return circle; + }; + return circle.angle(90); + }; + function d3_geo_circleInterpolate(radius, precision) { + var cr = Math.cos(radius), sr = Math.sin(radius); + return function(from, to, direction, listener) { + var step = direction * precision; + if (from != null) { + from = d3_geo_circleAngle(cr, from); + to = d3_geo_circleAngle(cr, to); + if (direction > 0 ? from < to : from > to) from += direction * τ; + } else { + from = radius + direction * τ; + to = radius - .5 * step; + } + for (var point, t = from; direction > 0 ? t > to : t < to; t -= step) { + listener.point((point = d3_geo_spherical([ cr, -sr * Math.cos(t), -sr * Math.sin(t) ]))[0], point[1]); + } + }; + } + function d3_geo_circleAngle(cr, point) { + var a = d3_geo_cartesian(point); + a[0] -= cr; + d3_geo_cartesianNormalize(a); + var angle = d3_acos(-a[1]); + return ((-a[2] < 0 ? -angle : angle) + 2 * Math.PI - ε) % (2 * Math.PI); + } + d3.geo.distance = function(a, b) { + var Δλ = (b[0] - a[0]) * d3_radians, φ0 = a[1] * d3_radians, φ1 = b[1] * d3_radians, sinΔλ = Math.sin(Δλ), cosΔλ = Math.cos(Δλ), sinφ0 = Math.sin(φ0), cosφ0 = Math.cos(φ0), sinφ1 = Math.sin(φ1), cosφ1 = Math.cos(φ1), t; + return Math.atan2(Math.sqrt((t = cosφ1 * sinΔλ) * t + (t = cosφ0 * sinφ1 - sinφ0 * cosφ1 * cosΔλ) * t), sinφ0 * sinφ1 + cosφ0 * cosφ1 * cosΔλ); + }; + d3.geo.graticule = function() { + var x1, x0, X1, X0, y1, y0, Y1, Y0, dx = 10, dy = dx, DX = 90, DY = 360, x, y, X, Y, precision = 2.5; + function graticule() { + return { + type: "MultiLineString", + coordinates: lines() + }; + } + function lines() { + return d3.range(Math.ceil(X0 / DX) * DX, X1, DX).map(X).concat(d3.range(Math.ceil(Y0 / DY) * DY, Y1, DY).map(Y)).concat(d3.range(Math.ceil(x0 / dx) * dx, x1, dx).filter(function(x) { + return abs(x % DX) > ε; + }).map(x)).concat(d3.range(Math.ceil(y0 / dy) * dy, y1, dy).filter(function(y) { + return abs(y % DY) > ε; + }).map(y)); + } + graticule.lines = function() { + return lines().map(function(coordinates) { + return { + type: "LineString", + coordinates: coordinates + }; + }); + }; + graticule.outline = function() { + return { + type: "Polygon", + coordinates: [ X(X0).concat(Y(Y1).slice(1), X(X1).reverse().slice(1), Y(Y0).reverse().slice(1)) ] + }; + }; + graticule.extent = function(_) { + if (!arguments.length) return graticule.minorExtent(); + return graticule.majorExtent(_).minorExtent(_); + }; + graticule.majorExtent = function(_) { + if (!arguments.length) return [ [ X0, Y0 ], [ X1, Y1 ] ]; + X0 = +_[0][0], X1 = +_[1][0]; + Y0 = +_[0][1], Y1 = +_[1][1]; + if (X0 > X1) _ = X0, X0 = X1, X1 = _; + if (Y0 > Y1) _ = Y0, Y0 = Y1, Y1 = _; + return graticule.precision(precision); + }; + graticule.minorExtent = function(_) { + if (!arguments.length) return [ [ x0, y0 ], [ x1, y1 ] ]; + x0 = +_[0][0], x1 = +_[1][0]; + y0 = +_[0][1], y1 = +_[1][1]; + if (x0 > x1) _ = x0, x0 = x1, x1 = _; + if (y0 > y1) _ = y0, y0 = y1, y1 = _; + return graticule.precision(precision); + }; + graticule.step = function(_) { + if (!arguments.length) return graticule.minorStep(); + return graticule.majorStep(_).minorStep(_); + }; + graticule.majorStep = function(_) { + if (!arguments.length) return [ DX, DY ]; + DX = +_[0], DY = +_[1]; + return graticule; + }; + graticule.minorStep = function(_) { + if (!arguments.length) return [ dx, dy ]; + dx = +_[0], dy = +_[1]; + return graticule; + }; + graticule.precision = function(_) { + if (!arguments.length) return precision; + precision = +_; + x = d3_geo_graticuleX(y0, y1, 90); + y = d3_geo_graticuleY(x0, x1, precision); + X = d3_geo_graticuleX(Y0, Y1, 90); + Y = d3_geo_graticuleY(X0, X1, precision); + return graticule; + }; + return graticule.majorExtent([ [ -180, -90 + ε ], [ 180, 90 - ε ] ]).minorExtent([ [ -180, -80 - ε ], [ 180, 80 + ε ] ]); + }; + function d3_geo_graticuleX(y0, y1, dy) { + var y = d3.range(y0, y1 - ε, dy).concat(y1); + return function(x) { + return y.map(function(y) { + return [ x, y ]; + }); + }; + } + function d3_geo_graticuleY(x0, x1, dx) { + var x = d3.range(x0, x1 - ε, dx).concat(x1); + return function(y) { + return x.map(function(x) { + return [ x, y ]; + }); + }; + } + function d3_source(d) { + return d.source; + } + function d3_target(d) { + return d.target; + } + d3.geo.greatArc = function() { + var source = d3_source, source_, target = d3_target, target_; + function greatArc() { + return { + type: "LineString", + coordinates: [ source_ || source.apply(this, arguments), target_ || target.apply(this, arguments) ] + }; + } + greatArc.distance = function() { + return d3.geo.distance(source_ || source.apply(this, arguments), target_ || target.apply(this, arguments)); + }; + greatArc.source = function(_) { + if (!arguments.length) return source; + source = _, source_ = typeof _ === "function" ? null : _; + return greatArc; + }; + greatArc.target = function(_) { + if (!arguments.length) return target; + target = _, target_ = typeof _ === "function" ? null : _; + return greatArc; + }; + greatArc.precision = function() { + return arguments.length ? greatArc : 0; + }; + return greatArc; + }; + d3.geo.interpolate = function(source, target) { + return d3_geo_interpolate(source[0] * d3_radians, source[1] * d3_radians, target[0] * d3_radians, target[1] * d3_radians); + }; + function d3_geo_interpolate(x0, y0, x1, y1) { + var cy0 = Math.cos(y0), sy0 = Math.sin(y0), cy1 = Math.cos(y1), sy1 = Math.sin(y1), kx0 = cy0 * Math.cos(x0), ky0 = cy0 * Math.sin(x0), kx1 = cy1 * Math.cos(x1), ky1 = cy1 * Math.sin(x1), d = 2 * Math.asin(Math.sqrt(d3_haversin(y1 - y0) + cy0 * cy1 * d3_haversin(x1 - x0))), k = 1 / Math.sin(d); + var interpolate = d ? function(t) { + var B = Math.sin(t *= d) * k, A = Math.sin(d - t) * k, x = A * kx0 + B * kx1, y = A * ky0 + B * ky1, z = A * sy0 + B * sy1; + return [ Math.atan2(y, x) * d3_degrees, Math.atan2(z, Math.sqrt(x * x + y * y)) * d3_degrees ]; + } : function() { + return [ x0 * d3_degrees, y0 * d3_degrees ]; + }; + interpolate.distance = d; + return interpolate; + } + d3.geo.length = function(object) { + d3_geo_lengthSum = 0; + d3.geo.stream(object, d3_geo_length); + return d3_geo_lengthSum; + }; + var d3_geo_lengthSum; + var d3_geo_length = { + sphere: d3_noop, + point: d3_noop, + lineStart: d3_geo_lengthLineStart, + lineEnd: d3_noop, + polygonStart: d3_noop, + polygonEnd: d3_noop + }; + function d3_geo_lengthLineStart() { + var λ0, sinφ0, cosφ0; + d3_geo_length.point = function(λ, φ) { + λ0 = λ * d3_radians, sinφ0 = Math.sin(φ *= d3_radians), cosφ0 = Math.cos(φ); + d3_geo_length.point = nextPoint; + }; + d3_geo_length.lineEnd = function() { + d3_geo_length.point = d3_geo_length.lineEnd = d3_noop; + }; + function nextPoint(λ, φ) { + var sinφ = Math.sin(φ *= d3_radians), cosφ = Math.cos(φ), t = abs((λ *= d3_radians) - λ0), cosΔλ = Math.cos(t); + d3_geo_lengthSum += Math.atan2(Math.sqrt((t = cosφ * Math.sin(t)) * t + (t = cosφ0 * sinφ - sinφ0 * cosφ * cosΔλ) * t), sinφ0 * sinφ + cosφ0 * cosφ * cosΔλ); + λ0 = λ, sinφ0 = sinφ, cosφ0 = cosφ; + } + } + function d3_geo_azimuthal(scale, angle) { + function azimuthal(λ, φ) { + var cosλ = Math.cos(λ), cosφ = Math.cos(φ), k = scale(cosλ * cosφ); + return [ k * cosφ * Math.sin(λ), k * Math.sin(φ) ]; + } + azimuthal.invert = function(x, y) { + var ρ = Math.sqrt(x * x + y * y), c = angle(ρ), sinc = Math.sin(c), cosc = Math.cos(c); + return [ Math.atan2(x * sinc, ρ * cosc), Math.asin(ρ && y * sinc / ρ) ]; + }; + return azimuthal; + } + var d3_geo_azimuthalEqualArea = d3_geo_azimuthal(function(cosλcosφ) { + return Math.sqrt(2 / (1 + cosλcosφ)); + }, function(ρ) { + return 2 * Math.asin(ρ / 2); + }); + (d3.geo.azimuthalEqualArea = function() { + return d3_geo_projection(d3_geo_azimuthalEqualArea); + }).raw = d3_geo_azimuthalEqualArea; + var d3_geo_azimuthalEquidistant = d3_geo_azimuthal(function(cosλcosφ) { + var c = Math.acos(cosλcosφ); + return c && c / Math.sin(c); + }, d3_identity); + (d3.geo.azimuthalEquidistant = function() { + return d3_geo_projection(d3_geo_azimuthalEquidistant); + }).raw = d3_geo_azimuthalEquidistant; + function d3_geo_conicConformal(φ0, φ1) { + var cosφ0 = Math.cos(φ0), t = function(φ) { + return Math.tan(π / 4 + φ / 2); + }, n = φ0 === φ1 ? Math.sin(φ0) : Math.log(cosφ0 / Math.cos(φ1)) / Math.log(t(φ1) / t(φ0)), F = cosφ0 * Math.pow(t(φ0), n) / n; + if (!n) return d3_geo_mercator; + function forward(λ, φ) { + if (F > 0) { + if (φ < -halfπ + ε) φ = -halfπ + ε; + } else { + if (φ > halfπ - ε) φ = halfπ - ε; + } + var ρ = F / Math.pow(t(φ), n); + return [ ρ * Math.sin(n * λ), F - ρ * Math.cos(n * λ) ]; + } + forward.invert = function(x, y) { + var ρ0_y = F - y, ρ = d3_sgn(n) * Math.sqrt(x * x + ρ0_y * ρ0_y); + return [ Math.atan2(x, ρ0_y) / n, 2 * Math.atan(Math.pow(F / ρ, 1 / n)) - halfπ ]; + }; + return forward; + } + (d3.geo.conicConformal = function() { + return d3_geo_conic(d3_geo_conicConformal); + }).raw = d3_geo_conicConformal; + function d3_geo_conicEquidistant(φ0, φ1) { + var cosφ0 = Math.cos(φ0), n = φ0 === φ1 ? Math.sin(φ0) : (cosφ0 - Math.cos(φ1)) / (φ1 - φ0), G = cosφ0 / n + φ0; + if (abs(n) < ε) return d3_geo_equirectangular; + function forward(λ, φ) { + var ρ = G - φ; + return [ ρ * Math.sin(n * λ), G - ρ * Math.cos(n * λ) ]; + } + forward.invert = function(x, y) { + var ρ0_y = G - y; + return [ Math.atan2(x, ρ0_y) / n, G - d3_sgn(n) * Math.sqrt(x * x + ρ0_y * ρ0_y) ]; + }; + return forward; + } + (d3.geo.conicEquidistant = function() { + return d3_geo_conic(d3_geo_conicEquidistant); + }).raw = d3_geo_conicEquidistant; + var d3_geo_gnomonic = d3_geo_azimuthal(function(cosλcosφ) { + return 1 / cosλcosφ; + }, Math.atan); + (d3.geo.gnomonic = function() { + return d3_geo_projection(d3_geo_gnomonic); + }).raw = d3_geo_gnomonic; + function d3_geo_mercator(λ, φ) { + return [ λ, Math.log(Math.tan(π / 4 + φ / 2)) ]; + } + d3_geo_mercator.invert = function(x, y) { + return [ x, 2 * Math.atan(Math.exp(y)) - halfπ ]; + }; + function d3_geo_mercatorProjection(project) { + var m = d3_geo_projection(project), scale = m.scale, translate = m.translate, clipExtent = m.clipExtent, clipAuto; + m.scale = function() { + var v = scale.apply(m, arguments); + return v === m ? clipAuto ? m.clipExtent(null) : m : v; + }; + m.translate = function() { + var v = translate.apply(m, arguments); + return v === m ? clipAuto ? m.clipExtent(null) : m : v; + }; + m.clipExtent = function(_) { + var v = clipExtent.apply(m, arguments); + if (v === m) { + if (clipAuto = _ == null) { + var k = π * scale(), t = translate(); + clipExtent([ [ t[0] - k, t[1] - k ], [ t[0] + k, t[1] + k ] ]); + } + } else if (clipAuto) { + v = null; + } + return v; + }; + return m.clipExtent(null); + } + (d3.geo.mercator = function() { + return d3_geo_mercatorProjection(d3_geo_mercator); + }).raw = d3_geo_mercator; + var d3_geo_orthographic = d3_geo_azimuthal(function() { + return 1; + }, Math.asin); + (d3.geo.orthographic = function() { + return d3_geo_projection(d3_geo_orthographic); + }).raw = d3_geo_orthographic; + var d3_geo_stereographic = d3_geo_azimuthal(function(cosλcosφ) { + return 1 / (1 + cosλcosφ); + }, function(ρ) { + return 2 * Math.atan(ρ); + }); + (d3.geo.stereographic = function() { + return d3_geo_projection(d3_geo_stereographic); + }).raw = d3_geo_stereographic; + function d3_geo_transverseMercator(λ, φ) { + return [ Math.log(Math.tan(π / 4 + φ / 2)), -λ ]; + } + d3_geo_transverseMercator.invert = function(x, y) { + return [ -y, 2 * Math.atan(Math.exp(x)) - halfπ ]; + }; + (d3.geo.transverseMercator = function() { + var projection = d3_geo_mercatorProjection(d3_geo_transverseMercator), center = projection.center, rotate = projection.rotate; + projection.center = function(_) { + return _ ? center([ -_[1], _[0] ]) : (_ = center(), [ -_[1], _[0] ]); + }; + projection.rotate = function(_) { + return _ ? rotate([ _[0], _[1], _.length > 2 ? _[2] + 90 : 90 ]) : (_ = rotate(), + [ _[0], _[1], _[2] - 90 ]); + }; + return projection.rotate([ 0, 0 ]); + }).raw = d3_geo_transverseMercator; + d3.geom = {}; + function d3_geom_pointX(d) { + return d[0]; + } + function d3_geom_pointY(d) { + return d[1]; + } + d3.geom.hull = function(vertices) { + var x = d3_geom_pointX, y = d3_geom_pointY; + if (arguments.length) return hull(vertices); + function hull(data) { + if (data.length < 3) return []; + var fx = d3_functor(x), fy = d3_functor(y), i, n = data.length, points = [], flippedPoints = []; + for (i = 0; i < n; i++) { + points.push([ +fx.call(this, data[i], i), +fy.call(this, data[i], i), i ]); + } + points.sort(d3_geom_hullOrder); + for (i = 0; i < n; i++) flippedPoints.push([ points[i][0], -points[i][1] ]); + var upper = d3_geom_hullUpper(points), lower = d3_geom_hullUpper(flippedPoints); + var skipLeft = lower[0] === upper[0], skipRight = lower[lower.length - 1] === upper[upper.length - 1], polygon = []; + for (i = upper.length - 1; i >= 0; --i) polygon.push(data[points[upper[i]][2]]); + for (i = +skipLeft; i < lower.length - skipRight; ++i) polygon.push(data[points[lower[i]][2]]); + return polygon; + } + hull.x = function(_) { + return arguments.length ? (x = _, hull) : x; + }; + hull.y = function(_) { + return arguments.length ? (y = _, hull) : y; + }; + return hull; + }; + function d3_geom_hullUpper(points) { + var n = points.length, hull = [ 0, 1 ], hs = 2; + for (var i = 2; i < n; i++) { + while (hs > 1 && d3_cross2d(points[hull[hs - 2]], points[hull[hs - 1]], points[i]) <= 0) --hs; + hull[hs++] = i; + } + return hull.slice(0, hs); + } + function d3_geom_hullOrder(a, b) { + return a[0] - b[0] || a[1] - b[1]; + } + d3.geom.polygon = function(coordinates) { + d3_subclass(coordinates, d3_geom_polygonPrototype); + return coordinates; + }; + var d3_geom_polygonPrototype = d3.geom.polygon.prototype = []; + d3_geom_polygonPrototype.area = function() { + var i = -1, n = this.length, a, b = this[n - 1], area = 0; + while (++i < n) { + a = b; + b = this[i]; + area += a[1] * b[0] - a[0] * b[1]; + } + return area * .5; + }; + d3_geom_polygonPrototype.centroid = function(k) { + var i = -1, n = this.length, x = 0, y = 0, a, b = this[n - 1], c; + if (!arguments.length) k = -1 / (6 * this.area()); + while (++i < n) { + a = b; + b = this[i]; + c = a[0] * b[1] - b[0] * a[1]; + x += (a[0] + b[0]) * c; + y += (a[1] + b[1]) * c; + } + return [ x * k, y * k ]; + }; + d3_geom_polygonPrototype.clip = function(subject) { + var input, closed = d3_geom_polygonClosed(subject), i = -1, n = this.length - d3_geom_polygonClosed(this), j, m, a = this[n - 1], b, c, d; + while (++i < n) { + input = subject.slice(); + subject.length = 0; + b = this[i]; + c = input[(m = input.length - closed) - 1]; + j = -1; + while (++j < m) { + d = input[j]; + if (d3_geom_polygonInside(d, a, b)) { + if (!d3_geom_polygonInside(c, a, b)) { + subject.push(d3_geom_polygonIntersect(c, d, a, b)); + } + subject.push(d); + } else if (d3_geom_polygonInside(c, a, b)) { + subject.push(d3_geom_polygonIntersect(c, d, a, b)); + } + c = d; + } + if (closed) subject.push(subject[0]); + a = b; + } + return subject; + }; + function d3_geom_polygonInside(p, a, b) { + return (b[0] - a[0]) * (p[1] - a[1]) < (b[1] - a[1]) * (p[0] - a[0]); + } + function d3_geom_polygonIntersect(c, d, a, b) { + var x1 = c[0], x3 = a[0], x21 = d[0] - x1, x43 = b[0] - x3, y1 = c[1], y3 = a[1], y21 = d[1] - y1, y43 = b[1] - y3, ua = (x43 * (y1 - y3) - y43 * (x1 - x3)) / (y43 * x21 - x43 * y21); + return [ x1 + ua * x21, y1 + ua * y21 ]; + } + function d3_geom_polygonClosed(coordinates) { + var a = coordinates[0], b = coordinates[coordinates.length - 1]; + return !(a[0] - b[0] || a[1] - b[1]); + } + var d3_geom_voronoiEdges, d3_geom_voronoiCells, d3_geom_voronoiBeaches, d3_geom_voronoiBeachPool = [], d3_geom_voronoiFirstCircle, d3_geom_voronoiCircles, d3_geom_voronoiCirclePool = []; + function d3_geom_voronoiBeach() { + d3_geom_voronoiRedBlackNode(this); + this.edge = this.site = this.circle = null; + } + function d3_geom_voronoiCreateBeach(site) { + var beach = d3_geom_voronoiBeachPool.pop() || new d3_geom_voronoiBeach(); + beach.site = site; + return beach; + } + function d3_geom_voronoiDetachBeach(beach) { + d3_geom_voronoiDetachCircle(beach); + d3_geom_voronoiBeaches.remove(beach); + d3_geom_voronoiBeachPool.push(beach); + d3_geom_voronoiRedBlackNode(beach); + } + function d3_geom_voronoiRemoveBeach(beach) { + var circle = beach.circle, x = circle.x, y = circle.cy, vertex = { + x: x, + y: y + }, previous = beach.P, next = beach.N, disappearing = [ beach ]; + d3_geom_voronoiDetachBeach(beach); + var lArc = previous; + while (lArc.circle && abs(x - lArc.circle.x) < ε && abs(y - lArc.circle.cy) < ε) { + previous = lArc.P; + disappearing.unshift(lArc); + d3_geom_voronoiDetachBeach(lArc); + lArc = previous; + } + disappearing.unshift(lArc); + d3_geom_voronoiDetachCircle(lArc); + var rArc = next; + while (rArc.circle && abs(x - rArc.circle.x) < ε && abs(y - rArc.circle.cy) < ε) { + next = rArc.N; + disappearing.push(rArc); + d3_geom_voronoiDetachBeach(rArc); + rArc = next; + } + disappearing.push(rArc); + d3_geom_voronoiDetachCircle(rArc); + var nArcs = disappearing.length, iArc; + for (iArc = 1; iArc < nArcs; ++iArc) { + rArc = disappearing[iArc]; + lArc = disappearing[iArc - 1]; + d3_geom_voronoiSetEdgeEnd(rArc.edge, lArc.site, rArc.site, vertex); + } + lArc = disappearing[0]; + rArc = disappearing[nArcs - 1]; + rArc.edge = d3_geom_voronoiCreateEdge(lArc.site, rArc.site, null, vertex); + d3_geom_voronoiAttachCircle(lArc); + d3_geom_voronoiAttachCircle(rArc); + } + function d3_geom_voronoiAddBeach(site) { + var x = site.x, directrix = site.y, lArc, rArc, dxl, dxr, node = d3_geom_voronoiBeaches._; + while (node) { + dxl = d3_geom_voronoiLeftBreakPoint(node, directrix) - x; + if (dxl > ε) node = node.L; else { + dxr = x - d3_geom_voronoiRightBreakPoint(node, directrix); + if (dxr > ε) { + if (!node.R) { + lArc = node; + break; + } + node = node.R; + } else { + if (dxl > -ε) { + lArc = node.P; + rArc = node; + } else if (dxr > -ε) { + lArc = node; + rArc = node.N; + } else { + lArc = rArc = node; + } + break; + } + } + } + var newArc = d3_geom_voronoiCreateBeach(site); + d3_geom_voronoiBeaches.insert(lArc, newArc); + if (!lArc && !rArc) return; + if (lArc === rArc) { + d3_geom_voronoiDetachCircle(lArc); + rArc = d3_geom_voronoiCreateBeach(lArc.site); + d3_geom_voronoiBeaches.insert(newArc, rArc); + newArc.edge = rArc.edge = d3_geom_voronoiCreateEdge(lArc.site, newArc.site); + d3_geom_voronoiAttachCircle(lArc); + d3_geom_voronoiAttachCircle(rArc); + return; + } + if (!rArc) { + newArc.edge = d3_geom_voronoiCreateEdge(lArc.site, newArc.site); + return; + } + d3_geom_voronoiDetachCircle(lArc); + d3_geom_voronoiDetachCircle(rArc); + var lSite = lArc.site, ax = lSite.x, ay = lSite.y, bx = site.x - ax, by = site.y - ay, rSite = rArc.site, cx = rSite.x - ax, cy = rSite.y - ay, d = 2 * (bx * cy - by * cx), hb = bx * bx + by * by, hc = cx * cx + cy * cy, vertex = { + x: (cy * hb - by * hc) / d + ax, + y: (bx * hc - cx * hb) / d + ay + }; + d3_geom_voronoiSetEdgeEnd(rArc.edge, lSite, rSite, vertex); + newArc.edge = d3_geom_voronoiCreateEdge(lSite, site, null, vertex); + rArc.edge = d3_geom_voronoiCreateEdge(site, rSite, null, vertex); + d3_geom_voronoiAttachCircle(lArc); + d3_geom_voronoiAttachCircle(rArc); + } + function d3_geom_voronoiLeftBreakPoint(arc, directrix) { + var site = arc.site, rfocx = site.x, rfocy = site.y, pby2 = rfocy - directrix; + if (!pby2) return rfocx; + var lArc = arc.P; + if (!lArc) return -Infinity; + site = lArc.site; + var lfocx = site.x, lfocy = site.y, plby2 = lfocy - directrix; + if (!plby2) return lfocx; + var hl = lfocx - rfocx, aby2 = 1 / pby2 - 1 / plby2, b = hl / plby2; + if (aby2) return (-b + Math.sqrt(b * b - 2 * aby2 * (hl * hl / (-2 * plby2) - lfocy + plby2 / 2 + rfocy - pby2 / 2))) / aby2 + rfocx; + return (rfocx + lfocx) / 2; + } + function d3_geom_voronoiRightBreakPoint(arc, directrix) { + var rArc = arc.N; + if (rArc) return d3_geom_voronoiLeftBreakPoint(rArc, directrix); + var site = arc.site; + return site.y === directrix ? site.x : Infinity; + } + function d3_geom_voronoiCell(site) { + this.site = site; + this.edges = []; + } + d3_geom_voronoiCell.prototype.prepare = function() { + var halfEdges = this.edges, iHalfEdge = halfEdges.length, edge; + while (iHalfEdge--) { + edge = halfEdges[iHalfEdge].edge; + if (!edge.b || !edge.a) halfEdges.splice(iHalfEdge, 1); + } + halfEdges.sort(d3_geom_voronoiHalfEdgeOrder); + return halfEdges.length; + }; + function d3_geom_voronoiCloseCells(extent) { + var x0 = extent[0][0], x1 = extent[1][0], y0 = extent[0][1], y1 = extent[1][1], x2, y2, x3, y3, cells = d3_geom_voronoiCells, iCell = cells.length, cell, iHalfEdge, halfEdges, nHalfEdges, start, end; + while (iCell--) { + cell = cells[iCell]; + if (!cell || !cell.prepare()) continue; + halfEdges = cell.edges; + nHalfEdges = halfEdges.length; + iHalfEdge = 0; + while (iHalfEdge < nHalfEdges) { + end = halfEdges[iHalfEdge].end(), x3 = end.x, y3 = end.y; + start = halfEdges[++iHalfEdge % nHalfEdges].start(), x2 = start.x, y2 = start.y; + if (abs(x3 - x2) > ε || abs(y3 - y2) > ε) { + halfEdges.splice(iHalfEdge, 0, new d3_geom_voronoiHalfEdge(d3_geom_voronoiCreateBorderEdge(cell.site, end, abs(x3 - x0) < ε && y1 - y3 > ε ? { + x: x0, + y: abs(x2 - x0) < ε ? y2 : y1 + } : abs(y3 - y1) < ε && x1 - x3 > ε ? { + x: abs(y2 - y1) < ε ? x2 : x1, + y: y1 + } : abs(x3 - x1) < ε && y3 - y0 > ε ? { + x: x1, + y: abs(x2 - x1) < ε ? y2 : y0 + } : abs(y3 - y0) < ε && x3 - x0 > ε ? { + x: abs(y2 - y0) < ε ? x2 : x0, + y: y0 + } : null), cell.site, null)); + ++nHalfEdges; + } + } + } + } + function d3_geom_voronoiHalfEdgeOrder(a, b) { + return b.angle - a.angle; + } + function d3_geom_voronoiCircle() { + d3_geom_voronoiRedBlackNode(this); + this.x = this.y = this.arc = this.site = this.cy = null; + } + function d3_geom_voronoiAttachCircle(arc) { + var lArc = arc.P, rArc = arc.N; + if (!lArc || !rArc) return; + var lSite = lArc.site, cSite = arc.site, rSite = rArc.site; + if (lSite === rSite) return; + var bx = cSite.x, by = cSite.y, ax = lSite.x - bx, ay = lSite.y - by, cx = rSite.x - bx, cy = rSite.y - by; + var d = 2 * (ax * cy - ay * cx); + if (d >= -ε2) return; + var ha = ax * ax + ay * ay, hc = cx * cx + cy * cy, x = (cy * ha - ay * hc) / d, y = (ax * hc - cx * ha) / d, cy = y + by; + var circle = d3_geom_voronoiCirclePool.pop() || new d3_geom_voronoiCircle(); + circle.arc = arc; + circle.site = cSite; + circle.x = x + bx; + circle.y = cy + Math.sqrt(x * x + y * y); + circle.cy = cy; + arc.circle = circle; + var before = null, node = d3_geom_voronoiCircles._; + while (node) { + if (circle.y < node.y || circle.y === node.y && circle.x <= node.x) { + if (node.L) node = node.L; else { + before = node.P; + break; + } + } else { + if (node.R) node = node.R; else { + before = node; + break; + } + } + } + d3_geom_voronoiCircles.insert(before, circle); + if (!before) d3_geom_voronoiFirstCircle = circle; + } + function d3_geom_voronoiDetachCircle(arc) { + var circle = arc.circle; + if (circle) { + if (!circle.P) d3_geom_voronoiFirstCircle = circle.N; + d3_geom_voronoiCircles.remove(circle); + d3_geom_voronoiCirclePool.push(circle); + d3_geom_voronoiRedBlackNode(circle); + arc.circle = null; + } + } + function d3_geom_voronoiClipEdges(extent) { + var edges = d3_geom_voronoiEdges, clip = d3_geom_clipLine(extent[0][0], extent[0][1], extent[1][0], extent[1][1]), i = edges.length, e; + while (i--) { + e = edges[i]; + if (!d3_geom_voronoiConnectEdge(e, extent) || !clip(e) || abs(e.a.x - e.b.x) < ε && abs(e.a.y - e.b.y) < ε) { + e.a = e.b = null; + edges.splice(i, 1); + } + } + } + function d3_geom_voronoiConnectEdge(edge, extent) { + var vb = edge.b; + if (vb) return true; + var va = edge.a, x0 = extent[0][0], x1 = extent[1][0], y0 = extent[0][1], y1 = extent[1][1], lSite = edge.l, rSite = edge.r, lx = lSite.x, ly = lSite.y, rx = rSite.x, ry = rSite.y, fx = (lx + rx) / 2, fy = (ly + ry) / 2, fm, fb; + if (ry === ly) { + if (fx < x0 || fx >= x1) return; + if (lx > rx) { + if (!va) va = { + x: fx, + y: y0 + }; else if (va.y >= y1) return; + vb = { + x: fx, + y: y1 + }; + } else { + if (!va) va = { + x: fx, + y: y1 + }; else if (va.y < y0) return; + vb = { + x: fx, + y: y0 + }; + } + } else { + fm = (lx - rx) / (ry - ly); + fb = fy - fm * fx; + if (fm < -1 || fm > 1) { + if (lx > rx) { + if (!va) va = { + x: (y0 - fb) / fm, + y: y0 + }; else if (va.y >= y1) return; + vb = { + x: (y1 - fb) / fm, + y: y1 + }; + } else { + if (!va) va = { + x: (y1 - fb) / fm, + y: y1 + }; else if (va.y < y0) return; + vb = { + x: (y0 - fb) / fm, + y: y0 + }; + } + } else { + if (ly < ry) { + if (!va) va = { + x: x0, + y: fm * x0 + fb + }; else if (va.x >= x1) return; + vb = { + x: x1, + y: fm * x1 + fb + }; + } else { + if (!va) va = { + x: x1, + y: fm * x1 + fb + }; else if (va.x < x0) return; + vb = { + x: x0, + y: fm * x0 + fb + }; + } + } + } + edge.a = va; + edge.b = vb; + return true; + } + function d3_geom_voronoiEdge(lSite, rSite) { + this.l = lSite; + this.r = rSite; + this.a = this.b = null; + } + function d3_geom_voronoiCreateEdge(lSite, rSite, va, vb) { + var edge = new d3_geom_voronoiEdge(lSite, rSite); + d3_geom_voronoiEdges.push(edge); + if (va) d3_geom_voronoiSetEdgeEnd(edge, lSite, rSite, va); + if (vb) d3_geom_voronoiSetEdgeEnd(edge, rSite, lSite, vb); + d3_geom_voronoiCells[lSite.i].edges.push(new d3_geom_voronoiHalfEdge(edge, lSite, rSite)); + d3_geom_voronoiCells[rSite.i].edges.push(new d3_geom_voronoiHalfEdge(edge, rSite, lSite)); + return edge; + } + function d3_geom_voronoiCreateBorderEdge(lSite, va, vb) { + var edge = new d3_geom_voronoiEdge(lSite, null); + edge.a = va; + edge.b = vb; + d3_geom_voronoiEdges.push(edge); + return edge; + } + function d3_geom_voronoiSetEdgeEnd(edge, lSite, rSite, vertex) { + if (!edge.a && !edge.b) { + edge.a = vertex; + edge.l = lSite; + edge.r = rSite; + } else if (edge.l === rSite) { + edge.b = vertex; + } else { + edge.a = vertex; + } + } + function d3_geom_voronoiHalfEdge(edge, lSite, rSite) { + var va = edge.a, vb = edge.b; + this.edge = edge; + this.site = lSite; + this.angle = rSite ? Math.atan2(rSite.y - lSite.y, rSite.x - lSite.x) : edge.l === lSite ? Math.atan2(vb.x - va.x, va.y - vb.y) : Math.atan2(va.x - vb.x, vb.y - va.y); + } + d3_geom_voronoiHalfEdge.prototype = { + start: function() { + return this.edge.l === this.site ? this.edge.a : this.edge.b; + }, + end: function() { + return this.edge.l === this.site ? this.edge.b : this.edge.a; + } + }; + function d3_geom_voronoiRedBlackTree() { + this._ = null; + } + function d3_geom_voronoiRedBlackNode(node) { + node.U = node.C = node.L = node.R = node.P = node.N = null; + } + d3_geom_voronoiRedBlackTree.prototype = { + insert: function(after, node) { + var parent, grandpa, uncle; + if (after) { + node.P = after; + node.N = after.N; + if (after.N) after.N.P = node; + after.N = node; + if (after.R) { + after = after.R; + while (after.L) after = after.L; + after.L = node; + } else { + after.R = node; + } + parent = after; + } else if (this._) { + after = d3_geom_voronoiRedBlackFirst(this._); + node.P = null; + node.N = after; + after.P = after.L = node; + parent = after; + } else { + node.P = node.N = null; + this._ = node; + parent = null; + } + node.L = node.R = null; + node.U = parent; + node.C = true; + after = node; + while (parent && parent.C) { + grandpa = parent.U; + if (parent === grandpa.L) { + uncle = grandpa.R; + if (uncle && uncle.C) { + parent.C = uncle.C = false; + grandpa.C = true; + after = grandpa; + } else { + if (after === parent.R) { + d3_geom_voronoiRedBlackRotateLeft(this, parent); + after = parent; + parent = after.U; + } + parent.C = false; + grandpa.C = true; + d3_geom_voronoiRedBlackRotateRight(this, grandpa); + } + } else { + uncle = grandpa.L; + if (uncle && uncle.C) { + parent.C = uncle.C = false; + grandpa.C = true; + after = grandpa; + } else { + if (after === parent.L) { + d3_geom_voronoiRedBlackRotateRight(this, parent); + after = parent; + parent = after.U; + } + parent.C = false; + grandpa.C = true; + d3_geom_voronoiRedBlackRotateLeft(this, grandpa); + } + } + parent = after.U; + } + this._.C = false; + }, + remove: function(node) { + if (node.N) node.N.P = node.P; + if (node.P) node.P.N = node.N; + node.N = node.P = null; + var parent = node.U, sibling, left = node.L, right = node.R, next, red; + if (!left) next = right; else if (!right) next = left; else next = d3_geom_voronoiRedBlackFirst(right); + if (parent) { + if (parent.L === node) parent.L = next; else parent.R = next; + } else { + this._ = next; + } + if (left && right) { + red = next.C; + next.C = node.C; + next.L = left; + left.U = next; + if (next !== right) { + parent = next.U; + next.U = node.U; + node = next.R; + parent.L = node; + next.R = right; + right.U = next; + } else { + next.U = parent; + parent = next; + node = next.R; + } + } else { + red = node.C; + node = next; + } + if (node) node.U = parent; + if (red) return; + if (node && node.C) { + node.C = false; + return; + } + do { + if (node === this._) break; + if (node === parent.L) { + sibling = parent.R; + if (sibling.C) { + sibling.C = false; + parent.C = true; + d3_geom_voronoiRedBlackRotateLeft(this, parent); + sibling = parent.R; + } + if (sibling.L && sibling.L.C || sibling.R && sibling.R.C) { + if (!sibling.R || !sibling.R.C) { + sibling.L.C = false; + sibling.C = true; + d3_geom_voronoiRedBlackRotateRight(this, sibling); + sibling = parent.R; + } + sibling.C = parent.C; + parent.C = sibling.R.C = false; + d3_geom_voronoiRedBlackRotateLeft(this, parent); + node = this._; + break; + } + } else { + sibling = parent.L; + if (sibling.C) { + sibling.C = false; + parent.C = true; + d3_geom_voronoiRedBlackRotateRight(this, parent); + sibling = parent.L; + } + if (sibling.L && sibling.L.C || sibling.R && sibling.R.C) { + if (!sibling.L || !sibling.L.C) { + sibling.R.C = false; + sibling.C = true; + d3_geom_voronoiRedBlackRotateLeft(this, sibling); + sibling = parent.L; + } + sibling.C = parent.C; + parent.C = sibling.L.C = false; + d3_geom_voronoiRedBlackRotateRight(this, parent); + node = this._; + break; + } + } + sibling.C = true; + node = parent; + parent = parent.U; + } while (!node.C); + if (node) node.C = false; + } + }; + function d3_geom_voronoiRedBlackRotateLeft(tree, node) { + var p = node, q = node.R, parent = p.U; + if (parent) { + if (parent.L === p) parent.L = q; else parent.R = q; + } else { + tree._ = q; + } + q.U = parent; + p.U = q; + p.R = q.L; + if (p.R) p.R.U = p; + q.L = p; + } + function d3_geom_voronoiRedBlackRotateRight(tree, node) { + var p = node, q = node.L, parent = p.U; + if (parent) { + if (parent.L === p) parent.L = q; else parent.R = q; + } else { + tree._ = q; + } + q.U = parent; + p.U = q; + p.L = q.R; + if (p.L) p.L.U = p; + q.R = p; + } + function d3_geom_voronoiRedBlackFirst(node) { + while (node.L) node = node.L; + return node; + } + function d3_geom_voronoi(sites, bbox) { + var site = sites.sort(d3_geom_voronoiVertexOrder).pop(), x0, y0, circle; + d3_geom_voronoiEdges = []; + d3_geom_voronoiCells = new Array(sites.length); + d3_geom_voronoiBeaches = new d3_geom_voronoiRedBlackTree(); + d3_geom_voronoiCircles = new d3_geom_voronoiRedBlackTree(); + while (true) { + circle = d3_geom_voronoiFirstCircle; + if (site && (!circle || site.y < circle.y || site.y === circle.y && site.x < circle.x)) { + if (site.x !== x0 || site.y !== y0) { + d3_geom_voronoiCells[site.i] = new d3_geom_voronoiCell(site); + d3_geom_voronoiAddBeach(site); + x0 = site.x, y0 = site.y; + } + site = sites.pop(); + } else if (circle) { + d3_geom_voronoiRemoveBeach(circle.arc); + } else { + break; + } + } + if (bbox) d3_geom_voronoiClipEdges(bbox), d3_geom_voronoiCloseCells(bbox); + var diagram = { + cells: d3_geom_voronoiCells, + edges: d3_geom_voronoiEdges + }; + d3_geom_voronoiBeaches = d3_geom_voronoiCircles = d3_geom_voronoiEdges = d3_geom_voronoiCells = null; + return diagram; + } + function d3_geom_voronoiVertexOrder(a, b) { + return b.y - a.y || b.x - a.x; + } + d3.geom.voronoi = function(points) { + var x = d3_geom_pointX, y = d3_geom_pointY, fx = x, fy = y, clipExtent = d3_geom_voronoiClipExtent; + if (points) return voronoi(points); + function voronoi(data) { + var polygons = new Array(data.length), x0 = clipExtent[0][0], y0 = clipExtent[0][1], x1 = clipExtent[1][0], y1 = clipExtent[1][1]; + d3_geom_voronoi(sites(data), clipExtent).cells.forEach(function(cell, i) { + var edges = cell.edges, site = cell.site, polygon = polygons[i] = edges.length ? edges.map(function(e) { + var s = e.start(); + return [ s.x, s.y ]; + }) : site.x >= x0 && site.x <= x1 && site.y >= y0 && site.y <= y1 ? [ [ x0, y1 ], [ x1, y1 ], [ x1, y0 ], [ x0, y0 ] ] : []; + polygon.point = data[i]; + }); + return polygons; + } + function sites(data) { + return data.map(function(d, i) { + return { + x: Math.round(fx(d, i) / ε) * ε, + y: Math.round(fy(d, i) / ε) * ε, + i: i + }; + }); + } + voronoi.links = function(data) { + return d3_geom_voronoi(sites(data)).edges.filter(function(edge) { + return edge.l && edge.r; + }).map(function(edge) { + return { + source: data[edge.l.i], + target: data[edge.r.i] + }; + }); + }; + voronoi.triangles = function(data) { + var triangles = []; + d3_geom_voronoi(sites(data)).cells.forEach(function(cell, i) { + var site = cell.site, edges = cell.edges.sort(d3_geom_voronoiHalfEdgeOrder), j = -1, m = edges.length, e0, s0, e1 = edges[m - 1].edge, s1 = e1.l === site ? e1.r : e1.l; + while (++j < m) { + e0 = e1; + s0 = s1; + e1 = edges[j].edge; + s1 = e1.l === site ? e1.r : e1.l; + if (i < s0.i && i < s1.i && d3_geom_voronoiTriangleArea(site, s0, s1) < 0) { + triangles.push([ data[i], data[s0.i], data[s1.i] ]); + } + } + }); + return triangles; + }; + voronoi.x = function(_) { + return arguments.length ? (fx = d3_functor(x = _), voronoi) : x; + }; + voronoi.y = function(_) { + return arguments.length ? (fy = d3_functor(y = _), voronoi) : y; + }; + voronoi.clipExtent = function(_) { + if (!arguments.length) return clipExtent === d3_geom_voronoiClipExtent ? null : clipExtent; + clipExtent = _ == null ? d3_geom_voronoiClipExtent : _; + return voronoi; + }; + voronoi.size = function(_) { + if (!arguments.length) return clipExtent === d3_geom_voronoiClipExtent ? null : clipExtent && clipExtent[1]; + return voronoi.clipExtent(_ && [ [ 0, 0 ], _ ]); + }; + return voronoi; + }; + var d3_geom_voronoiClipExtent = [ [ -1e6, -1e6 ], [ 1e6, 1e6 ] ]; + function d3_geom_voronoiTriangleArea(a, b, c) { + return (a.x - c.x) * (b.y - a.y) - (a.x - b.x) * (c.y - a.y); + } + d3.geom.delaunay = function(vertices) { + return d3.geom.voronoi().triangles(vertices); + }; + d3.geom.quadtree = function(points, x1, y1, x2, y2) { + var x = d3_geom_pointX, y = d3_geom_pointY, compat; + if (compat = arguments.length) { + x = d3_geom_quadtreeCompatX; + y = d3_geom_quadtreeCompatY; + if (compat === 3) { + y2 = y1; + x2 = x1; + y1 = x1 = 0; + } + return quadtree(points); + } + function quadtree(data) { + var d, fx = d3_functor(x), fy = d3_functor(y), xs, ys, i, n, x1_, y1_, x2_, y2_; + if (x1 != null) { + x1_ = x1, y1_ = y1, x2_ = x2, y2_ = y2; + } else { + x2_ = y2_ = -(x1_ = y1_ = Infinity); + xs = [], ys = []; + n = data.length; + if (compat) for (i = 0; i < n; ++i) { + d = data[i]; + if (d.x < x1_) x1_ = d.x; + if (d.y < y1_) y1_ = d.y; + if (d.x > x2_) x2_ = d.x; + if (d.y > y2_) y2_ = d.y; + xs.push(d.x); + ys.push(d.y); + } else for (i = 0; i < n; ++i) { + var x_ = +fx(d = data[i], i), y_ = +fy(d, i); + if (x_ < x1_) x1_ = x_; + if (y_ < y1_) y1_ = y_; + if (x_ > x2_) x2_ = x_; + if (y_ > y2_) y2_ = y_; + xs.push(x_); + ys.push(y_); + } + } + var dx = x2_ - x1_, dy = y2_ - y1_; + if (dx > dy) y2_ = y1_ + dx; else x2_ = x1_ + dy; + function insert(n, d, x, y, x1, y1, x2, y2) { + if (isNaN(x) || isNaN(y)) return; + if (n.leaf) { + var nx = n.x, ny = n.y; + if (nx != null) { + if (abs(nx - x) + abs(ny - y) < .01) { + insertChild(n, d, x, y, x1, y1, x2, y2); + } else { + var nPoint = n.point; + n.x = n.y = n.point = null; + insertChild(n, nPoint, nx, ny, x1, y1, x2, y2); + insertChild(n, d, x, y, x1, y1, x2, y2); + } + } else { + n.x = x, n.y = y, n.point = d; + } + } else { + insertChild(n, d, x, y, x1, y1, x2, y2); + } + } + function insertChild(n, d, x, y, x1, y1, x2, y2) { + var sx = (x1 + x2) * .5, sy = (y1 + y2) * .5, right = x >= sx, bottom = y >= sy, i = (bottom << 1) + right; + n.leaf = false; + n = n.nodes[i] || (n.nodes[i] = d3_geom_quadtreeNode()); + if (right) x1 = sx; else x2 = sx; + if (bottom) y1 = sy; else y2 = sy; + insert(n, d, x, y, x1, y1, x2, y2); + } + var root = d3_geom_quadtreeNode(); + root.add = function(d) { + insert(root, d, +fx(d, ++i), +fy(d, i), x1_, y1_, x2_, y2_); + }; + root.visit = function(f) { + d3_geom_quadtreeVisit(f, root, x1_, y1_, x2_, y2_); + }; + i = -1; + if (x1 == null) { + while (++i < n) { + insert(root, data[i], xs[i], ys[i], x1_, y1_, x2_, y2_); + } + --i; + } else data.forEach(root.add); + xs = ys = data = d = null; + return root; + } + quadtree.x = function(_) { + return arguments.length ? (x = _, quadtree) : x; + }; + quadtree.y = function(_) { + return arguments.length ? (y = _, quadtree) : y; + }; + quadtree.extent = function(_) { + if (!arguments.length) return x1 == null ? null : [ [ x1, y1 ], [ x2, y2 ] ]; + if (_ == null) x1 = y1 = x2 = y2 = null; else x1 = +_[0][0], y1 = +_[0][1], x2 = +_[1][0], + y2 = +_[1][1]; + return quadtree; + }; + quadtree.size = function(_) { + if (!arguments.length) return x1 == null ? null : [ x2 - x1, y2 - y1 ]; + if (_ == null) x1 = y1 = x2 = y2 = null; else x1 = y1 = 0, x2 = +_[0], y2 = +_[1]; + return quadtree; + }; + return quadtree; + }; + function d3_geom_quadtreeCompatX(d) { + return d.x; + } + function d3_geom_quadtreeCompatY(d) { + return d.y; + } + function d3_geom_quadtreeNode() { + return { + leaf: true, + nodes: [], + point: null, + x: null, + y: null + }; + } + function d3_geom_quadtreeVisit(f, node, x1, y1, x2, y2) { + if (!f(node, x1, y1, x2, y2)) { + var sx = (x1 + x2) * .5, sy = (y1 + y2) * .5, children = node.nodes; + if (children[0]) d3_geom_quadtreeVisit(f, children[0], x1, y1, sx, sy); + if (children[1]) d3_geom_quadtreeVisit(f, children[1], sx, y1, x2, sy); + if (children[2]) d3_geom_quadtreeVisit(f, children[2], x1, sy, sx, y2); + if (children[3]) d3_geom_quadtreeVisit(f, children[3], sx, sy, x2, y2); + } + } + d3.interpolateRgb = d3_interpolateRgb; + function d3_interpolateRgb(a, b) { + a = d3.rgb(a); + b = d3.rgb(b); + var ar = a.r, ag = a.g, ab = a.b, br = b.r - ar, bg = b.g - ag, bb = b.b - ab; + return function(t) { + return "#" + d3_rgb_hex(Math.round(ar + br * t)) + d3_rgb_hex(Math.round(ag + bg * t)) + d3_rgb_hex(Math.round(ab + bb * t)); + }; + } + d3.interpolateObject = d3_interpolateObject; + function d3_interpolateObject(a, b) { + var i = {}, c = {}, k; + for (k in a) { + if (k in b) { + i[k] = d3_interpolate(a[k], b[k]); + } else { + c[k] = a[k]; + } + } + for (k in b) { + if (!(k in a)) { + c[k] = b[k]; + } + } + return function(t) { + for (k in i) c[k] = i[k](t); + return c; + }; + } + d3.interpolateNumber = d3_interpolateNumber; + function d3_interpolateNumber(a, b) { + b -= a = +a; + return function(t) { + return a + b * t; + }; + } + d3.interpolateString = d3_interpolateString; + function d3_interpolateString(a, b) { + var bi = d3_interpolate_numberA.lastIndex = d3_interpolate_numberB.lastIndex = 0, am, bm, bs, i = -1, s = [], q = []; + a = a + "", b = b + ""; + while ((am = d3_interpolate_numberA.exec(a)) && (bm = d3_interpolate_numberB.exec(b))) { + if ((bs = bm.index) > bi) { + bs = b.substring(bi, bs); + if (s[i]) s[i] += bs; else s[++i] = bs; + } + if ((am = am[0]) === (bm = bm[0])) { + if (s[i]) s[i] += bm; else s[++i] = bm; + } else { + s[++i] = null; + q.push({ + i: i, + x: d3_interpolateNumber(am, bm) + }); + } + bi = d3_interpolate_numberB.lastIndex; + } + if (bi < b.length) { + bs = b.substring(bi); + if (s[i]) s[i] += bs; else s[++i] = bs; + } + return s.length < 2 ? q[0] ? (b = q[0].x, function(t) { + return b(t) + ""; + }) : function() { + return b; + } : (b = q.length, function(t) { + for (var i = 0, o; i < b; ++i) s[(o = q[i]).i] = o.x(t); + return s.join(""); + }); + } + var d3_interpolate_numberA = /[-+]?(?:\d+\.?\d*|\.?\d+)(?:[eE][-+]?\d+)?/g, d3_interpolate_numberB = new RegExp(d3_interpolate_numberA.source, "g"); + d3.interpolate = d3_interpolate; + function d3_interpolate(a, b) { + var i = d3.interpolators.length, f; + while (--i >= 0 && !(f = d3.interpolators[i](a, b))) ; + return f; + } + d3.interpolators = [ function(a, b) { + var t = typeof b; + return (t === "string" ? d3_rgb_names.has(b) || /^(#|rgb\(|hsl\()/.test(b) ? d3_interpolateRgb : d3_interpolateString : b instanceof d3_Color ? d3_interpolateRgb : Array.isArray(b) ? d3_interpolateArray : t === "object" && isNaN(b) ? d3_interpolateObject : d3_interpolateNumber)(a, b); + } ]; + d3.interpolateArray = d3_interpolateArray; + function d3_interpolateArray(a, b) { + var x = [], c = [], na = a.length, nb = b.length, n0 = Math.min(a.length, b.length), i; + for (i = 0; i < n0; ++i) x.push(d3_interpolate(a[i], b[i])); + for (;i < na; ++i) c[i] = a[i]; + for (;i < nb; ++i) c[i] = b[i]; + return function(t) { + for (i = 0; i < n0; ++i) c[i] = x[i](t); + return c; + }; + } + var d3_ease_default = function() { + return d3_identity; + }; + var d3_ease = d3.map({ + linear: d3_ease_default, + poly: d3_ease_poly, + quad: function() { + return d3_ease_quad; + }, + cubic: function() { + return d3_ease_cubic; + }, + sin: function() { + return d3_ease_sin; + }, + exp: function() { + return d3_ease_exp; + }, + circle: function() { + return d3_ease_circle; + }, + elastic: d3_ease_elastic, + back: d3_ease_back, + bounce: function() { + return d3_ease_bounce; + } + }); + var d3_ease_mode = d3.map({ + "in": d3_identity, + out: d3_ease_reverse, + "in-out": d3_ease_reflect, + "out-in": function(f) { + return d3_ease_reflect(d3_ease_reverse(f)); + } + }); + d3.ease = function(name) { + var i = name.indexOf("-"), t = i >= 0 ? name.substring(0, i) : name, m = i >= 0 ? name.substring(i + 1) : "in"; + t = d3_ease.get(t) || d3_ease_default; + m = d3_ease_mode.get(m) || d3_identity; + return d3_ease_clamp(m(t.apply(null, d3_arraySlice.call(arguments, 1)))); + }; + function d3_ease_clamp(f) { + return function(t) { + return t <= 0 ? 0 : t >= 1 ? 1 : f(t); + }; + } + function d3_ease_reverse(f) { + return function(t) { + return 1 - f(1 - t); + }; + } + function d3_ease_reflect(f) { + return function(t) { + return .5 * (t < .5 ? f(2 * t) : 2 - f(2 - 2 * t)); + }; + } + function d3_ease_quad(t) { + return t * t; + } + function d3_ease_cubic(t) { + return t * t * t; + } + function d3_ease_cubicInOut(t) { + if (t <= 0) return 0; + if (t >= 1) return 1; + var t2 = t * t, t3 = t2 * t; + return 4 * (t < .5 ? t3 : 3 * (t - t2) + t3 - .75); + } + function d3_ease_poly(e) { + return function(t) { + return Math.pow(t, e); + }; + } + function d3_ease_sin(t) { + return 1 - Math.cos(t * halfπ); + } + function d3_ease_exp(t) { + return Math.pow(2, 10 * (t - 1)); + } + function d3_ease_circle(t) { + return 1 - Math.sqrt(1 - t * t); + } + function d3_ease_elastic(a, p) { + var s; + if (arguments.length < 2) p = .45; + if (arguments.length) s = p / τ * Math.asin(1 / a); else a = 1, s = p / 4; + return function(t) { + return 1 + a * Math.pow(2, -10 * t) * Math.sin((t - s) * τ / p); + }; + } + function d3_ease_back(s) { + if (!s) s = 1.70158; + return function(t) { + return t * t * ((s + 1) * t - s); + }; + } + function d3_ease_bounce(t) { + return t < 1 / 2.75 ? 7.5625 * t * t : t < 2 / 2.75 ? 7.5625 * (t -= 1.5 / 2.75) * t + .75 : t < 2.5 / 2.75 ? 7.5625 * (t -= 2.25 / 2.75) * t + .9375 : 7.5625 * (t -= 2.625 / 2.75) * t + .984375; + } + d3.interpolateHcl = d3_interpolateHcl; + function d3_interpolateHcl(a, b) { + a = d3.hcl(a); + b = d3.hcl(b); + var ah = a.h, ac = a.c, al = a.l, bh = b.h - ah, bc = b.c - ac, bl = b.l - al; + if (isNaN(bc)) bc = 0, ac = isNaN(ac) ? b.c : ac; + if (isNaN(bh)) bh = 0, ah = isNaN(ah) ? b.h : ah; else if (bh > 180) bh -= 360; else if (bh < -180) bh += 360; + return function(t) { + return d3_hcl_lab(ah + bh * t, ac + bc * t, al + bl * t) + ""; + }; + } + d3.interpolateHsl = d3_interpolateHsl; + function d3_interpolateHsl(a, b) { + a = d3.hsl(a); + b = d3.hsl(b); + var ah = a.h, as = a.s, al = a.l, bh = b.h - ah, bs = b.s - as, bl = b.l - al; + if (isNaN(bs)) bs = 0, as = isNaN(as) ? b.s : as; + if (isNaN(bh)) bh = 0, ah = isNaN(ah) ? b.h : ah; else if (bh > 180) bh -= 360; else if (bh < -180) bh += 360; + return function(t) { + return d3_hsl_rgb(ah + bh * t, as + bs * t, al + bl * t) + ""; + }; + } + d3.interpolateLab = d3_interpolateLab; + function d3_interpolateLab(a, b) { + a = d3.lab(a); + b = d3.lab(b); + var al = a.l, aa = a.a, ab = a.b, bl = b.l - al, ba = b.a - aa, bb = b.b - ab; + return function(t) { + return d3_lab_rgb(al + bl * t, aa + ba * t, ab + bb * t) + ""; + }; + } + d3.interpolateRound = d3_interpolateRound; + function d3_interpolateRound(a, b) { + b -= a; + return function(t) { + return Math.round(a + b * t); + }; + } + d3.transform = function(string) { + var g = d3_document.createElementNS(d3.ns.prefix.svg, "g"); + return (d3.transform = function(string) { + if (string != null) { + g.setAttribute("transform", string); + var t = g.transform.baseVal.consolidate(); + } + return new d3_transform(t ? t.matrix : d3_transformIdentity); + })(string); + }; + function d3_transform(m) { + var r0 = [ m.a, m.b ], r1 = [ m.c, m.d ], kx = d3_transformNormalize(r0), kz = d3_transformDot(r0, r1), ky = d3_transformNormalize(d3_transformCombine(r1, r0, -kz)) || 0; + if (r0[0] * r1[1] < r1[0] * r0[1]) { + r0[0] *= -1; + r0[1] *= -1; + kx *= -1; + kz *= -1; + } + this.rotate = (kx ? Math.atan2(r0[1], r0[0]) : Math.atan2(-r1[0], r1[1])) * d3_degrees; + this.translate = [ m.e, m.f ]; + this.scale = [ kx, ky ]; + this.skew = ky ? Math.atan2(kz, ky) * d3_degrees : 0; + } + d3_transform.prototype.toString = function() { + return "translate(" + this.translate + ")rotate(" + this.rotate + ")skewX(" + this.skew + ")scale(" + this.scale + ")"; + }; + function d3_transformDot(a, b) { + return a[0] * b[0] + a[1] * b[1]; + } + function d3_transformNormalize(a) { + var k = Math.sqrt(d3_transformDot(a, a)); + if (k) { + a[0] /= k; + a[1] /= k; + } + return k; + } + function d3_transformCombine(a, b, k) { + a[0] += k * b[0]; + a[1] += k * b[1]; + return a; + } + var d3_transformIdentity = { + a: 1, + b: 0, + c: 0, + d: 1, + e: 0, + f: 0 + }; + d3.interpolateTransform = d3_interpolateTransform; + function d3_interpolateTransform(a, b) { + var s = [], q = [], n, A = d3.transform(a), B = d3.transform(b), ta = A.translate, tb = B.translate, ra = A.rotate, rb = B.rotate, wa = A.skew, wb = B.skew, ka = A.scale, kb = B.scale; + if (ta[0] != tb[0] || ta[1] != tb[1]) { + s.push("translate(", null, ",", null, ")"); + q.push({ + i: 1, + x: d3_interpolateNumber(ta[0], tb[0]) + }, { + i: 3, + x: d3_interpolateNumber(ta[1], tb[1]) + }); + } else if (tb[0] || tb[1]) { + s.push("translate(" + tb + ")"); + } else { + s.push(""); + } + if (ra != rb) { + if (ra - rb > 180) rb += 360; else if (rb - ra > 180) ra += 360; + q.push({ + i: s.push(s.pop() + "rotate(", null, ")") - 2, + x: d3_interpolateNumber(ra, rb) + }); + } else if (rb) { + s.push(s.pop() + "rotate(" + rb + ")"); + } + if (wa != wb) { + q.push({ + i: s.push(s.pop() + "skewX(", null, ")") - 2, + x: d3_interpolateNumber(wa, wb) + }); + } else if (wb) { + s.push(s.pop() + "skewX(" + wb + ")"); + } + if (ka[0] != kb[0] || ka[1] != kb[1]) { + n = s.push(s.pop() + "scale(", null, ",", null, ")"); + q.push({ + i: n - 4, + x: d3_interpolateNumber(ka[0], kb[0]) + }, { + i: n - 2, + x: d3_interpolateNumber(ka[1], kb[1]) + }); + } else if (kb[0] != 1 || kb[1] != 1) { + s.push(s.pop() + "scale(" + kb + ")"); + } + n = q.length; + return function(t) { + var i = -1, o; + while (++i < n) s[(o = q[i]).i] = o.x(t); + return s.join(""); + }; + } + function d3_uninterpolateNumber(a, b) { + b = b - (a = +a) ? 1 / (b - a) : 0; + return function(x) { + return (x - a) * b; + }; + } + function d3_uninterpolateClamp(a, b) { + b = b - (a = +a) ? 1 / (b - a) : 0; + return function(x) { + return Math.max(0, Math.min(1, (x - a) * b)); + }; + } + d3.layout = {}; + d3.layout.bundle = function() { + return function(links) { + var paths = [], i = -1, n = links.length; + while (++i < n) paths.push(d3_layout_bundlePath(links[i])); + return paths; + }; + }; + function d3_layout_bundlePath(link) { + var start = link.source, end = link.target, lca = d3_layout_bundleLeastCommonAncestor(start, end), points = [ start ]; + while (start !== lca) { + start = start.parent; + points.push(start); + } + var k = points.length; + while (end !== lca) { + points.splice(k, 0, end); + end = end.parent; + } + return points; + } + function d3_layout_bundleAncestors(node) { + var ancestors = [], parent = node.parent; + while (parent != null) { + ancestors.push(node); + node = parent; + parent = parent.parent; + } + ancestors.push(node); + return ancestors; + } + function d3_layout_bundleLeastCommonAncestor(a, b) { + if (a === b) return a; + var aNodes = d3_layout_bundleAncestors(a), bNodes = d3_layout_bundleAncestors(b), aNode = aNodes.pop(), bNode = bNodes.pop(), sharedNode = null; + while (aNode === bNode) { + sharedNode = aNode; + aNode = aNodes.pop(); + bNode = bNodes.pop(); + } + return sharedNode; + } + d3.layout.chord = function() { + var chord = {}, chords, groups, matrix, n, padding = 0, sortGroups, sortSubgroups, sortChords; + function relayout() { + var subgroups = {}, groupSums = [], groupIndex = d3.range(n), subgroupIndex = [], k, x, x0, i, j; + chords = []; + groups = []; + k = 0, i = -1; + while (++i < n) { + x = 0, j = -1; + while (++j < n) { + x += matrix[i][j]; + } + groupSums.push(x); + subgroupIndex.push(d3.range(n)); + k += x; + } + if (sortGroups) { + groupIndex.sort(function(a, b) { + return sortGroups(groupSums[a], groupSums[b]); + }); + } + if (sortSubgroups) { + subgroupIndex.forEach(function(d, i) { + d.sort(function(a, b) { + return sortSubgroups(matrix[i][a], matrix[i][b]); + }); + }); + } + k = (τ - padding * n) / k; + x = 0, i = -1; + while (++i < n) { + x0 = x, j = -1; + while (++j < n) { + var di = groupIndex[i], dj = subgroupIndex[di][j], v = matrix[di][dj], a0 = x, a1 = x += v * k; + subgroups[di + "-" + dj] = { + index: di, + subindex: dj, + startAngle: a0, + endAngle: a1, + value: v + }; + } + groups[di] = { + index: di, + startAngle: x0, + endAngle: x, + value: (x - x0) / k + }; + x += padding; + } + i = -1; + while (++i < n) { + j = i - 1; + while (++j < n) { + var source = subgroups[i + "-" + j], target = subgroups[j + "-" + i]; + if (source.value || target.value) { + chords.push(source.value < target.value ? { + source: target, + target: source + } : { + source: source, + target: target + }); + } + } + } + if (sortChords) resort(); + } + function resort() { + chords.sort(function(a, b) { + return sortChords((a.source.value + a.target.value) / 2, (b.source.value + b.target.value) / 2); + }); + } + chord.matrix = function(x) { + if (!arguments.length) return matrix; + n = (matrix = x) && matrix.length; + chords = groups = null; + return chord; + }; + chord.padding = function(x) { + if (!arguments.length) return padding; + padding = x; + chords = groups = null; + return chord; + }; + chord.sortGroups = function(x) { + if (!arguments.length) return sortGroups; + sortGroups = x; + chords = groups = null; + return chord; + }; + chord.sortSubgroups = function(x) { + if (!arguments.length) return sortSubgroups; + sortSubgroups = x; + chords = null; + return chord; + }; + chord.sortChords = function(x) { + if (!arguments.length) return sortChords; + sortChords = x; + if (chords) resort(); + return chord; + }; + chord.chords = function() { + if (!chords) relayout(); + return chords; + }; + chord.groups = function() { + if (!groups) relayout(); + return groups; + }; + return chord; + }; + d3.layout.force = function() { + var force = {}, event = d3.dispatch("start", "tick", "end"), size = [ 1, 1 ], drag, alpha, friction = .9, linkDistance = d3_layout_forceLinkDistance, linkStrength = d3_layout_forceLinkStrength, charge = -30, chargeDistance2 = d3_layout_forceChargeDistance2, gravity = .1, theta2 = .64, nodes = [], links = [], distances, strengths, charges; + function repulse(node) { + return function(quad, x1, _, x2) { + if (quad.point !== node) { + var dx = quad.cx - node.x, dy = quad.cy - node.y, dw = x2 - x1, dn = dx * dx + dy * dy; + if (dw * dw / theta2 < dn) { + if (dn < chargeDistance2) { + var k = quad.charge / dn; + node.px -= dx * k; + node.py -= dy * k; + } + return true; + } + if (quad.point && dn && dn < chargeDistance2) { + var k = quad.pointCharge / dn; + node.px -= dx * k; + node.py -= dy * k; + } + } + return !quad.charge; + }; + } + force.tick = function() { + if ((alpha *= .99) < .005) { + event.end({ + type: "end", + alpha: alpha = 0 + }); + return true; + } + var n = nodes.length, m = links.length, q, i, o, s, t, l, k, x, y; + for (i = 0; i < m; ++i) { + o = links[i]; + s = o.source; + t = o.target; + x = t.x - s.x; + y = t.y - s.y; + if (l = x * x + y * y) { + l = alpha * strengths[i] * ((l = Math.sqrt(l)) - distances[i]) / l; + x *= l; + y *= l; + t.x -= x * (k = s.weight / (t.weight + s.weight)); + t.y -= y * k; + s.x += x * (k = 1 - k); + s.y += y * k; + } + } + if (k = alpha * gravity) { + x = size[0] / 2; + y = size[1] / 2; + i = -1; + if (k) while (++i < n) { + o = nodes[i]; + o.x += (x - o.x) * k; + o.y += (y - o.y) * k; + } + } + if (charge) { + d3_layout_forceAccumulate(q = d3.geom.quadtree(nodes), alpha, charges); + i = -1; + while (++i < n) { + if (!(o = nodes[i]).fixed) { + q.visit(repulse(o)); + } + } + } + i = -1; + while (++i < n) { + o = nodes[i]; + if (o.fixed) { + o.x = o.px; + o.y = o.py; + } else { + o.x -= (o.px - (o.px = o.x)) * friction; + o.y -= (o.py - (o.py = o.y)) * friction; + } + } + event.tick({ + type: "tick", + alpha: alpha + }); + }; + force.nodes = function(x) { + if (!arguments.length) return nodes; + nodes = x; + return force; + }; + force.links = function(x) { + if (!arguments.length) return links; + links = x; + return force; + }; + force.size = function(x) { + if (!arguments.length) return size; + size = x; + return force; + }; + force.linkDistance = function(x) { + if (!arguments.length) return linkDistance; + linkDistance = typeof x === "function" ? x : +x; + return force; + }; + force.distance = force.linkDistance; + force.linkStrength = function(x) { + if (!arguments.length) return linkStrength; + linkStrength = typeof x === "function" ? x : +x; + return force; + }; + force.friction = function(x) { + if (!arguments.length) return friction; + friction = +x; + return force; + }; + force.charge = function(x) { + if (!arguments.length) return charge; + charge = typeof x === "function" ? x : +x; + return force; + }; + force.chargeDistance = function(x) { + if (!arguments.length) return Math.sqrt(chargeDistance2); + chargeDistance2 = x * x; + return force; + }; + force.gravity = function(x) { + if (!arguments.length) return gravity; + gravity = +x; + return force; + }; + force.theta = function(x) { + if (!arguments.length) return Math.sqrt(theta2); + theta2 = x * x; + return force; + }; + force.alpha = function(x) { + if (!arguments.length) return alpha; + x = +x; + if (alpha) { + if (x > 0) alpha = x; else alpha = 0; + } else if (x > 0) { + event.start({ + type: "start", + alpha: alpha = x + }); + d3.timer(force.tick); + } + return force; + }; + force.start = function() { + var i, n = nodes.length, m = links.length, w = size[0], h = size[1], neighbors, o; + for (i = 0; i < n; ++i) { + (o = nodes[i]).index = i; + o.weight = 0; + } + for (i = 0; i < m; ++i) { + o = links[i]; + if (typeof o.source == "number") o.source = nodes[o.source]; + if (typeof o.target == "number") o.target = nodes[o.target]; + ++o.source.weight; + ++o.target.weight; + } + for (i = 0; i < n; ++i) { + o = nodes[i]; + if (isNaN(o.x)) o.x = position("x", w); + if (isNaN(o.y)) o.y = position("y", h); + if (isNaN(o.px)) o.px = o.x; + if (isNaN(o.py)) o.py = o.y; + } + distances = []; + if (typeof linkDistance === "function") for (i = 0; i < m; ++i) distances[i] = +linkDistance.call(this, links[i], i); else for (i = 0; i < m; ++i) distances[i] = linkDistance; + strengths = []; + if (typeof linkStrength === "function") for (i = 0; i < m; ++i) strengths[i] = +linkStrength.call(this, links[i], i); else for (i = 0; i < m; ++i) strengths[i] = linkStrength; + charges = []; + if (typeof charge === "function") for (i = 0; i < n; ++i) charges[i] = +charge.call(this, nodes[i], i); else for (i = 0; i < n; ++i) charges[i] = charge; + function position(dimension, size) { + if (!neighbors) { + neighbors = new Array(n); + for (j = 0; j < n; ++j) { + neighbors[j] = []; + } + for (j = 0; j < m; ++j) { + var o = links[j]; + neighbors[o.source.index].push(o.target); + neighbors[o.target.index].push(o.source); + } + } + var candidates = neighbors[i], j = -1, m = candidates.length, x; + while (++j < m) if (!isNaN(x = candidates[j][dimension])) return x; + return Math.random() * size; + } + return force.resume(); + }; + force.resume = function() { + return force.alpha(.1); + }; + force.stop = function() { + return force.alpha(0); + }; + force.drag = function() { + if (!drag) drag = d3.behavior.drag().origin(d3_identity).on("dragstart.force", d3_layout_forceDragstart).on("drag.force", dragmove).on("dragend.force", d3_layout_forceDragend); + if (!arguments.length) return drag; + this.on("mouseover.force", d3_layout_forceMouseover).on("mouseout.force", d3_layout_forceMouseout).call(drag); + }; + function dragmove(d) { + d.px = d3.event.x, d.py = d3.event.y; + force.resume(); + } + return d3.rebind(force, event, "on"); + }; + function d3_layout_forceDragstart(d) { + d.fixed |= 2; + } + function d3_layout_forceDragend(d) { + d.fixed &= ~6; + } + function d3_layout_forceMouseover(d) { + d.fixed |= 4; + d.px = d.x, d.py = d.y; + } + function d3_layout_forceMouseout(d) { + d.fixed &= ~4; + } + function d3_layout_forceAccumulate(quad, alpha, charges) { + var cx = 0, cy = 0; + quad.charge = 0; + if (!quad.leaf) { + var nodes = quad.nodes, n = nodes.length, i = -1, c; + while (++i < n) { + c = nodes[i]; + if (c == null) continue; + d3_layout_forceAccumulate(c, alpha, charges); + quad.charge += c.charge; + cx += c.charge * c.cx; + cy += c.charge * c.cy; + } + } + if (quad.point) { + if (!quad.leaf) { + quad.point.x += Math.random() - .5; + quad.point.y += Math.random() - .5; + } + var k = alpha * charges[quad.point.index]; + quad.charge += quad.pointCharge = k; + cx += k * quad.point.x; + cy += k * quad.point.y; + } + quad.cx = cx / quad.charge; + quad.cy = cy / quad.charge; + } + var d3_layout_forceLinkDistance = 20, d3_layout_forceLinkStrength = 1, d3_layout_forceChargeDistance2 = Infinity; + d3.layout.hierarchy = function() { + var sort = d3_layout_hierarchySort, children = d3_layout_hierarchyChildren, value = d3_layout_hierarchyValue; + function recurse(node, depth, nodes) { + var childs = children.call(hierarchy, node, depth); + node.depth = depth; + nodes.push(node); + if (childs && (n = childs.length)) { + var i = -1, n, c = node.children = new Array(n), v = 0, j = depth + 1, d; + while (++i < n) { + d = c[i] = recurse(childs[i], j, nodes); + d.parent = node; + v += d.value; + } + if (sort) c.sort(sort); + if (value) node.value = v; + } else { + delete node.children; + if (value) { + node.value = +value.call(hierarchy, node, depth) || 0; + } + } + return node; + } + function revalue(node, depth) { + var children = node.children, v = 0; + if (children && (n = children.length)) { + var i = -1, n, j = depth + 1; + while (++i < n) v += revalue(children[i], j); + } else if (value) { + v = +value.call(hierarchy, node, depth) || 0; + } + if (value) node.value = v; + return v; + } + function hierarchy(d) { + var nodes = []; + recurse(d, 0, nodes); + return nodes; + } + hierarchy.sort = function(x) { + if (!arguments.length) return sort; + sort = x; + return hierarchy; + }; + hierarchy.children = function(x) { + if (!arguments.length) return children; + children = x; + return hierarchy; + }; + hierarchy.value = function(x) { + if (!arguments.length) return value; + value = x; + return hierarchy; + }; + hierarchy.revalue = function(root) { + revalue(root, 0); + return root; + }; + return hierarchy; + }; + function d3_layout_hierarchyRebind(object, hierarchy) { + d3.rebind(object, hierarchy, "sort", "children", "value"); + object.nodes = object; + object.links = d3_layout_hierarchyLinks; + return object; + } + function d3_layout_hierarchyChildren(d) { + return d.children; + } + function d3_layout_hierarchyValue(d) { + return d.value; + } + function d3_layout_hierarchySort(a, b) { + return b.value - a.value; + } + function d3_layout_hierarchyLinks(nodes) { + return d3.merge(nodes.map(function(parent) { + return (parent.children || []).map(function(child) { + return { + source: parent, + target: child + }; + }); + })); + } + d3.layout.partition = function() { + var hierarchy = d3.layout.hierarchy(), size = [ 1, 1 ]; + function position(node, x, dx, dy) { + var children = node.children; + node.x = x; + node.y = node.depth * dy; + node.dx = dx; + node.dy = dy; + if (children && (n = children.length)) { + var i = -1, n, c, d; + dx = node.value ? dx / node.value : 0; + while (++i < n) { + position(c = children[i], x, d = c.value * dx, dy); + x += d; + } + } + } + function depth(node) { + var children = node.children, d = 0; + if (children && (n = children.length)) { + var i = -1, n; + while (++i < n) d = Math.max(d, depth(children[i])); + } + return 1 + d; + } + function partition(d, i) { + var nodes = hierarchy.call(this, d, i); + position(nodes[0], 0, size[0], size[1] / depth(nodes[0])); + return nodes; + } + partition.size = function(x) { + if (!arguments.length) return size; + size = x; + return partition; + }; + return d3_layout_hierarchyRebind(partition, hierarchy); + }; + d3.layout.pie = function() { + var value = Number, sort = d3_layout_pieSortByValue, startAngle = 0, endAngle = τ; + function pie(data) { + var values = data.map(function(d, i) { + return +value.call(pie, d, i); + }); + var a = +(typeof startAngle === "function" ? startAngle.apply(this, arguments) : startAngle); + var k = ((typeof endAngle === "function" ? endAngle.apply(this, arguments) : endAngle) - a) / d3.sum(values); + var index = d3.range(data.length); + if (sort != null) index.sort(sort === d3_layout_pieSortByValue ? function(i, j) { + return values[j] - values[i]; + } : function(i, j) { + return sort(data[i], data[j]); + }); + var arcs = []; + index.forEach(function(i) { + var d; + arcs[i] = { + data: data[i], + value: d = values[i], + startAngle: a, + endAngle: a += d * k + }; + }); + return arcs; + } + pie.value = function(x) { + if (!arguments.length) return value; + value = x; + return pie; + }; + pie.sort = function(x) { + if (!arguments.length) return sort; + sort = x; + return pie; + }; + pie.startAngle = function(x) { + if (!arguments.length) return startAngle; + startAngle = x; + return pie; + }; + pie.endAngle = function(x) { + if (!arguments.length) return endAngle; + endAngle = x; + return pie; + }; + return pie; + }; + var d3_layout_pieSortByValue = {}; + d3.layout.stack = function() { + var values = d3_identity, order = d3_layout_stackOrderDefault, offset = d3_layout_stackOffsetZero, out = d3_layout_stackOut, x = d3_layout_stackX, y = d3_layout_stackY; + function stack(data, index) { + var series = data.map(function(d, i) { + return values.call(stack, d, i); + }); + var points = series.map(function(d) { + return d.map(function(v, i) { + return [ x.call(stack, v, i), y.call(stack, v, i) ]; + }); + }); + var orders = order.call(stack, points, index); + series = d3.permute(series, orders); + points = d3.permute(points, orders); + var offsets = offset.call(stack, points, index); + var n = series.length, m = series[0].length, i, j, o; + for (j = 0; j < m; ++j) { + out.call(stack, series[0][j], o = offsets[j], points[0][j][1]); + for (i = 1; i < n; ++i) { + out.call(stack, series[i][j], o += points[i - 1][j][1], points[i][j][1]); + } + } + return data; + } + stack.values = function(x) { + if (!arguments.length) return values; + values = x; + return stack; + }; + stack.order = function(x) { + if (!arguments.length) return order; + order = typeof x === "function" ? x : d3_layout_stackOrders.get(x) || d3_layout_stackOrderDefault; + return stack; + }; + stack.offset = function(x) { + if (!arguments.length) return offset; + offset = typeof x === "function" ? x : d3_layout_stackOffsets.get(x) || d3_layout_stackOffsetZero; + return stack; + }; + stack.x = function(z) { + if (!arguments.length) return x; + x = z; + return stack; + }; + stack.y = function(z) { + if (!arguments.length) return y; + y = z; + return stack; + }; + stack.out = function(z) { + if (!arguments.length) return out; + out = z; + return stack; + }; + return stack; + }; + function d3_layout_stackX(d) { + return d.x; + } + function d3_layout_stackY(d) { + return d.y; + } + function d3_layout_stackOut(d, y0, y) { + d.y0 = y0; + d.y = y; + } + var d3_layout_stackOrders = d3.map({ + "inside-out": function(data) { + var n = data.length, i, j, max = data.map(d3_layout_stackMaxIndex), sums = data.map(d3_layout_stackReduceSum), index = d3.range(n).sort(function(a, b) { + return max[a] - max[b]; + }), top = 0, bottom = 0, tops = [], bottoms = []; + for (i = 0; i < n; ++i) { + j = index[i]; + if (top < bottom) { + top += sums[j]; + tops.push(j); + } else { + bottom += sums[j]; + bottoms.push(j); + } + } + return bottoms.reverse().concat(tops); + }, + reverse: function(data) { + return d3.range(data.length).reverse(); + }, + "default": d3_layout_stackOrderDefault + }); + var d3_layout_stackOffsets = d3.map({ + silhouette: function(data) { + var n = data.length, m = data[0].length, sums = [], max = 0, i, j, o, y0 = []; + for (j = 0; j < m; ++j) { + for (i = 0, o = 0; i < n; i++) o += data[i][j][1]; + if (o > max) max = o; + sums.push(o); + } + for (j = 0; j < m; ++j) { + y0[j] = (max - sums[j]) / 2; + } + return y0; + }, + wiggle: function(data) { + var n = data.length, x = data[0], m = x.length, i, j, k, s1, s2, s3, dx, o, o0, y0 = []; + y0[0] = o = o0 = 0; + for (j = 1; j < m; ++j) { + for (i = 0, s1 = 0; i < n; ++i) s1 += data[i][j][1]; + for (i = 0, s2 = 0, dx = x[j][0] - x[j - 1][0]; i < n; ++i) { + for (k = 0, s3 = (data[i][j][1] - data[i][j - 1][1]) / (2 * dx); k < i; ++k) { + s3 += (data[k][j][1] - data[k][j - 1][1]) / dx; + } + s2 += s3 * data[i][j][1]; + } + y0[j] = o -= s1 ? s2 / s1 * dx : 0; + if (o < o0) o0 = o; + } + for (j = 0; j < m; ++j) y0[j] -= o0; + return y0; + }, + expand: function(data) { + var n = data.length, m = data[0].length, k = 1 / n, i, j, o, y0 = []; + for (j = 0; j < m; ++j) { + for (i = 0, o = 0; i < n; i++) o += data[i][j][1]; + if (o) for (i = 0; i < n; i++) data[i][j][1] /= o; else for (i = 0; i < n; i++) data[i][j][1] = k; + } + for (j = 0; j < m; ++j) y0[j] = 0; + return y0; + }, + zero: d3_layout_stackOffsetZero + }); + function d3_layout_stackOrderDefault(data) { + return d3.range(data.length); + } + function d3_layout_stackOffsetZero(data) { + var j = -1, m = data[0].length, y0 = []; + while (++j < m) y0[j] = 0; + return y0; + } + function d3_layout_stackMaxIndex(array) { + var i = 1, j = 0, v = array[0][1], k, n = array.length; + for (;i < n; ++i) { + if ((k = array[i][1]) > v) { + j = i; + v = k; + } + } + return j; + } + function d3_layout_stackReduceSum(d) { + return d.reduce(d3_layout_stackSum, 0); + } + function d3_layout_stackSum(p, d) { + return p + d[1]; + } + d3.layout.histogram = function() { + var frequency = true, valuer = Number, ranger = d3_layout_histogramRange, binner = d3_layout_histogramBinSturges; + function histogram(data, i) { + var bins = [], values = data.map(valuer, this), range = ranger.call(this, values, i), thresholds = binner.call(this, range, values, i), bin, i = -1, n = values.length, m = thresholds.length - 1, k = frequency ? 1 : 1 / n, x; + while (++i < m) { + bin = bins[i] = []; + bin.dx = thresholds[i + 1] - (bin.x = thresholds[i]); + bin.y = 0; + } + if (m > 0) { + i = -1; + while (++i < n) { + x = values[i]; + if (x >= range[0] && x <= range[1]) { + bin = bins[d3.bisect(thresholds, x, 1, m) - 1]; + bin.y += k; + bin.push(data[i]); + } + } + } + return bins; + } + histogram.value = function(x) { + if (!arguments.length) return valuer; + valuer = x; + return histogram; + }; + histogram.range = function(x) { + if (!arguments.length) return ranger; + ranger = d3_functor(x); + return histogram; + }; + histogram.bins = function(x) { + if (!arguments.length) return binner; + binner = typeof x === "number" ? function(range) { + return d3_layout_histogramBinFixed(range, x); + } : d3_functor(x); + return histogram; + }; + histogram.frequency = function(x) { + if (!arguments.length) return frequency; + frequency = !!x; + return histogram; + }; + return histogram; + }; + function d3_layout_histogramBinSturges(range, values) { + return d3_layout_histogramBinFixed(range, Math.ceil(Math.log(values.length) / Math.LN2 + 1)); + } + function d3_layout_histogramBinFixed(range, n) { + var x = -1, b = +range[0], m = (range[1] - b) / n, f = []; + while (++x <= n) f[x] = m * x + b; + return f; + } + function d3_layout_histogramRange(values) { + return [ d3.min(values), d3.max(values) ]; + } + d3.layout.tree = function() { + var hierarchy = d3.layout.hierarchy().sort(null).value(null), separation = d3_layout_treeSeparation, size = [ 1, 1 ], nodeSize = false; + function tree(d, i) { + var nodes = hierarchy.call(this, d, i), root = nodes[0]; + function firstWalk(node, previousSibling) { + var children = node.children, layout = node._tree; + if (children && (n = children.length)) { + var n, firstChild = children[0], previousChild, ancestor = firstChild, child, i = -1; + while (++i < n) { + child = children[i]; + firstWalk(child, previousChild); + ancestor = apportion(child, previousChild, ancestor); + previousChild = child; + } + d3_layout_treeShift(node); + var midpoint = .5 * (firstChild._tree.prelim + child._tree.prelim); + if (previousSibling) { + layout.prelim = previousSibling._tree.prelim + separation(node, previousSibling); + layout.mod = layout.prelim - midpoint; + } else { + layout.prelim = midpoint; + } + } else { + if (previousSibling) { + layout.prelim = previousSibling._tree.prelim + separation(node, previousSibling); + } + } + } + function secondWalk(node, x) { + node.x = node._tree.prelim + x; + var children = node.children; + if (children && (n = children.length)) { + var i = -1, n; + x += node._tree.mod; + while (++i < n) { + secondWalk(children[i], x); + } + } + } + function apportion(node, previousSibling, ancestor) { + if (previousSibling) { + var vip = node, vop = node, vim = previousSibling, vom = node.parent.children[0], sip = vip._tree.mod, sop = vop._tree.mod, sim = vim._tree.mod, som = vom._tree.mod, shift; + while (vim = d3_layout_treeRight(vim), vip = d3_layout_treeLeft(vip), vim && vip) { + vom = d3_layout_treeLeft(vom); + vop = d3_layout_treeRight(vop); + vop._tree.ancestor = node; + shift = vim._tree.prelim + sim - vip._tree.prelim - sip + separation(vim, vip); + if (shift > 0) { + d3_layout_treeMove(d3_layout_treeAncestor(vim, node, ancestor), node, shift); + sip += shift; + sop += shift; + } + sim += vim._tree.mod; + sip += vip._tree.mod; + som += vom._tree.mod; + sop += vop._tree.mod; + } + if (vim && !d3_layout_treeRight(vop)) { + vop._tree.thread = vim; + vop._tree.mod += sim - sop; + } + if (vip && !d3_layout_treeLeft(vom)) { + vom._tree.thread = vip; + vom._tree.mod += sip - som; + ancestor = node; + } + } + return ancestor; + } + d3_layout_treeVisitAfter(root, function(node, previousSibling) { + node._tree = { + ancestor: node, + prelim: 0, + mod: 0, + change: 0, + shift: 0, + number: previousSibling ? previousSibling._tree.number + 1 : 0 + }; + }); + firstWalk(root); + secondWalk(root, -root._tree.prelim); + var left = d3_layout_treeSearch(root, d3_layout_treeLeftmost), right = d3_layout_treeSearch(root, d3_layout_treeRightmost), deep = d3_layout_treeSearch(root, d3_layout_treeDeepest), x0 = left.x - separation(left, right) / 2, x1 = right.x + separation(right, left) / 2, y1 = deep.depth || 1; + d3_layout_treeVisitAfter(root, nodeSize ? function(node) { + node.x *= size[0]; + node.y = node.depth * size[1]; + delete node._tree; + } : function(node) { + node.x = (node.x - x0) / (x1 - x0) * size[0]; + node.y = node.depth / y1 * size[1]; + delete node._tree; + }); + return nodes; + } + tree.separation = function(x) { + if (!arguments.length) return separation; + separation = x; + return tree; + }; + tree.size = function(x) { + if (!arguments.length) return nodeSize ? null : size; + nodeSize = (size = x) == null; + return tree; + }; + tree.nodeSize = function(x) { + if (!arguments.length) return nodeSize ? size : null; + nodeSize = (size = x) != null; + return tree; + }; + return d3_layout_hierarchyRebind(tree, hierarchy); + }; + function d3_layout_treeSeparation(a, b) { + return a.parent == b.parent ? 1 : 2; + } + function d3_layout_treeLeft(node) { + var children = node.children; + return children && children.length ? children[0] : node._tree.thread; + } + function d3_layout_treeRight(node) { + var children = node.children, n; + return children && (n = children.length) ? children[n - 1] : node._tree.thread; + } + function d3_layout_treeSearch(node, compare) { + var children = node.children; + if (children && (n = children.length)) { + var child, n, i = -1; + while (++i < n) { + if (compare(child = d3_layout_treeSearch(children[i], compare), node) > 0) { + node = child; + } + } + } + return node; + } + function d3_layout_treeRightmost(a, b) { + return a.x - b.x; + } + function d3_layout_treeLeftmost(a, b) { + return b.x - a.x; + } + function d3_layout_treeDeepest(a, b) { + return a.depth - b.depth; + } + function d3_layout_treeVisitAfter(node, callback) { + function visit(node, previousSibling) { + var children = node.children; + if (children && (n = children.length)) { + var child, previousChild = null, i = -1, n; + while (++i < n) { + child = children[i]; + visit(child, previousChild); + previousChild = child; + } + } + callback(node, previousSibling); + } + visit(node, null); + } + function d3_layout_treeShift(node) { + var shift = 0, change = 0, children = node.children, i = children.length, child; + while (--i >= 0) { + child = children[i]._tree; + child.prelim += shift; + child.mod += shift; + shift += child.shift + (change += child.change); + } + } + function d3_layout_treeMove(ancestor, node, shift) { + ancestor = ancestor._tree; + node = node._tree; + var change = shift / (node.number - ancestor.number); + ancestor.change += change; + node.change -= change; + node.shift += shift; + node.prelim += shift; + node.mod += shift; + } + function d3_layout_treeAncestor(vim, node, ancestor) { + return vim._tree.ancestor.parent == node.parent ? vim._tree.ancestor : ancestor; + } + d3.layout.pack = function() { + var hierarchy = d3.layout.hierarchy().sort(d3_layout_packSort), padding = 0, size = [ 1, 1 ], radius; + function pack(d, i) { + var nodes = hierarchy.call(this, d, i), root = nodes[0], w = size[0], h = size[1], r = radius == null ? Math.sqrt : typeof radius === "function" ? radius : function() { + return radius; + }; + root.x = root.y = 0; + d3_layout_treeVisitAfter(root, function(d) { + d.r = +r(d.value); + }); + d3_layout_treeVisitAfter(root, d3_layout_packSiblings); + if (padding) { + var dr = padding * (radius ? 1 : Math.max(2 * root.r / w, 2 * root.r / h)) / 2; + d3_layout_treeVisitAfter(root, function(d) { + d.r += dr; + }); + d3_layout_treeVisitAfter(root, d3_layout_packSiblings); + d3_layout_treeVisitAfter(root, function(d) { + d.r -= dr; + }); + } + d3_layout_packTransform(root, w / 2, h / 2, radius ? 1 : 1 / Math.max(2 * root.r / w, 2 * root.r / h)); + return nodes; + } + pack.size = function(_) { + if (!arguments.length) return size; + size = _; + return pack; + }; + pack.radius = function(_) { + if (!arguments.length) return radius; + radius = _ == null || typeof _ === "function" ? _ : +_; + return pack; + }; + pack.padding = function(_) { + if (!arguments.length) return padding; + padding = +_; + return pack; + }; + return d3_layout_hierarchyRebind(pack, hierarchy); + }; + function d3_layout_packSort(a, b) { + return a.value - b.value; + } + function d3_layout_packInsert(a, b) { + var c = a._pack_next; + a._pack_next = b; + b._pack_prev = a; + b._pack_next = c; + c._pack_prev = b; + } + function d3_layout_packSplice(a, b) { + a._pack_next = b; + b._pack_prev = a; + } + function d3_layout_packIntersects(a, b) { + var dx = b.x - a.x, dy = b.y - a.y, dr = a.r + b.r; + return .999 * dr * dr > dx * dx + dy * dy; + } + function d3_layout_packSiblings(node) { + if (!(nodes = node.children) || !(n = nodes.length)) return; + var nodes, xMin = Infinity, xMax = -Infinity, yMin = Infinity, yMax = -Infinity, a, b, c, i, j, k, n; + function bound(node) { + xMin = Math.min(node.x - node.r, xMin); + xMax = Math.max(node.x + node.r, xMax); + yMin = Math.min(node.y - node.r, yMin); + yMax = Math.max(node.y + node.r, yMax); + } + nodes.forEach(d3_layout_packLink); + a = nodes[0]; + a.x = -a.r; + a.y = 0; + bound(a); + if (n > 1) { + b = nodes[1]; + b.x = b.r; + b.y = 0; + bound(b); + if (n > 2) { + c = nodes[2]; + d3_layout_packPlace(a, b, c); + bound(c); + d3_layout_packInsert(a, c); + a._pack_prev = c; + d3_layout_packInsert(c, b); + b = a._pack_next; + for (i = 3; i < n; i++) { + d3_layout_packPlace(a, b, c = nodes[i]); + var isect = 0, s1 = 1, s2 = 1; + for (j = b._pack_next; j !== b; j = j._pack_next, s1++) { + if (d3_layout_packIntersects(j, c)) { + isect = 1; + break; + } + } + if (isect == 1) { + for (k = a._pack_prev; k !== j._pack_prev; k = k._pack_prev, s2++) { + if (d3_layout_packIntersects(k, c)) { + break; + } + } + } + if (isect) { + if (s1 < s2 || s1 == s2 && b.r < a.r) d3_layout_packSplice(a, b = j); else d3_layout_packSplice(a = k, b); + i--; + } else { + d3_layout_packInsert(a, c); + b = c; + bound(c); + } + } + } + } + var cx = (xMin + xMax) / 2, cy = (yMin + yMax) / 2, cr = 0; + for (i = 0; i < n; i++) { + c = nodes[i]; + c.x -= cx; + c.y -= cy; + cr = Math.max(cr, c.r + Math.sqrt(c.x * c.x + c.y * c.y)); + } + node.r = cr; + nodes.forEach(d3_layout_packUnlink); + } + function d3_layout_packLink(node) { + node._pack_next = node._pack_prev = node; + } + function d3_layout_packUnlink(node) { + delete node._pack_next; + delete node._pack_prev; + } + function d3_layout_packTransform(node, x, y, k) { + var children = node.children; + node.x = x += k * node.x; + node.y = y += k * node.y; + node.r *= k; + if (children) { + var i = -1, n = children.length; + while (++i < n) d3_layout_packTransform(children[i], x, y, k); + } + } + function d3_layout_packPlace(a, b, c) { + var db = a.r + c.r, dx = b.x - a.x, dy = b.y - a.y; + if (db && (dx || dy)) { + var da = b.r + c.r, dc = dx * dx + dy * dy; + da *= da; + db *= db; + var x = .5 + (db - da) / (2 * dc), y = Math.sqrt(Math.max(0, 2 * da * (db + dc) - (db -= dc) * db - da * da)) / (2 * dc); + c.x = a.x + x * dx + y * dy; + c.y = a.y + x * dy - y * dx; + } else { + c.x = a.x + db; + c.y = a.y; + } + } + d3.layout.cluster = function() { + var hierarchy = d3.layout.hierarchy().sort(null).value(null), separation = d3_layout_treeSeparation, size = [ 1, 1 ], nodeSize = false; + function cluster(d, i) { + var nodes = hierarchy.call(this, d, i), root = nodes[0], previousNode, x = 0; + d3_layout_treeVisitAfter(root, function(node) { + var children = node.children; + if (children && children.length) { + node.x = d3_layout_clusterX(children); + node.y = d3_layout_clusterY(children); + } else { + node.x = previousNode ? x += separation(node, previousNode) : 0; + node.y = 0; + previousNode = node; + } + }); + var left = d3_layout_clusterLeft(root), right = d3_layout_clusterRight(root), x0 = left.x - separation(left, right) / 2, x1 = right.x + separation(right, left) / 2; + d3_layout_treeVisitAfter(root, nodeSize ? function(node) { + node.x = (node.x - root.x) * size[0]; + node.y = (root.y - node.y) * size[1]; + } : function(node) { + node.x = (node.x - x0) / (x1 - x0) * size[0]; + node.y = (1 - (root.y ? node.y / root.y : 1)) * size[1]; + }); + return nodes; + } + cluster.separation = function(x) { + if (!arguments.length) return separation; + separation = x; + return cluster; + }; + cluster.size = function(x) { + if (!arguments.length) return nodeSize ? null : size; + nodeSize = (size = x) == null; + return cluster; + }; + cluster.nodeSize = function(x) { + if (!arguments.length) return nodeSize ? size : null; + nodeSize = (size = x) != null; + return cluster; + }; + return d3_layout_hierarchyRebind(cluster, hierarchy); + }; + function d3_layout_clusterY(children) { + return 1 + d3.max(children, function(child) { + return child.y; + }); + } + function d3_layout_clusterX(children) { + return children.reduce(function(x, child) { + return x + child.x; + }, 0) / children.length; + } + function d3_layout_clusterLeft(node) { + var children = node.children; + return children && children.length ? d3_layout_clusterLeft(children[0]) : node; + } + function d3_layout_clusterRight(node) { + var children = node.children, n; + return children && (n = children.length) ? d3_layout_clusterRight(children[n - 1]) : node; + } + d3.layout.treemap = function() { + var hierarchy = d3.layout.hierarchy(), round = Math.round, size = [ 1, 1 ], padding = null, pad = d3_layout_treemapPadNull, sticky = false, stickies, mode = "squarify", ratio = .5 * (1 + Math.sqrt(5)); + function scale(children, k) { + var i = -1, n = children.length, child, area; + while (++i < n) { + area = (child = children[i]).value * (k < 0 ? 0 : k); + child.area = isNaN(area) || area <= 0 ? 0 : area; + } + } + function squarify(node) { + var children = node.children; + if (children && children.length) { + var rect = pad(node), row = [], remaining = children.slice(), child, best = Infinity, score, u = mode === "slice" ? rect.dx : mode === "dice" ? rect.dy : mode === "slice-dice" ? node.depth & 1 ? rect.dy : rect.dx : Math.min(rect.dx, rect.dy), n; + scale(remaining, rect.dx * rect.dy / node.value); + row.area = 0; + while ((n = remaining.length) > 0) { + row.push(child = remaining[n - 1]); + row.area += child.area; + if (mode !== "squarify" || (score = worst(row, u)) <= best) { + remaining.pop(); + best = score; + } else { + row.area -= row.pop().area; + position(row, u, rect, false); + u = Math.min(rect.dx, rect.dy); + row.length = row.area = 0; + best = Infinity; + } + } + if (row.length) { + position(row, u, rect, true); + row.length = row.area = 0; + } + children.forEach(squarify); + } + } + function stickify(node) { + var children = node.children; + if (children && children.length) { + var rect = pad(node), remaining = children.slice(), child, row = []; + scale(remaining, rect.dx * rect.dy / node.value); + row.area = 0; + while (child = remaining.pop()) { + row.push(child); + row.area += child.area; + if (child.z != null) { + position(row, child.z ? rect.dx : rect.dy, rect, !remaining.length); + row.length = row.area = 0; + } + } + children.forEach(stickify); + } + } + function worst(row, u) { + var s = row.area, r, rmax = 0, rmin = Infinity, i = -1, n = row.length; + while (++i < n) { + if (!(r = row[i].area)) continue; + if (r < rmin) rmin = r; + if (r > rmax) rmax = r; + } + s *= s; + u *= u; + return s ? Math.max(u * rmax * ratio / s, s / (u * rmin * ratio)) : Infinity; + } + function position(row, u, rect, flush) { + var i = -1, n = row.length, x = rect.x, y = rect.y, v = u ? round(row.area / u) : 0, o; + if (u == rect.dx) { + if (flush || v > rect.dy) v = rect.dy; + while (++i < n) { + o = row[i]; + o.x = x; + o.y = y; + o.dy = v; + x += o.dx = Math.min(rect.x + rect.dx - x, v ? round(o.area / v) : 0); + } + o.z = true; + o.dx += rect.x + rect.dx - x; + rect.y += v; + rect.dy -= v; + } else { + if (flush || v > rect.dx) v = rect.dx; + while (++i < n) { + o = row[i]; + o.x = x; + o.y = y; + o.dx = v; + y += o.dy = Math.min(rect.y + rect.dy - y, v ? round(o.area / v) : 0); + } + o.z = false; + o.dy += rect.y + rect.dy - y; + rect.x += v; + rect.dx -= v; + } + } + function treemap(d) { + var nodes = stickies || hierarchy(d), root = nodes[0]; + root.x = 0; + root.y = 0; + root.dx = size[0]; + root.dy = size[1]; + if (stickies) hierarchy.revalue(root); + scale([ root ], root.dx * root.dy / root.value); + (stickies ? stickify : squarify)(root); + if (sticky) stickies = nodes; + return nodes; + } + treemap.size = function(x) { + if (!arguments.length) return size; + size = x; + return treemap; + }; + treemap.padding = function(x) { + if (!arguments.length) return padding; + function padFunction(node) { + var p = x.call(treemap, node, node.depth); + return p == null ? d3_layout_treemapPadNull(node) : d3_layout_treemapPad(node, typeof p === "number" ? [ p, p, p, p ] : p); + } + function padConstant(node) { + return d3_layout_treemapPad(node, x); + } + var type; + pad = (padding = x) == null ? d3_layout_treemapPadNull : (type = typeof x) === "function" ? padFunction : type === "number" ? (x = [ x, x, x, x ], + padConstant) : padConstant; + return treemap; + }; + treemap.round = function(x) { + if (!arguments.length) return round != Number; + round = x ? Math.round : Number; + return treemap; + }; + treemap.sticky = function(x) { + if (!arguments.length) return sticky; + sticky = x; + stickies = null; + return treemap; + }; + treemap.ratio = function(x) { + if (!arguments.length) return ratio; + ratio = x; + return treemap; + }; + treemap.mode = function(x) { + if (!arguments.length) return mode; + mode = x + ""; + return treemap; + }; + return d3_layout_hierarchyRebind(treemap, hierarchy); + }; + function d3_layout_treemapPadNull(node) { + return { + x: node.x, + y: node.y, + dx: node.dx, + dy: node.dy + }; + } + function d3_layout_treemapPad(node, padding) { + var x = node.x + padding[3], y = node.y + padding[0], dx = node.dx - padding[1] - padding[3], dy = node.dy - padding[0] - padding[2]; + if (dx < 0) { + x += dx / 2; + dx = 0; + } + if (dy < 0) { + y += dy / 2; + dy = 0; + } + return { + x: x, + y: y, + dx: dx, + dy: dy + }; + } + d3.random = { + normal: function(µ, σ) { + var n = arguments.length; + if (n < 2) σ = 1; + if (n < 1) µ = 0; + return function() { + var x, y, r; + do { + x = Math.random() * 2 - 1; + y = Math.random() * 2 - 1; + r = x * x + y * y; + } while (!r || r > 1); + return µ + σ * x * Math.sqrt(-2 * Math.log(r) / r); + }; + }, + logNormal: function() { + var random = d3.random.normal.apply(d3, arguments); + return function() { + return Math.exp(random()); + }; + }, + bates: function(m) { + var random = d3.random.irwinHall(m); + return function() { + return random() / m; + }; + }, + irwinHall: function(m) { + return function() { + for (var s = 0, j = 0; j < m; j++) s += Math.random(); + return s; + }; + } + }; + d3.scale = {}; + function d3_scaleExtent(domain) { + var start = domain[0], stop = domain[domain.length - 1]; + return start < stop ? [ start, stop ] : [ stop, start ]; + } + function d3_scaleRange(scale) { + return scale.rangeExtent ? scale.rangeExtent() : d3_scaleExtent(scale.range()); + } + function d3_scale_bilinear(domain, range, uninterpolate, interpolate) { + var u = uninterpolate(domain[0], domain[1]), i = interpolate(range[0], range[1]); + return function(x) { + return i(u(x)); + }; + } + function d3_scale_nice(domain, nice) { + var i0 = 0, i1 = domain.length - 1, x0 = domain[i0], x1 = domain[i1], dx; + if (x1 < x0) { + dx = i0, i0 = i1, i1 = dx; + dx = x0, x0 = x1, x1 = dx; + } + domain[i0] = nice.floor(x0); + domain[i1] = nice.ceil(x1); + return domain; + } + function d3_scale_niceStep(step) { + return step ? { + floor: function(x) { + return Math.floor(x / step) * step; + }, + ceil: function(x) { + return Math.ceil(x / step) * step; + } + } : d3_scale_niceIdentity; + } + var d3_scale_niceIdentity = { + floor: d3_identity, + ceil: d3_identity + }; + function d3_scale_polylinear(domain, range, uninterpolate, interpolate) { + var u = [], i = [], j = 0, k = Math.min(domain.length, range.length) - 1; + if (domain[k] < domain[0]) { + domain = domain.slice().reverse(); + range = range.slice().reverse(); + } + while (++j <= k) { + u.push(uninterpolate(domain[j - 1], domain[j])); + i.push(interpolate(range[j - 1], range[j])); + } + return function(x) { + var j = d3.bisect(domain, x, 1, k) - 1; + return i[j](u[j](x)); + }; + } + d3.scale.linear = function() { + return d3_scale_linear([ 0, 1 ], [ 0, 1 ], d3_interpolate, false); + }; + function d3_scale_linear(domain, range, interpolate, clamp) { + var output, input; + function rescale() { + var linear = Math.min(domain.length, range.length) > 2 ? d3_scale_polylinear : d3_scale_bilinear, uninterpolate = clamp ? d3_uninterpolateClamp : d3_uninterpolateNumber; + output = linear(domain, range, uninterpolate, interpolate); + input = linear(range, domain, uninterpolate, d3_interpolate); + return scale; + } + function scale(x) { + return output(x); + } + scale.invert = function(y) { + return input(y); + }; + scale.domain = function(x) { + if (!arguments.length) return domain; + domain = x.map(Number); + return rescale(); + }; + scale.range = function(x) { + if (!arguments.length) return range; + range = x; + return rescale(); + }; + scale.rangeRound = function(x) { + return scale.range(x).interpolate(d3_interpolateRound); + }; + scale.clamp = function(x) { + if (!arguments.length) return clamp; + clamp = x; + return rescale(); + }; + scale.interpolate = function(x) { + if (!arguments.length) return interpolate; + interpolate = x; + return rescale(); + }; + scale.ticks = function(m) { + return d3_scale_linearTicks(domain, m); + }; + scale.tickFormat = function(m, format) { + return d3_scale_linearTickFormat(domain, m, format); + }; + scale.nice = function(m) { + d3_scale_linearNice(domain, m); + return rescale(); + }; + scale.copy = function() { + return d3_scale_linear(domain, range, interpolate, clamp); + }; + return rescale(); + } + function d3_scale_linearRebind(scale, linear) { + return d3.rebind(scale, linear, "range", "rangeRound", "interpolate", "clamp"); + } + function d3_scale_linearNice(domain, m) { + return d3_scale_nice(domain, d3_scale_niceStep(d3_scale_linearTickRange(domain, m)[2])); + } + function d3_scale_linearTickRange(domain, m) { + if (m == null) m = 10; + var extent = d3_scaleExtent(domain), span = extent[1] - extent[0], step = Math.pow(10, Math.floor(Math.log(span / m) / Math.LN10)), err = m / span * step; + if (err <= .15) step *= 10; else if (err <= .35) step *= 5; else if (err <= .75) step *= 2; + extent[0] = Math.ceil(extent[0] / step) * step; + extent[1] = Math.floor(extent[1] / step) * step + step * .5; + extent[2] = step; + return extent; + } + function d3_scale_linearTicks(domain, m) { + return d3.range.apply(d3, d3_scale_linearTickRange(domain, m)); + } + function d3_scale_linearTickFormat(domain, m, format) { + var range = d3_scale_linearTickRange(domain, m); + if (format) { + var match = d3_format_re.exec(format); + match.shift(); + if (match[8] === "s") { + var prefix = d3.formatPrefix(Math.max(abs(range[0]), abs(range[1]))); + if (!match[7]) match[7] = "." + d3_scale_linearPrecision(prefix.scale(range[2])); + match[8] = "f"; + format = d3.format(match.join("")); + return function(d) { + return format(prefix.scale(d)) + prefix.symbol; + }; + } + if (!match[7]) match[7] = "." + d3_scale_linearFormatPrecision(match[8], range); + format = match.join(""); + } else { + format = ",." + d3_scale_linearPrecision(range[2]) + "f"; + } + return d3.format(format); + } + var d3_scale_linearFormatSignificant = { + s: 1, + g: 1, + p: 1, + r: 1, + e: 1 + }; + function d3_scale_linearPrecision(value) { + return -Math.floor(Math.log(value) / Math.LN10 + .01); + } + function d3_scale_linearFormatPrecision(type, range) { + var p = d3_scale_linearPrecision(range[2]); + return type in d3_scale_linearFormatSignificant ? Math.abs(p - d3_scale_linearPrecision(Math.max(abs(range[0]), abs(range[1])))) + +(type !== "e") : p - (type === "%") * 2; + } + d3.scale.log = function() { + return d3_scale_log(d3.scale.linear().domain([ 0, 1 ]), 10, true, [ 1, 10 ]); + }; + function d3_scale_log(linear, base, positive, domain) { + function log(x) { + return (positive ? Math.log(x < 0 ? 0 : x) : -Math.log(x > 0 ? 0 : -x)) / Math.log(base); + } + function pow(x) { + return positive ? Math.pow(base, x) : -Math.pow(base, -x); + } + function scale(x) { + return linear(log(x)); + } + scale.invert = function(x) { + return pow(linear.invert(x)); + }; + scale.domain = function(x) { + if (!arguments.length) return domain; + positive = x[0] >= 0; + linear.domain((domain = x.map(Number)).map(log)); + return scale; + }; + scale.base = function(_) { + if (!arguments.length) return base; + base = +_; + linear.domain(domain.map(log)); + return scale; + }; + scale.nice = function() { + var niced = d3_scale_nice(domain.map(log), positive ? Math : d3_scale_logNiceNegative); + linear.domain(niced); + domain = niced.map(pow); + return scale; + }; + scale.ticks = function() { + var extent = d3_scaleExtent(domain), ticks = [], u = extent[0], v = extent[1], i = Math.floor(log(u)), j = Math.ceil(log(v)), n = base % 1 ? 2 : base; + if (isFinite(j - i)) { + if (positive) { + for (;i < j; i++) for (var k = 1; k < n; k++) ticks.push(pow(i) * k); + ticks.push(pow(i)); + } else { + ticks.push(pow(i)); + for (;i++ < j; ) for (var k = n - 1; k > 0; k--) ticks.push(pow(i) * k); + } + for (i = 0; ticks[i] < u; i++) {} + for (j = ticks.length; ticks[j - 1] > v; j--) {} + ticks = ticks.slice(i, j); + } + return ticks; + }; + scale.tickFormat = function(n, format) { + if (!arguments.length) return d3_scale_logFormat; + if (arguments.length < 2) format = d3_scale_logFormat; else if (typeof format !== "function") format = d3.format(format); + var k = Math.max(.1, n / scale.ticks().length), f = positive ? (e = 1e-12, Math.ceil) : (e = -1e-12, + Math.floor), e; + return function(d) { + return d / pow(f(log(d) + e)) <= k ? format(d) : ""; + }; + }; + scale.copy = function() { + return d3_scale_log(linear.copy(), base, positive, domain); + }; + return d3_scale_linearRebind(scale, linear); + } + var d3_scale_logFormat = d3.format(".0e"), d3_scale_logNiceNegative = { + floor: function(x) { + return -Math.ceil(-x); + }, + ceil: function(x) { + return -Math.floor(-x); + } + }; + d3.scale.pow = function() { + return d3_scale_pow(d3.scale.linear(), 1, [ 0, 1 ]); + }; + function d3_scale_pow(linear, exponent, domain) { + var powp = d3_scale_powPow(exponent), powb = d3_scale_powPow(1 / exponent); + function scale(x) { + return linear(powp(x)); + } + scale.invert = function(x) { + return powb(linear.invert(x)); + }; + scale.domain = function(x) { + if (!arguments.length) return domain; + linear.domain((domain = x.map(Number)).map(powp)); + return scale; + }; + scale.ticks = function(m) { + return d3_scale_linearTicks(domain, m); + }; + scale.tickFormat = function(m, format) { + return d3_scale_linearTickFormat(domain, m, format); + }; + scale.nice = function(m) { + return scale.domain(d3_scale_linearNice(domain, m)); + }; + scale.exponent = function(x) { + if (!arguments.length) return exponent; + powp = d3_scale_powPow(exponent = x); + powb = d3_scale_powPow(1 / exponent); + linear.domain(domain.map(powp)); + return scale; + }; + scale.copy = function() { + return d3_scale_pow(linear.copy(), exponent, domain); + }; + return d3_scale_linearRebind(scale, linear); + } + function d3_scale_powPow(e) { + return function(x) { + return x < 0 ? -Math.pow(-x, e) : Math.pow(x, e); + }; + } + d3.scale.sqrt = function() { + return d3.scale.pow().exponent(.5); + }; + d3.scale.ordinal = function() { + return d3_scale_ordinal([], { + t: "range", + a: [ [] ] + }); + }; + function d3_scale_ordinal(domain, ranger) { + var index, range, rangeBand; + function scale(x) { + return range[((index.get(x) || (ranger.t === "range" ? index.set(x, domain.push(x)) : NaN)) - 1) % range.length]; + } + function steps(start, step) { + return d3.range(domain.length).map(function(i) { + return start + step * i; + }); + } + scale.domain = function(x) { + if (!arguments.length) return domain; + domain = []; + index = new d3_Map(); + var i = -1, n = x.length, xi; + while (++i < n) if (!index.has(xi = x[i])) index.set(xi, domain.push(xi)); + return scale[ranger.t].apply(scale, ranger.a); + }; + scale.range = function(x) { + if (!arguments.length) return range; + range = x; + rangeBand = 0; + ranger = { + t: "range", + a: arguments + }; + return scale; + }; + scale.rangePoints = function(x, padding) { + if (arguments.length < 2) padding = 0; + var start = x[0], stop = x[1], step = (stop - start) / (Math.max(1, domain.length - 1) + padding); + range = steps(domain.length < 2 ? (start + stop) / 2 : start + step * padding / 2, step); + rangeBand = 0; + ranger = { + t: "rangePoints", + a: arguments + }; + return scale; + }; + scale.rangeBands = function(x, padding, outerPadding) { + if (arguments.length < 2) padding = 0; + if (arguments.length < 3) outerPadding = padding; + var reverse = x[1] < x[0], start = x[reverse - 0], stop = x[1 - reverse], step = (stop - start) / (domain.length - padding + 2 * outerPadding); + range = steps(start + step * outerPadding, step); + if (reverse) range.reverse(); + rangeBand = step * (1 - padding); + ranger = { + t: "rangeBands", + a: arguments + }; + return scale; + }; + scale.rangeRoundBands = function(x, padding, outerPadding) { + if (arguments.length < 2) padding = 0; + if (arguments.length < 3) outerPadding = padding; + var reverse = x[1] < x[0], start = x[reverse - 0], stop = x[1 - reverse], step = Math.floor((stop - start) / (domain.length - padding + 2 * outerPadding)), error = stop - start - (domain.length - padding) * step; + range = steps(start + Math.round(error / 2), step); + if (reverse) range.reverse(); + rangeBand = Math.round(step * (1 - padding)); + ranger = { + t: "rangeRoundBands", + a: arguments + }; + return scale; + }; + scale.rangeBand = function() { + return rangeBand; + }; + scale.rangeExtent = function() { + return d3_scaleExtent(ranger.a[0]); + }; + scale.copy = function() { + return d3_scale_ordinal(domain, ranger); + }; + return scale.domain(domain); + } + d3.scale.category10 = function() { + return d3.scale.ordinal().range(d3_category10); + }; + d3.scale.category20 = function() { + return d3.scale.ordinal().range(d3_category20); + }; + d3.scale.category20b = function() { + return d3.scale.ordinal().range(d3_category20b); + }; + d3.scale.category20c = function() { + return d3.scale.ordinal().range(d3_category20c); + }; + var d3_category10 = [ 2062260, 16744206, 2924588, 14034728, 9725885, 9197131, 14907330, 8355711, 12369186, 1556175 ].map(d3_rgbString); + var d3_category20 = [ 2062260, 11454440, 16744206, 16759672, 2924588, 10018698, 14034728, 16750742, 9725885, 12955861, 9197131, 12885140, 14907330, 16234194, 8355711, 13092807, 12369186, 14408589, 1556175, 10410725 ].map(d3_rgbString); + var d3_category20b = [ 3750777, 5395619, 7040719, 10264286, 6519097, 9216594, 11915115, 13556636, 9202993, 12426809, 15186514, 15190932, 8666169, 11356490, 14049643, 15177372, 8077683, 10834324, 13528509, 14589654 ].map(d3_rgbString); + var d3_category20c = [ 3244733, 7057110, 10406625, 13032431, 15095053, 16616764, 16625259, 16634018, 3253076, 7652470, 10607003, 13101504, 7695281, 10394312, 12369372, 14342891, 6513507, 9868950, 12434877, 14277081 ].map(d3_rgbString); + d3.scale.quantile = function() { + return d3_scale_quantile([], []); + }; + function d3_scale_quantile(domain, range) { + var thresholds; + function rescale() { + var k = 0, q = range.length; + thresholds = []; + while (++k < q) thresholds[k - 1] = d3.quantile(domain, k / q); + return scale; + } + function scale(x) { + if (!isNaN(x = +x)) return range[d3.bisect(thresholds, x)]; + } + scale.domain = function(x) { + if (!arguments.length) return domain; + domain = x.filter(function(d) { + return !isNaN(d); + }).sort(d3_ascending); + return rescale(); + }; + scale.range = function(x) { + if (!arguments.length) return range; + range = x; + return rescale(); + }; + scale.quantiles = function() { + return thresholds; + }; + scale.invertExtent = function(y) { + y = range.indexOf(y); + return y < 0 ? [ NaN, NaN ] : [ y > 0 ? thresholds[y - 1] : domain[0], y < thresholds.length ? thresholds[y] : domain[domain.length - 1] ]; + }; + scale.copy = function() { + return d3_scale_quantile(domain, range); + }; + return rescale(); + } + d3.scale.quantize = function() { + return d3_scale_quantize(0, 1, [ 0, 1 ]); + }; + function d3_scale_quantize(x0, x1, range) { + var kx, i; + function scale(x) { + return range[Math.max(0, Math.min(i, Math.floor(kx * (x - x0))))]; + } + function rescale() { + kx = range.length / (x1 - x0); + i = range.length - 1; + return scale; + } + scale.domain = function(x) { + if (!arguments.length) return [ x0, x1 ]; + x0 = +x[0]; + x1 = +x[x.length - 1]; + return rescale(); + }; + scale.range = function(x) { + if (!arguments.length) return range; + range = x; + return rescale(); + }; + scale.invertExtent = function(y) { + y = range.indexOf(y); + y = y < 0 ? NaN : y / kx + x0; + return [ y, y + 1 / kx ]; + }; + scale.copy = function() { + return d3_scale_quantize(x0, x1, range); + }; + return rescale(); + } + d3.scale.threshold = function() { + return d3_scale_threshold([ .5 ], [ 0, 1 ]); + }; + function d3_scale_threshold(domain, range) { + function scale(x) { + if (x <= x) return range[d3.bisect(domain, x)]; + } + scale.domain = function(_) { + if (!arguments.length) return domain; + domain = _; + return scale; + }; + scale.range = function(_) { + if (!arguments.length) return range; + range = _; + return scale; + }; + scale.invertExtent = function(y) { + y = range.indexOf(y); + return [ domain[y - 1], domain[y] ]; + }; + scale.copy = function() { + return d3_scale_threshold(domain, range); + }; + return scale; + } + d3.scale.identity = function() { + return d3_scale_identity([ 0, 1 ]); + }; + function d3_scale_identity(domain) { + function identity(x) { + return +x; + } + identity.invert = identity; + identity.domain = identity.range = function(x) { + if (!arguments.length) return domain; + domain = x.map(identity); + return identity; + }; + identity.ticks = function(m) { + return d3_scale_linearTicks(domain, m); + }; + identity.tickFormat = function(m, format) { + return d3_scale_linearTickFormat(domain, m, format); + }; + identity.copy = function() { + return d3_scale_identity(domain); + }; + return identity; + } + d3.svg = {}; + d3.svg.arc = function() { + var innerRadius = d3_svg_arcInnerRadius, outerRadius = d3_svg_arcOuterRadius, startAngle = d3_svg_arcStartAngle, endAngle = d3_svg_arcEndAngle; + function arc() { + var r0 = innerRadius.apply(this, arguments), r1 = outerRadius.apply(this, arguments), a0 = startAngle.apply(this, arguments) + d3_svg_arcOffset, a1 = endAngle.apply(this, arguments) + d3_svg_arcOffset, da = (a1 < a0 && (da = a0, + a0 = a1, a1 = da), a1 - a0), df = da < π ? "0" : "1", c0 = Math.cos(a0), s0 = Math.sin(a0), c1 = Math.cos(a1), s1 = Math.sin(a1); + return da >= d3_svg_arcMax ? r0 ? "M0," + r1 + "A" + r1 + "," + r1 + " 0 1,1 0," + -r1 + "A" + r1 + "," + r1 + " 0 1,1 0," + r1 + "M0," + r0 + "A" + r0 + "," + r0 + " 0 1,0 0," + -r0 + "A" + r0 + "," + r0 + " 0 1,0 0," + r0 + "Z" : "M0," + r1 + "A" + r1 + "," + r1 + " 0 1,1 0," + -r1 + "A" + r1 + "," + r1 + " 0 1,1 0," + r1 + "Z" : r0 ? "M" + r1 * c0 + "," + r1 * s0 + "A" + r1 + "," + r1 + " 0 " + df + ",1 " + r1 * c1 + "," + r1 * s1 + "L" + r0 * c1 + "," + r0 * s1 + "A" + r0 + "," + r0 + " 0 " + df + ",0 " + r0 * c0 + "," + r0 * s0 + "Z" : "M" + r1 * c0 + "," + r1 * s0 + "A" + r1 + "," + r1 + " 0 " + df + ",1 " + r1 * c1 + "," + r1 * s1 + "L0,0" + "Z"; + } + arc.innerRadius = function(v) { + if (!arguments.length) return innerRadius; + innerRadius = d3_functor(v); + return arc; + }; + arc.outerRadius = function(v) { + if (!arguments.length) return outerRadius; + outerRadius = d3_functor(v); + return arc; + }; + arc.startAngle = function(v) { + if (!arguments.length) return startAngle; + startAngle = d3_functor(v); + return arc; + }; + arc.endAngle = function(v) { + if (!arguments.length) return endAngle; + endAngle = d3_functor(v); + return arc; + }; + arc.centroid = function() { + var r = (innerRadius.apply(this, arguments) + outerRadius.apply(this, arguments)) / 2, a = (startAngle.apply(this, arguments) + endAngle.apply(this, arguments)) / 2 + d3_svg_arcOffset; + return [ Math.cos(a) * r, Math.sin(a) * r ]; + }; + return arc; + }; + var d3_svg_arcOffset = -halfπ, d3_svg_arcMax = τ - ε; + function d3_svg_arcInnerRadius(d) { + return d.innerRadius; + } + function d3_svg_arcOuterRadius(d) { + return d.outerRadius; + } + function d3_svg_arcStartAngle(d) { + return d.startAngle; + } + function d3_svg_arcEndAngle(d) { + return d.endAngle; + } + function d3_svg_line(projection) { + var x = d3_geom_pointX, y = d3_geom_pointY, defined = d3_true, interpolate = d3_svg_lineLinear, interpolateKey = interpolate.key, tension = .7; + function line(data) { + var segments = [], points = [], i = -1, n = data.length, d, fx = d3_functor(x), fy = d3_functor(y); + function segment() { + segments.push("M", interpolate(projection(points), tension)); + } + while (++i < n) { + if (defined.call(this, d = data[i], i)) { + points.push([ +fx.call(this, d, i), +fy.call(this, d, i) ]); + } else if (points.length) { + segment(); + points = []; + } + } + if (points.length) segment(); + return segments.length ? segments.join("") : null; + } + line.x = function(_) { + if (!arguments.length) return x; + x = _; + return line; + }; + line.y = function(_) { + if (!arguments.length) return y; + y = _; + return line; + }; + line.defined = function(_) { + if (!arguments.length) return defined; + defined = _; + return line; + }; + line.interpolate = function(_) { + if (!arguments.length) return interpolateKey; + if (typeof _ === "function") interpolateKey = interpolate = _; else interpolateKey = (interpolate = d3_svg_lineInterpolators.get(_) || d3_svg_lineLinear).key; + return line; + }; + line.tension = function(_) { + if (!arguments.length) return tension; + tension = _; + return line; + }; + return line; + } + d3.svg.line = function() { + return d3_svg_line(d3_identity); + }; + var d3_svg_lineInterpolators = d3.map({ + linear: d3_svg_lineLinear, + "linear-closed": d3_svg_lineLinearClosed, + step: d3_svg_lineStep, + "step-before": d3_svg_lineStepBefore, + "step-after": d3_svg_lineStepAfter, + basis: d3_svg_lineBasis, + "basis-open": d3_svg_lineBasisOpen, + "basis-closed": d3_svg_lineBasisClosed, + bundle: d3_svg_lineBundle, + cardinal: d3_svg_lineCardinal, + "cardinal-open": d3_svg_lineCardinalOpen, + "cardinal-closed": d3_svg_lineCardinalClosed, + monotone: d3_svg_lineMonotone + }); + d3_svg_lineInterpolators.forEach(function(key, value) { + value.key = key; + value.closed = /-closed$/.test(key); + }); + function d3_svg_lineLinear(points) { + return points.join("L"); + } + function d3_svg_lineLinearClosed(points) { + return d3_svg_lineLinear(points) + "Z"; + } + function d3_svg_lineStep(points) { + var i = 0, n = points.length, p = points[0], path = [ p[0], ",", p[1] ]; + while (++i < n) path.push("H", (p[0] + (p = points[i])[0]) / 2, "V", p[1]); + if (n > 1) path.push("H", p[0]); + return path.join(""); + } + function d3_svg_lineStepBefore(points) { + var i = 0, n = points.length, p = points[0], path = [ p[0], ",", p[1] ]; + while (++i < n) path.push("V", (p = points[i])[1], "H", p[0]); + return path.join(""); + } + function d3_svg_lineStepAfter(points) { + var i = 0, n = points.length, p = points[0], path = [ p[0], ",", p[1] ]; + while (++i < n) path.push("H", (p = points[i])[0], "V", p[1]); + return path.join(""); + } + function d3_svg_lineCardinalOpen(points, tension) { + return points.length < 4 ? d3_svg_lineLinear(points) : points[1] + d3_svg_lineHermite(points.slice(1, points.length - 1), d3_svg_lineCardinalTangents(points, tension)); + } + function d3_svg_lineCardinalClosed(points, tension) { + return points.length < 3 ? d3_svg_lineLinear(points) : points[0] + d3_svg_lineHermite((points.push(points[0]), + points), d3_svg_lineCardinalTangents([ points[points.length - 2] ].concat(points, [ points[1] ]), tension)); + } + function d3_svg_lineCardinal(points, tension) { + return points.length < 3 ? d3_svg_lineLinear(points) : points[0] + d3_svg_lineHermite(points, d3_svg_lineCardinalTangents(points, tension)); + } + function d3_svg_lineHermite(points, tangents) { + if (tangents.length < 1 || points.length != tangents.length && points.length != tangents.length + 2) { + return d3_svg_lineLinear(points); + } + var quad = points.length != tangents.length, path = "", p0 = points[0], p = points[1], t0 = tangents[0], t = t0, pi = 1; + if (quad) { + path += "Q" + (p[0] - t0[0] * 2 / 3) + "," + (p[1] - t0[1] * 2 / 3) + "," + p[0] + "," + p[1]; + p0 = points[1]; + pi = 2; + } + if (tangents.length > 1) { + t = tangents[1]; + p = points[pi]; + pi++; + path += "C" + (p0[0] + t0[0]) + "," + (p0[1] + t0[1]) + "," + (p[0] - t[0]) + "," + (p[1] - t[1]) + "," + p[0] + "," + p[1]; + for (var i = 2; i < tangents.length; i++, pi++) { + p = points[pi]; + t = tangents[i]; + path += "S" + (p[0] - t[0]) + "," + (p[1] - t[1]) + "," + p[0] + "," + p[1]; + } + } + if (quad) { + var lp = points[pi]; + path += "Q" + (p[0] + t[0] * 2 / 3) + "," + (p[1] + t[1] * 2 / 3) + "," + lp[0] + "," + lp[1]; + } + return path; + } + function d3_svg_lineCardinalTangents(points, tension) { + var tangents = [], a = (1 - tension) / 2, p0, p1 = points[0], p2 = points[1], i = 1, n = points.length; + while (++i < n) { + p0 = p1; + p1 = p2; + p2 = points[i]; + tangents.push([ a * (p2[0] - p0[0]), a * (p2[1] - p0[1]) ]); + } + return tangents; + } + function d3_svg_lineBasis(points) { + if (points.length < 3) return d3_svg_lineLinear(points); + var i = 1, n = points.length, pi = points[0], x0 = pi[0], y0 = pi[1], px = [ x0, x0, x0, (pi = points[1])[0] ], py = [ y0, y0, y0, pi[1] ], path = [ x0, ",", y0, "L", d3_svg_lineDot4(d3_svg_lineBasisBezier3, px), ",", d3_svg_lineDot4(d3_svg_lineBasisBezier3, py) ]; + points.push(points[n - 1]); + while (++i <= n) { + pi = points[i]; + px.shift(); + px.push(pi[0]); + py.shift(); + py.push(pi[1]); + d3_svg_lineBasisBezier(path, px, py); + } + points.pop(); + path.push("L", pi); + return path.join(""); + } + function d3_svg_lineBasisOpen(points) { + if (points.length < 4) return d3_svg_lineLinear(points); + var path = [], i = -1, n = points.length, pi, px = [ 0 ], py = [ 0 ]; + while (++i < 3) { + pi = points[i]; + px.push(pi[0]); + py.push(pi[1]); + } + path.push(d3_svg_lineDot4(d3_svg_lineBasisBezier3, px) + "," + d3_svg_lineDot4(d3_svg_lineBasisBezier3, py)); + --i; + while (++i < n) { + pi = points[i]; + px.shift(); + px.push(pi[0]); + py.shift(); + py.push(pi[1]); + d3_svg_lineBasisBezier(path, px, py); + } + return path.join(""); + } + function d3_svg_lineBasisClosed(points) { + var path, i = -1, n = points.length, m = n + 4, pi, px = [], py = []; + while (++i < 4) { + pi = points[i % n]; + px.push(pi[0]); + py.push(pi[1]); + } + path = [ d3_svg_lineDot4(d3_svg_lineBasisBezier3, px), ",", d3_svg_lineDot4(d3_svg_lineBasisBezier3, py) ]; + --i; + while (++i < m) { + pi = points[i % n]; + px.shift(); + px.push(pi[0]); + py.shift(); + py.push(pi[1]); + d3_svg_lineBasisBezier(path, px, py); + } + return path.join(""); + } + function d3_svg_lineBundle(points, tension) { + var n = points.length - 1; + if (n) { + var x0 = points[0][0], y0 = points[0][1], dx = points[n][0] - x0, dy = points[n][1] - y0, i = -1, p, t; + while (++i <= n) { + p = points[i]; + t = i / n; + p[0] = tension * p[0] + (1 - tension) * (x0 + t * dx); + p[1] = tension * p[1] + (1 - tension) * (y0 + t * dy); + } + } + return d3_svg_lineBasis(points); + } + function d3_svg_lineDot4(a, b) { + return a[0] * b[0] + a[1] * b[1] + a[2] * b[2] + a[3] * b[3]; + } + var d3_svg_lineBasisBezier1 = [ 0, 2 / 3, 1 / 3, 0 ], d3_svg_lineBasisBezier2 = [ 0, 1 / 3, 2 / 3, 0 ], d3_svg_lineBasisBezier3 = [ 0, 1 / 6, 2 / 3, 1 / 6 ]; + function d3_svg_lineBasisBezier(path, x, y) { + path.push("C", d3_svg_lineDot4(d3_svg_lineBasisBezier1, x), ",", d3_svg_lineDot4(d3_svg_lineBasisBezier1, y), ",", d3_svg_lineDot4(d3_svg_lineBasisBezier2, x), ",", d3_svg_lineDot4(d3_svg_lineBasisBezier2, y), ",", d3_svg_lineDot4(d3_svg_lineBasisBezier3, x), ",", d3_svg_lineDot4(d3_svg_lineBasisBezier3, y)); + } + function d3_svg_lineSlope(p0, p1) { + return (p1[1] - p0[1]) / (p1[0] - p0[0]); + } + function d3_svg_lineFiniteDifferences(points) { + var i = 0, j = points.length - 1, m = [], p0 = points[0], p1 = points[1], d = m[0] = d3_svg_lineSlope(p0, p1); + while (++i < j) { + m[i] = (d + (d = d3_svg_lineSlope(p0 = p1, p1 = points[i + 1]))) / 2; + } + m[i] = d; + return m; + } + function d3_svg_lineMonotoneTangents(points) { + var tangents = [], d, a, b, s, m = d3_svg_lineFiniteDifferences(points), i = -1, j = points.length - 1; + while (++i < j) { + d = d3_svg_lineSlope(points[i], points[i + 1]); + if (abs(d) < ε) { + m[i] = m[i + 1] = 0; + } else { + a = m[i] / d; + b = m[i + 1] / d; + s = a * a + b * b; + if (s > 9) { + s = d * 3 / Math.sqrt(s); + m[i] = s * a; + m[i + 1] = s * b; + } + } + } + i = -1; + while (++i <= j) { + s = (points[Math.min(j, i + 1)][0] - points[Math.max(0, i - 1)][0]) / (6 * (1 + m[i] * m[i])); + tangents.push([ s || 0, m[i] * s || 0 ]); + } + return tangents; + } + function d3_svg_lineMonotone(points) { + return points.length < 3 ? d3_svg_lineLinear(points) : points[0] + d3_svg_lineHermite(points, d3_svg_lineMonotoneTangents(points)); + } + d3.svg.line.radial = function() { + var line = d3_svg_line(d3_svg_lineRadial); + line.radius = line.x, delete line.x; + line.angle = line.y, delete line.y; + return line; + }; + function d3_svg_lineRadial(points) { + var point, i = -1, n = points.length, r, a; + while (++i < n) { + point = points[i]; + r = point[0]; + a = point[1] + d3_svg_arcOffset; + point[0] = r * Math.cos(a); + point[1] = r * Math.sin(a); + } + return points; + } + function d3_svg_area(projection) { + var x0 = d3_geom_pointX, x1 = d3_geom_pointX, y0 = 0, y1 = d3_geom_pointY, defined = d3_true, interpolate = d3_svg_lineLinear, interpolateKey = interpolate.key, interpolateReverse = interpolate, L = "L", tension = .7; + function area(data) { + var segments = [], points0 = [], points1 = [], i = -1, n = data.length, d, fx0 = d3_functor(x0), fy0 = d3_functor(y0), fx1 = x0 === x1 ? function() { + return x; + } : d3_functor(x1), fy1 = y0 === y1 ? function() { + return y; + } : d3_functor(y1), x, y; + function segment() { + segments.push("M", interpolate(projection(points1), tension), L, interpolateReverse(projection(points0.reverse()), tension), "Z"); + } + while (++i < n) { + if (defined.call(this, d = data[i], i)) { + points0.push([ x = +fx0.call(this, d, i), y = +fy0.call(this, d, i) ]); + points1.push([ +fx1.call(this, d, i), +fy1.call(this, d, i) ]); + } else if (points0.length) { + segment(); + points0 = []; + points1 = []; + } + } + if (points0.length) segment(); + return segments.length ? segments.join("") : null; + } + area.x = function(_) { + if (!arguments.length) return x1; + x0 = x1 = _; + return area; + }; + area.x0 = function(_) { + if (!arguments.length) return x0; + x0 = _; + return area; + }; + area.x1 = function(_) { + if (!arguments.length) return x1; + x1 = _; + return area; + }; + area.y = function(_) { + if (!arguments.length) return y1; + y0 = y1 = _; + return area; + }; + area.y0 = function(_) { + if (!arguments.length) return y0; + y0 = _; + return area; + }; + area.y1 = function(_) { + if (!arguments.length) return y1; + y1 = _; + return area; + }; + area.defined = function(_) { + if (!arguments.length) return defined; + defined = _; + return area; + }; + area.interpolate = function(_) { + if (!arguments.length) return interpolateKey; + if (typeof _ === "function") interpolateKey = interpolate = _; else interpolateKey = (interpolate = d3_svg_lineInterpolators.get(_) || d3_svg_lineLinear).key; + interpolateReverse = interpolate.reverse || interpolate; + L = interpolate.closed ? "M" : "L"; + return area; + }; + area.tension = function(_) { + if (!arguments.length) return tension; + tension = _; + return area; + }; + return area; + } + d3_svg_lineStepBefore.reverse = d3_svg_lineStepAfter; + d3_svg_lineStepAfter.reverse = d3_svg_lineStepBefore; + d3.svg.area = function() { + return d3_svg_area(d3_identity); + }; + d3.svg.area.radial = function() { + var area = d3_svg_area(d3_svg_lineRadial); + area.radius = area.x, delete area.x; + area.innerRadius = area.x0, delete area.x0; + area.outerRadius = area.x1, delete area.x1; + area.angle = area.y, delete area.y; + area.startAngle = area.y0, delete area.y0; + area.endAngle = area.y1, delete area.y1; + return area; + }; + d3.svg.chord = function() { + var source = d3_source, target = d3_target, radius = d3_svg_chordRadius, startAngle = d3_svg_arcStartAngle, endAngle = d3_svg_arcEndAngle; + function chord(d, i) { + var s = subgroup(this, source, d, i), t = subgroup(this, target, d, i); + return "M" + s.p0 + arc(s.r, s.p1, s.a1 - s.a0) + (equals(s, t) ? curve(s.r, s.p1, s.r, s.p0) : curve(s.r, s.p1, t.r, t.p0) + arc(t.r, t.p1, t.a1 - t.a0) + curve(t.r, t.p1, s.r, s.p0)) + "Z"; + } + function subgroup(self, f, d, i) { + var subgroup = f.call(self, d, i), r = radius.call(self, subgroup, i), a0 = startAngle.call(self, subgroup, i) + d3_svg_arcOffset, a1 = endAngle.call(self, subgroup, i) + d3_svg_arcOffset; + return { + r: r, + a0: a0, + a1: a1, + p0: [ r * Math.cos(a0), r * Math.sin(a0) ], + p1: [ r * Math.cos(a1), r * Math.sin(a1) ] + }; + } + function equals(a, b) { + return a.a0 == b.a0 && a.a1 == b.a1; + } + function arc(r, p, a) { + return "A" + r + "," + r + " 0 " + +(a > π) + ",1 " + p; + } + function curve(r0, p0, r1, p1) { + return "Q 0,0 " + p1; + } + chord.radius = function(v) { + if (!arguments.length) return radius; + radius = d3_functor(v); + return chord; + }; + chord.source = function(v) { + if (!arguments.length) return source; + source = d3_functor(v); + return chord; + }; + chord.target = function(v) { + if (!arguments.length) return target; + target = d3_functor(v); + return chord; + }; + chord.startAngle = function(v) { + if (!arguments.length) return startAngle; + startAngle = d3_functor(v); + return chord; + }; + chord.endAngle = function(v) { + if (!arguments.length) return endAngle; + endAngle = d3_functor(v); + return chord; + }; + return chord; + }; + function d3_svg_chordRadius(d) { + return d.radius; + } + d3.svg.diagonal = function() { + var source = d3_source, target = d3_target, projection = d3_svg_diagonalProjection; + function diagonal(d, i) { + var p0 = source.call(this, d, i), p3 = target.call(this, d, i), m = (p0.y + p3.y) / 2, p = [ p0, { + x: p0.x, + y: m + }, { + x: p3.x, + y: m + }, p3 ]; + p = p.map(projection); + return "M" + p[0] + "C" + p[1] + " " + p[2] + " " + p[3]; + } + diagonal.source = function(x) { + if (!arguments.length) return source; + source = d3_functor(x); + return diagonal; + }; + diagonal.target = function(x) { + if (!arguments.length) return target; + target = d3_functor(x); + return diagonal; + }; + diagonal.projection = function(x) { + if (!arguments.length) return projection; + projection = x; + return diagonal; + }; + return diagonal; + }; + function d3_svg_diagonalProjection(d) { + return [ d.x, d.y ]; + } + d3.svg.diagonal.radial = function() { + var diagonal = d3.svg.diagonal(), projection = d3_svg_diagonalProjection, projection_ = diagonal.projection; + diagonal.projection = function(x) { + return arguments.length ? projection_(d3_svg_diagonalRadialProjection(projection = x)) : projection; + }; + return diagonal; + }; + function d3_svg_diagonalRadialProjection(projection) { + return function() { + var d = projection.apply(this, arguments), r = d[0], a = d[1] + d3_svg_arcOffset; + return [ r * Math.cos(a), r * Math.sin(a) ]; + }; + } + d3.svg.symbol = function() { + var type = d3_svg_symbolType, size = d3_svg_symbolSize; + function symbol(d, i) { + return (d3_svg_symbols.get(type.call(this, d, i)) || d3_svg_symbolCircle)(size.call(this, d, i)); + } + symbol.type = function(x) { + if (!arguments.length) return type; + type = d3_functor(x); + return symbol; + }; + symbol.size = function(x) { + if (!arguments.length) return size; + size = d3_functor(x); + return symbol; + }; + return symbol; + }; + function d3_svg_symbolSize() { + return 64; + } + function d3_svg_symbolType() { + return "circle"; + } + function d3_svg_symbolCircle(size) { + var r = Math.sqrt(size / π); + return "M0," + r + "A" + r + "," + r + " 0 1,1 0," + -r + "A" + r + "," + r + " 0 1,1 0," + r + "Z"; + } + var d3_svg_symbols = d3.map({ + circle: d3_svg_symbolCircle, + cross: function(size) { + var r = Math.sqrt(size / 5) / 2; + return "M" + -3 * r + "," + -r + "H" + -r + "V" + -3 * r + "H" + r + "V" + -r + "H" + 3 * r + "V" + r + "H" + r + "V" + 3 * r + "H" + -r + "V" + r + "H" + -3 * r + "Z"; + }, + diamond: function(size) { + var ry = Math.sqrt(size / (2 * d3_svg_symbolTan30)), rx = ry * d3_svg_symbolTan30; + return "M0," + -ry + "L" + rx + ",0" + " 0," + ry + " " + -rx + ",0" + "Z"; + }, + square: function(size) { + var r = Math.sqrt(size) / 2; + return "M" + -r + "," + -r + "L" + r + "," + -r + " " + r + "," + r + " " + -r + "," + r + "Z"; + }, + "triangle-down": function(size) { + var rx = Math.sqrt(size / d3_svg_symbolSqrt3), ry = rx * d3_svg_symbolSqrt3 / 2; + return "M0," + ry + "L" + rx + "," + -ry + " " + -rx + "," + -ry + "Z"; + }, + "triangle-up": function(size) { + var rx = Math.sqrt(size / d3_svg_symbolSqrt3), ry = rx * d3_svg_symbolSqrt3 / 2; + return "M0," + -ry + "L" + rx + "," + ry + " " + -rx + "," + ry + "Z"; + } + }); + d3.svg.symbolTypes = d3_svg_symbols.keys(); + var d3_svg_symbolSqrt3 = Math.sqrt(3), d3_svg_symbolTan30 = Math.tan(30 * d3_radians); + function d3_transition(groups, id) { + d3_subclass(groups, d3_transitionPrototype); + groups.id = id; + return groups; + } + var d3_transitionPrototype = [], d3_transitionId = 0, d3_transitionInheritId, d3_transitionInherit; + d3_transitionPrototype.call = d3_selectionPrototype.call; + d3_transitionPrototype.empty = d3_selectionPrototype.empty; + d3_transitionPrototype.node = d3_selectionPrototype.node; + d3_transitionPrototype.size = d3_selectionPrototype.size; + d3.transition = function(selection) { + return arguments.length ? d3_transitionInheritId ? selection.transition() : selection : d3_selectionRoot.transition(); + }; + d3.transition.prototype = d3_transitionPrototype; + d3_transitionPrototype.select = function(selector) { + var id = this.id, subgroups = [], subgroup, subnode, node; + selector = d3_selection_selector(selector); + for (var j = -1, m = this.length; ++j < m; ) { + subgroups.push(subgroup = []); + for (var group = this[j], i = -1, n = group.length; ++i < n; ) { + if ((node = group[i]) && (subnode = selector.call(node, node.__data__, i, j))) { + if ("__data__" in node) subnode.__data__ = node.__data__; + d3_transitionNode(subnode, i, id, node.__transition__[id]); + subgroup.push(subnode); + } else { + subgroup.push(null); + } + } + } + return d3_transition(subgroups, id); + }; + d3_transitionPrototype.selectAll = function(selector) { + var id = this.id, subgroups = [], subgroup, subnodes, node, subnode, transition; + selector = d3_selection_selectorAll(selector); + for (var j = -1, m = this.length; ++j < m; ) { + for (var group = this[j], i = -1, n = group.length; ++i < n; ) { + if (node = group[i]) { + transition = node.__transition__[id]; + subnodes = selector.call(node, node.__data__, i, j); + subgroups.push(subgroup = []); + for (var k = -1, o = subnodes.length; ++k < o; ) { + if (subnode = subnodes[k]) d3_transitionNode(subnode, k, id, transition); + subgroup.push(subnode); + } + } + } + } + return d3_transition(subgroups, id); + }; + d3_transitionPrototype.filter = function(filter) { + var subgroups = [], subgroup, group, node; + if (typeof filter !== "function") filter = d3_selection_filter(filter); + for (var j = 0, m = this.length; j < m; j++) { + subgroups.push(subgroup = []); + for (var group = this[j], i = 0, n = group.length; i < n; i++) { + if ((node = group[i]) && filter.call(node, node.__data__, i, j)) { + subgroup.push(node); + } + } + } + return d3_transition(subgroups, this.id); + }; + d3_transitionPrototype.tween = function(name, tween) { + var id = this.id; + if (arguments.length < 2) return this.node().__transition__[id].tween.get(name); + return d3_selection_each(this, tween == null ? function(node) { + node.__transition__[id].tween.remove(name); + } : function(node) { + node.__transition__[id].tween.set(name, tween); + }); + }; + function d3_transition_tween(groups, name, value, tween) { + var id = groups.id; + return d3_selection_each(groups, typeof value === "function" ? function(node, i, j) { + node.__transition__[id].tween.set(name, tween(value.call(node, node.__data__, i, j))); + } : (value = tween(value), function(node) { + node.__transition__[id].tween.set(name, value); + })); + } + d3_transitionPrototype.attr = function(nameNS, value) { + if (arguments.length < 2) { + for (value in nameNS) this.attr(value, nameNS[value]); + return this; + } + var interpolate = nameNS == "transform" ? d3_interpolateTransform : d3_interpolate, name = d3.ns.qualify(nameNS); + function attrNull() { + this.removeAttribute(name); + } + function attrNullNS() { + this.removeAttributeNS(name.space, name.local); + } + function attrTween(b) { + return b == null ? attrNull : (b += "", function() { + var a = this.getAttribute(name), i; + return a !== b && (i = interpolate(a, b), function(t) { + this.setAttribute(name, i(t)); + }); + }); + } + function attrTweenNS(b) { + return b == null ? attrNullNS : (b += "", function() { + var a = this.getAttributeNS(name.space, name.local), i; + return a !== b && (i = interpolate(a, b), function(t) { + this.setAttributeNS(name.space, name.local, i(t)); + }); + }); + } + return d3_transition_tween(this, "attr." + nameNS, value, name.local ? attrTweenNS : attrTween); + }; + d3_transitionPrototype.attrTween = function(nameNS, tween) { + var name = d3.ns.qualify(nameNS); + function attrTween(d, i) { + var f = tween.call(this, d, i, this.getAttribute(name)); + return f && function(t) { + this.setAttribute(name, f(t)); + }; + } + function attrTweenNS(d, i) { + var f = tween.call(this, d, i, this.getAttributeNS(name.space, name.local)); + return f && function(t) { + this.setAttributeNS(name.space, name.local, f(t)); + }; + } + return this.tween("attr." + nameNS, name.local ? attrTweenNS : attrTween); + }; + d3_transitionPrototype.style = function(name, value, priority) { + var n = arguments.length; + if (n < 3) { + if (typeof name !== "string") { + if (n < 2) value = ""; + for (priority in name) this.style(priority, name[priority], value); + return this; + } + priority = ""; + } + function styleNull() { + this.style.removeProperty(name); + } + function styleString(b) { + return b == null ? styleNull : (b += "", function() { + var a = d3_window.getComputedStyle(this, null).getPropertyValue(name), i; + return a !== b && (i = d3_interpolate(a, b), function(t) { + this.style.setProperty(name, i(t), priority); + }); + }); + } + return d3_transition_tween(this, "style." + name, value, styleString); + }; + d3_transitionPrototype.styleTween = function(name, tween, priority) { + if (arguments.length < 3) priority = ""; + function styleTween(d, i) { + var f = tween.call(this, d, i, d3_window.getComputedStyle(this, null).getPropertyValue(name)); + return f && function(t) { + this.style.setProperty(name, f(t), priority); + }; + } + return this.tween("style." + name, styleTween); + }; + d3_transitionPrototype.text = function(value) { + return d3_transition_tween(this, "text", value, d3_transition_text); + }; + function d3_transition_text(b) { + if (b == null) b = ""; + return function() { + this.textContent = b; + }; + } + d3_transitionPrototype.remove = function() { + return this.each("end.transition", function() { + var p; + if (this.__transition__.count < 2 && (p = this.parentNode)) p.removeChild(this); + }); + }; + d3_transitionPrototype.ease = function(value) { + var id = this.id; + if (arguments.length < 1) return this.node().__transition__[id].ease; + if (typeof value !== "function") value = d3.ease.apply(d3, arguments); + return d3_selection_each(this, function(node) { + node.__transition__[id].ease = value; + }); + }; + d3_transitionPrototype.delay = function(value) { + var id = this.id; + if (arguments.length < 1) return this.node().__transition__[id].delay; + return d3_selection_each(this, typeof value === "function" ? function(node, i, j) { + node.__transition__[id].delay = +value.call(node, node.__data__, i, j); + } : (value = +value, function(node) { + node.__transition__[id].delay = value; + })); + }; + d3_transitionPrototype.duration = function(value) { + var id = this.id; + if (arguments.length < 1) return this.node().__transition__[id].duration; + return d3_selection_each(this, typeof value === "function" ? function(node, i, j) { + node.__transition__[id].duration = Math.max(1, value.call(node, node.__data__, i, j)); + } : (value = Math.max(1, value), function(node) { + node.__transition__[id].duration = value; + })); + }; + d3_transitionPrototype.each = function(type, listener) { + var id = this.id; + if (arguments.length < 2) { + var inherit = d3_transitionInherit, inheritId = d3_transitionInheritId; + d3_transitionInheritId = id; + d3_selection_each(this, function(node, i, j) { + d3_transitionInherit = node.__transition__[id]; + type.call(node, node.__data__, i, j); + }); + d3_transitionInherit = inherit; + d3_transitionInheritId = inheritId; + } else { + d3_selection_each(this, function(node) { + var transition = node.__transition__[id]; + (transition.event || (transition.event = d3.dispatch("start", "end"))).on(type, listener); + }); + } + return this; + }; + d3_transitionPrototype.transition = function() { + var id0 = this.id, id1 = ++d3_transitionId, subgroups = [], subgroup, group, node, transition; + for (var j = 0, m = this.length; j < m; j++) { + subgroups.push(subgroup = []); + for (var group = this[j], i = 0, n = group.length; i < n; i++) { + if (node = group[i]) { + transition = Object.create(node.__transition__[id0]); + transition.delay += transition.duration; + d3_transitionNode(node, i, id1, transition); + } + subgroup.push(node); + } + } + return d3_transition(subgroups, id1); + }; + function d3_transitionNode(node, i, id, inherit) { + var lock = node.__transition__ || (node.__transition__ = { + active: 0, + count: 0 + }), transition = lock[id]; + if (!transition) { + var time = inherit.time; + transition = lock[id] = { + tween: new d3_Map(), + time: time, + ease: inherit.ease, + delay: inherit.delay, + duration: inherit.duration + }; + ++lock.count; + d3.timer(function(elapsed) { + var d = node.__data__, ease = transition.ease, delay = transition.delay, duration = transition.duration, timer = d3_timer_active, tweened = []; + timer.t = delay + time; + if (delay <= elapsed) return start(elapsed - delay); + timer.c = start; + function start(elapsed) { + if (lock.active > id) return stop(); + lock.active = id; + transition.event && transition.event.start.call(node, d, i); + transition.tween.forEach(function(key, value) { + if (value = value.call(node, d, i)) { + tweened.push(value); + } + }); + d3.timer(function() { + timer.c = tick(elapsed || 1) ? d3_true : tick; + return 1; + }, 0, time); + } + function tick(elapsed) { + if (lock.active !== id) return stop(); + var t = elapsed / duration, e = ease(t), n = tweened.length; + while (n > 0) { + tweened[--n].call(node, e); + } + if (t >= 1) { + transition.event && transition.event.end.call(node, d, i); + return stop(); + } + } + function stop() { + if (--lock.count) delete lock[id]; else delete node.__transition__; + return 1; + } + }, 0, time); + } + } + d3.svg.axis = function() { + var scale = d3.scale.linear(), orient = d3_svg_axisDefaultOrient, innerTickSize = 6, outerTickSize = 6, tickPadding = 3, tickArguments_ = [ 10 ], tickValues = null, tickFormat_; + function axis(g) { + g.each(function() { + var g = d3.select(this); + var scale0 = this.__chart__ || scale, scale1 = this.__chart__ = scale.copy(); + var ticks = tickValues == null ? scale1.ticks ? scale1.ticks.apply(scale1, tickArguments_) : scale1.domain() : tickValues, tickFormat = tickFormat_ == null ? scale1.tickFormat ? scale1.tickFormat.apply(scale1, tickArguments_) : d3_identity : tickFormat_, tick = g.selectAll(".tick").data(ticks, scale1), tickEnter = tick.enter().insert("g", ".domain").attr("class", "tick").style("opacity", ε), tickExit = d3.transition(tick.exit()).style("opacity", ε).remove(), tickUpdate = d3.transition(tick.order()).style("opacity", 1), tickTransform; + var range = d3_scaleRange(scale1), path = g.selectAll(".domain").data([ 0 ]), pathUpdate = (path.enter().append("path").attr("class", "domain"), + d3.transition(path)); + tickEnter.append("line"); + tickEnter.append("text"); + var lineEnter = tickEnter.select("line"), lineUpdate = tickUpdate.select("line"), text = tick.select("text").text(tickFormat), textEnter = tickEnter.select("text"), textUpdate = tickUpdate.select("text"); + switch (orient) { + case "bottom": + { + tickTransform = d3_svg_axisX; + lineEnter.attr("y2", innerTickSize); + textEnter.attr("y", Math.max(innerTickSize, 0) + tickPadding); + lineUpdate.attr("x2", 0).attr("y2", innerTickSize); + textUpdate.attr("x", 0).attr("y", Math.max(innerTickSize, 0) + tickPadding); + text.attr("dy", ".71em").style("text-anchor", "middle"); + pathUpdate.attr("d", "M" + range[0] + "," + outerTickSize + "V0H" + range[1] + "V" + outerTickSize); + break; + } + + case "top": + { + tickTransform = d3_svg_axisX; + lineEnter.attr("y2", -innerTickSize); + textEnter.attr("y", -(Math.max(innerTickSize, 0) + tickPadding)); + lineUpdate.attr("x2", 0).attr("y2", -innerTickSize); + textUpdate.attr("x", 0).attr("y", -(Math.max(innerTickSize, 0) + tickPadding)); + text.attr("dy", "0em").style("text-anchor", "middle"); + pathUpdate.attr("d", "M" + range[0] + "," + -outerTickSize + "V0H" + range[1] + "V" + -outerTickSize); + break; + } + + case "left": + { + tickTransform = d3_svg_axisY; + lineEnter.attr("x2", -innerTickSize); + textEnter.attr("x", -(Math.max(innerTickSize, 0) + tickPadding)); + lineUpdate.attr("x2", -innerTickSize).attr("y2", 0); + textUpdate.attr("x", -(Math.max(innerTickSize, 0) + tickPadding)).attr("y", 0); + text.attr("dy", ".32em").style("text-anchor", "end"); + pathUpdate.attr("d", "M" + -outerTickSize + "," + range[0] + "H0V" + range[1] + "H" + -outerTickSize); + break; + } + + case "right": + { + tickTransform = d3_svg_axisY; + lineEnter.attr("x2", innerTickSize); + textEnter.attr("x", Math.max(innerTickSize, 0) + tickPadding); + lineUpdate.attr("x2", innerTickSize).attr("y2", 0); + textUpdate.attr("x", Math.max(innerTickSize, 0) + tickPadding).attr("y", 0); + text.attr("dy", ".32em").style("text-anchor", "start"); + pathUpdate.attr("d", "M" + outerTickSize + "," + range[0] + "H0V" + range[1] + "H" + outerTickSize); + break; + } + } + if (scale1.rangeBand) { + var x = scale1, dx = x.rangeBand() / 2; + scale0 = scale1 = function(d) { + return x(d) + dx; + }; + } else if (scale0.rangeBand) { + scale0 = scale1; + } else { + tickExit.call(tickTransform, scale1); + } + tickEnter.call(tickTransform, scale0); + tickUpdate.call(tickTransform, scale1); + }); + } + axis.scale = function(x) { + if (!arguments.length) return scale; + scale = x; + return axis; + }; + axis.orient = function(x) { + if (!arguments.length) return orient; + orient = x in d3_svg_axisOrients ? x + "" : d3_svg_axisDefaultOrient; + return axis; + }; + axis.ticks = function() { + if (!arguments.length) return tickArguments_; + tickArguments_ = arguments; + return axis; + }; + axis.tickValues = function(x) { + if (!arguments.length) return tickValues; + tickValues = x; + return axis; + }; + axis.tickFormat = function(x) { + if (!arguments.length) return tickFormat_; + tickFormat_ = x; + return axis; + }; + axis.tickSize = function(x) { + var n = arguments.length; + if (!n) return innerTickSize; + innerTickSize = +x; + outerTickSize = +arguments[n - 1]; + return axis; + }; + axis.innerTickSize = function(x) { + if (!arguments.length) return innerTickSize; + innerTickSize = +x; + return axis; + }; + axis.outerTickSize = function(x) { + if (!arguments.length) return outerTickSize; + outerTickSize = +x; + return axis; + }; + axis.tickPadding = function(x) { + if (!arguments.length) return tickPadding; + tickPadding = +x; + return axis; + }; + axis.tickSubdivide = function() { + return arguments.length && axis; + }; + return axis; + }; + var d3_svg_axisDefaultOrient = "bottom", d3_svg_axisOrients = { + top: 1, + right: 1, + bottom: 1, + left: 1 + }; + function d3_svg_axisX(selection, x) { + selection.attr("transform", function(d) { + return "translate(" + x(d) + ",0)"; + }); + } + function d3_svg_axisY(selection, y) { + selection.attr("transform", function(d) { + return "translate(0," + y(d) + ")"; + }); + } + d3.svg.brush = function() { + var event = d3_eventDispatch(brush, "brushstart", "brush", "brushend"), x = null, y = null, xExtent = [ 0, 0 ], yExtent = [ 0, 0 ], xExtentDomain, yExtentDomain, xClamp = true, yClamp = true, resizes = d3_svg_brushResizes[0]; + function brush(g) { + g.each(function() { + var g = d3.select(this).style("pointer-events", "all").style("-webkit-tap-highlight-color", "rgba(0,0,0,0)").on("mousedown.brush", brushstart).on("touchstart.brush", brushstart); + var background = g.selectAll(".background").data([ 0 ]); + background.enter().append("rect").attr("class", "background").style("visibility", "hidden").style("cursor", "crosshair"); + g.selectAll(".extent").data([ 0 ]).enter().append("rect").attr("class", "extent").style("cursor", "move"); + var resize = g.selectAll(".resize").data(resizes, d3_identity); + resize.exit().remove(); + resize.enter().append("g").attr("class", function(d) { + return "resize " + d; + }).style("cursor", function(d) { + return d3_svg_brushCursor[d]; + }).append("rect").attr("x", function(d) { + return /[ew]$/.test(d) ? -3 : null; + }).attr("y", function(d) { + return /^[ns]/.test(d) ? -3 : null; + }).attr("width", 6).attr("height", 6).style("visibility", "hidden"); + resize.style("display", brush.empty() ? "none" : null); + var gUpdate = d3.transition(g), backgroundUpdate = d3.transition(background), range; + if (x) { + range = d3_scaleRange(x); + backgroundUpdate.attr("x", range[0]).attr("width", range[1] - range[0]); + redrawX(gUpdate); + } + if (y) { + range = d3_scaleRange(y); + backgroundUpdate.attr("y", range[0]).attr("height", range[1] - range[0]); + redrawY(gUpdate); + } + redraw(gUpdate); + }); + } + brush.event = function(g) { + g.each(function() { + var event_ = event.of(this, arguments), extent1 = { + x: xExtent, + y: yExtent, + i: xExtentDomain, + j: yExtentDomain + }, extent0 = this.__chart__ || extent1; + this.__chart__ = extent1; + if (d3_transitionInheritId) { + d3.select(this).transition().each("start.brush", function() { + xExtentDomain = extent0.i; + yExtentDomain = extent0.j; + xExtent = extent0.x; + yExtent = extent0.y; + event_({ + type: "brushstart" + }); + }).tween("brush:brush", function() { + var xi = d3_interpolateArray(xExtent, extent1.x), yi = d3_interpolateArray(yExtent, extent1.y); + xExtentDomain = yExtentDomain = null; + return function(t) { + xExtent = extent1.x = xi(t); + yExtent = extent1.y = yi(t); + event_({ + type: "brush", + mode: "resize" + }); + }; + }).each("end.brush", function() { + xExtentDomain = extent1.i; + yExtentDomain = extent1.j; + event_({ + type: "brush", + mode: "resize" + }); + event_({ + type: "brushend" + }); + }); + } else { + event_({ + type: "brushstart" + }); + event_({ + type: "brush", + mode: "resize" + }); + event_({ + type: "brushend" + }); + } + }); + }; + function redraw(g) { + g.selectAll(".resize").attr("transform", function(d) { + return "translate(" + xExtent[+/e$/.test(d)] + "," + yExtent[+/^s/.test(d)] + ")"; + }); + } + function redrawX(g) { + g.select(".extent").attr("x", xExtent[0]); + g.selectAll(".extent,.n>rect,.s>rect").attr("width", xExtent[1] - xExtent[0]); + } + function redrawY(g) { + g.select(".extent").attr("y", yExtent[0]); + g.selectAll(".extent,.e>rect,.w>rect").attr("height", yExtent[1] - yExtent[0]); + } + function brushstart() { + var target = this, eventTarget = d3.select(d3.event.target), event_ = event.of(target, arguments), g = d3.select(target), resizing = eventTarget.datum(), resizingX = !/^(n|s)$/.test(resizing) && x, resizingY = !/^(e|w)$/.test(resizing) && y, dragging = eventTarget.classed("extent"), dragRestore = d3_event_dragSuppress(), center, origin = d3.mouse(target), offset; + var w = d3.select(d3_window).on("keydown.brush", keydown).on("keyup.brush", keyup); + if (d3.event.changedTouches) { + w.on("touchmove.brush", brushmove).on("touchend.brush", brushend); + } else { + w.on("mousemove.brush", brushmove).on("mouseup.brush", brushend); + } + g.interrupt().selectAll("*").interrupt(); + if (dragging) { + origin[0] = xExtent[0] - origin[0]; + origin[1] = yExtent[0] - origin[1]; + } else if (resizing) { + var ex = +/w$/.test(resizing), ey = +/^n/.test(resizing); + offset = [ xExtent[1 - ex] - origin[0], yExtent[1 - ey] - origin[1] ]; + origin[0] = xExtent[ex]; + origin[1] = yExtent[ey]; + } else if (d3.event.altKey) center = origin.slice(); + g.style("pointer-events", "none").selectAll(".resize").style("display", null); + d3.select("body").style("cursor", eventTarget.style("cursor")); + event_({ + type: "brushstart" + }); + brushmove(); + function keydown() { + if (d3.event.keyCode == 32) { + if (!dragging) { + center = null; + origin[0] -= xExtent[1]; + origin[1] -= yExtent[1]; + dragging = 2; + } + d3_eventPreventDefault(); + } + } + function keyup() { + if (d3.event.keyCode == 32 && dragging == 2) { + origin[0] += xExtent[1]; + origin[1] += yExtent[1]; + dragging = 0; + d3_eventPreventDefault(); + } + } + function brushmove() { + var point = d3.mouse(target), moved = false; + if (offset) { + point[0] += offset[0]; + point[1] += offset[1]; + } + if (!dragging) { + if (d3.event.altKey) { + if (!center) center = [ (xExtent[0] + xExtent[1]) / 2, (yExtent[0] + yExtent[1]) / 2 ]; + origin[0] = xExtent[+(point[0] < center[0])]; + origin[1] = yExtent[+(point[1] < center[1])]; + } else center = null; + } + if (resizingX && move1(point, x, 0)) { + redrawX(g); + moved = true; + } + if (resizingY && move1(point, y, 1)) { + redrawY(g); + moved = true; + } + if (moved) { + redraw(g); + event_({ + type: "brush", + mode: dragging ? "move" : "resize" + }); + } + } + function move1(point, scale, i) { + var range = d3_scaleRange(scale), r0 = range[0], r1 = range[1], position = origin[i], extent = i ? yExtent : xExtent, size = extent[1] - extent[0], min, max; + if (dragging) { + r0 -= position; + r1 -= size + position; + } + min = (i ? yClamp : xClamp) ? Math.max(r0, Math.min(r1, point[i])) : point[i]; + if (dragging) { + max = (min += position) + size; + } else { + if (center) position = Math.max(r0, Math.min(r1, 2 * center[i] - min)); + if (position < min) { + max = min; + min = position; + } else { + max = position; + } + } + if (extent[0] != min || extent[1] != max) { + if (i) yExtentDomain = null; else xExtentDomain = null; + extent[0] = min; + extent[1] = max; + return true; + } + } + function brushend() { + brushmove(); + g.style("pointer-events", "all").selectAll(".resize").style("display", brush.empty() ? "none" : null); + d3.select("body").style("cursor", null); + w.on("mousemove.brush", null).on("mouseup.brush", null).on("touchmove.brush", null).on("touchend.brush", null).on("keydown.brush", null).on("keyup.brush", null); + dragRestore(); + event_({ + type: "brushend" + }); + } + } + brush.x = function(z) { + if (!arguments.length) return x; + x = z; + resizes = d3_svg_brushResizes[!x << 1 | !y]; + return brush; + }; + brush.y = function(z) { + if (!arguments.length) return y; + y = z; + resizes = d3_svg_brushResizes[!x << 1 | !y]; + return brush; + }; + brush.clamp = function(z) { + if (!arguments.length) return x && y ? [ xClamp, yClamp ] : x ? xClamp : y ? yClamp : null; + if (x && y) xClamp = !!z[0], yClamp = !!z[1]; else if (x) xClamp = !!z; else if (y) yClamp = !!z; + return brush; + }; + brush.extent = function(z) { + var x0, x1, y0, y1, t; + if (!arguments.length) { + if (x) { + if (xExtentDomain) { + x0 = xExtentDomain[0], x1 = xExtentDomain[1]; + } else { + x0 = xExtent[0], x1 = xExtent[1]; + if (x.invert) x0 = x.invert(x0), x1 = x.invert(x1); + if (x1 < x0) t = x0, x0 = x1, x1 = t; + } + } + if (y) { + if (yExtentDomain) { + y0 = yExtentDomain[0], y1 = yExtentDomain[1]; + } else { + y0 = yExtent[0], y1 = yExtent[1]; + if (y.invert) y0 = y.invert(y0), y1 = y.invert(y1); + if (y1 < y0) t = y0, y0 = y1, y1 = t; + } + } + return x && y ? [ [ x0, y0 ], [ x1, y1 ] ] : x ? [ x0, x1 ] : y && [ y0, y1 ]; + } + if (x) { + x0 = z[0], x1 = z[1]; + if (y) x0 = x0[0], x1 = x1[0]; + xExtentDomain = [ x0, x1 ]; + if (x.invert) x0 = x(x0), x1 = x(x1); + if (x1 < x0) t = x0, x0 = x1, x1 = t; + if (x0 != xExtent[0] || x1 != xExtent[1]) xExtent = [ x0, x1 ]; + } + if (y) { + y0 = z[0], y1 = z[1]; + if (x) y0 = y0[1], y1 = y1[1]; + yExtentDomain = [ y0, y1 ]; + if (y.invert) y0 = y(y0), y1 = y(y1); + if (y1 < y0) t = y0, y0 = y1, y1 = t; + if (y0 != yExtent[0] || y1 != yExtent[1]) yExtent = [ y0, y1 ]; + } + return brush; + }; + brush.clear = function() { + if (!brush.empty()) { + xExtent = [ 0, 0 ], yExtent = [ 0, 0 ]; + xExtentDomain = yExtentDomain = null; + } + return brush; + }; + brush.empty = function() { + return !!x && xExtent[0] == xExtent[1] || !!y && yExtent[0] == yExtent[1]; + }; + return d3.rebind(brush, event, "on"); + }; + var d3_svg_brushCursor = { + n: "ns-resize", + e: "ew-resize", + s: "ns-resize", + w: "ew-resize", + nw: "nwse-resize", + ne: "nesw-resize", + se: "nwse-resize", + sw: "nesw-resize" + }; + var d3_svg_brushResizes = [ [ "n", "e", "s", "w", "nw", "ne", "se", "sw" ], [ "e", "w" ], [ "n", "s" ], [] ]; + var d3_time_format = d3_time.format = d3_locale_enUS.timeFormat; + var d3_time_formatUtc = d3_time_format.utc; + var d3_time_formatIso = d3_time_formatUtc("%Y-%m-%dT%H:%M:%S.%LZ"); + d3_time_format.iso = Date.prototype.toISOString && +new Date("2000-01-01T00:00:00.000Z") ? d3_time_formatIsoNative : d3_time_formatIso; + function d3_time_formatIsoNative(date) { + return date.toISOString(); + } + d3_time_formatIsoNative.parse = function(string) { + var date = new Date(string); + return isNaN(date) ? null : date; + }; + d3_time_formatIsoNative.toString = d3_time_formatIso.toString; + d3_time.second = d3_time_interval(function(date) { + return new d3_date(Math.floor(date / 1e3) * 1e3); + }, function(date, offset) { + date.setTime(date.getTime() + Math.floor(offset) * 1e3); + }, function(date) { + return date.getSeconds(); + }); + d3_time.seconds = d3_time.second.range; + d3_time.seconds.utc = d3_time.second.utc.range; + d3_time.minute = d3_time_interval(function(date) { + return new d3_date(Math.floor(date / 6e4) * 6e4); + }, function(date, offset) { + date.setTime(date.getTime() + Math.floor(offset) * 6e4); + }, function(date) { + return date.getMinutes(); + }); + d3_time.minutes = d3_time.minute.range; + d3_time.minutes.utc = d3_time.minute.utc.range; + d3_time.hour = d3_time_interval(function(date) { + var timezone = date.getTimezoneOffset() / 60; + return new d3_date((Math.floor(date / 36e5 - timezone) + timezone) * 36e5); + }, function(date, offset) { + date.setTime(date.getTime() + Math.floor(offset) * 36e5); + }, function(date) { + return date.getHours(); + }); + d3_time.hours = d3_time.hour.range; + d3_time.hours.utc = d3_time.hour.utc.range; + d3_time.month = d3_time_interval(function(date) { + date = d3_time.day(date); + date.setDate(1); + return date; + }, function(date, offset) { + date.setMonth(date.getMonth() + offset); + }, function(date) { + return date.getMonth(); + }); + d3_time.months = d3_time.month.range; + d3_time.months.utc = d3_time.month.utc.range; + function d3_time_scale(linear, methods, format) { + function scale(x) { + return linear(x); + } + scale.invert = function(x) { + return d3_time_scaleDate(linear.invert(x)); + }; + scale.domain = function(x) { + if (!arguments.length) return linear.domain().map(d3_time_scaleDate); + linear.domain(x); + return scale; + }; + function tickMethod(extent, count) { + var span = extent[1] - extent[0], target = span / count, i = d3.bisect(d3_time_scaleSteps, target); + return i == d3_time_scaleSteps.length ? [ methods.year, d3_scale_linearTickRange(extent.map(function(d) { + return d / 31536e6; + }), count)[2] ] : !i ? [ d3_time_scaleMilliseconds, d3_scale_linearTickRange(extent, count)[2] ] : methods[target / d3_time_scaleSteps[i - 1] < d3_time_scaleSteps[i] / target ? i - 1 : i]; + } + scale.nice = function(interval, skip) { + var domain = scale.domain(), extent = d3_scaleExtent(domain), method = interval == null ? tickMethod(extent, 10) : typeof interval === "number" && tickMethod(extent, interval); + if (method) interval = method[0], skip = method[1]; + function skipped(date) { + return !isNaN(date) && !interval.range(date, d3_time_scaleDate(+date + 1), skip).length; + } + return scale.domain(d3_scale_nice(domain, skip > 1 ? { + floor: function(date) { + while (skipped(date = interval.floor(date))) date = d3_time_scaleDate(date - 1); + return date; + }, + ceil: function(date) { + while (skipped(date = interval.ceil(date))) date = d3_time_scaleDate(+date + 1); + return date; + } + } : interval)); + }; + scale.ticks = function(interval, skip) { + var extent = d3_scaleExtent(scale.domain()), method = interval == null ? tickMethod(extent, 10) : typeof interval === "number" ? tickMethod(extent, interval) : !interval.range && [ { + range: interval + }, skip ]; + if (method) interval = method[0], skip = method[1]; + return interval.range(extent[0], d3_time_scaleDate(+extent[1] + 1), skip < 1 ? 1 : skip); + }; + scale.tickFormat = function() { + return format; + }; + scale.copy = function() { + return d3_time_scale(linear.copy(), methods, format); + }; + return d3_scale_linearRebind(scale, linear); + } + function d3_time_scaleDate(t) { + return new Date(t); + } + var d3_time_scaleSteps = [ 1e3, 5e3, 15e3, 3e4, 6e4, 3e5, 9e5, 18e5, 36e5, 108e5, 216e5, 432e5, 864e5, 1728e5, 6048e5, 2592e6, 7776e6, 31536e6 ]; + var d3_time_scaleLocalMethods = [ [ d3_time.second, 1 ], [ d3_time.second, 5 ], [ d3_time.second, 15 ], [ d3_time.second, 30 ], [ d3_time.minute, 1 ], [ d3_time.minute, 5 ], [ d3_time.minute, 15 ], [ d3_time.minute, 30 ], [ d3_time.hour, 1 ], [ d3_time.hour, 3 ], [ d3_time.hour, 6 ], [ d3_time.hour, 12 ], [ d3_time.day, 1 ], [ d3_time.day, 2 ], [ d3_time.week, 1 ], [ d3_time.month, 1 ], [ d3_time.month, 3 ], [ d3_time.year, 1 ] ]; + var d3_time_scaleLocalFormat = d3_time_format.multi([ [ ".%L", function(d) { + return d.getMilliseconds(); + } ], [ ":%S", function(d) { + return d.getSeconds(); + } ], [ "%I:%M", function(d) { + return d.getMinutes(); + } ], [ "%I %p", function(d) { + return d.getHours(); + } ], [ "%a %d", function(d) { + return d.getDay() && d.getDate() != 1; + } ], [ "%b %d", function(d) { + return d.getDate() != 1; + } ], [ "%B", function(d) { + return d.getMonth(); + } ], [ "%Y", d3_true ] ]); + var d3_time_scaleMilliseconds = { + range: function(start, stop, step) { + return d3.range(Math.ceil(start / step) * step, +stop, step).map(d3_time_scaleDate); + }, + floor: d3_identity, + ceil: d3_identity + }; + d3_time_scaleLocalMethods.year = d3_time.year; + d3_time.scale = function() { + return d3_time_scale(d3.scale.linear(), d3_time_scaleLocalMethods, d3_time_scaleLocalFormat); + }; + var d3_time_scaleUtcMethods = d3_time_scaleLocalMethods.map(function(m) { + return [ m[0].utc, m[1] ]; + }); + var d3_time_scaleUtcFormat = d3_time_formatUtc.multi([ [ ".%L", function(d) { + return d.getUTCMilliseconds(); + } ], [ ":%S", function(d) { + return d.getUTCSeconds(); + } ], [ "%I:%M", function(d) { + return d.getUTCMinutes(); + } ], [ "%I %p", function(d) { + return d.getUTCHours(); + } ], [ "%a %d", function(d) { + return d.getUTCDay() && d.getUTCDate() != 1; + } ], [ "%b %d", function(d) { + return d.getUTCDate() != 1; + } ], [ "%B", function(d) { + return d.getUTCMonth(); + } ], [ "%Y", d3_true ] ]); + d3_time_scaleUtcMethods.year = d3_time.year.utc; + d3_time.scale.utc = function() { + return d3_time_scale(d3.scale.linear(), d3_time_scaleUtcMethods, d3_time_scaleUtcFormat); + }; + d3.text = d3_xhrType(function(request) { + return request.responseText; + }); + d3.json = function(url, callback) { + return d3_xhr(url, "application/json", d3_json, callback); + }; + function d3_json(request) { + return JSON.parse(request.responseText); + } + d3.html = function(url, callback) { + return d3_xhr(url, "text/html", d3_html, callback); + }; + function d3_html(request) { + var range = d3_document.createRange(); + range.selectNode(d3_document.body); + return range.createContextualFragment(request.responseText); + } + d3.xml = d3_xhrType(function(request) { + return request.responseXML; + }); + if (typeof define === "function" && define.amd) { + define(d3); + } else if (typeof module === "object" && module.exports) { + module.exports = d3; + } else { + this.d3 = d3; + } +}(); +},{}],6:[function(require,module,exports){ +(function (global){ +;__browserify_shim_require__=require;(function browserifyShim(module, exports, require, define, browserify_shim__define__module__export__) { +/*! jQuery v2.1.1 | (c) 2005, 2014 jQuery Foundation, Inc. | jquery.org/license */ +!function(a,b){"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){var c=[],d=c.slice,e=c.concat,f=c.push,g=c.indexOf,h={},i=h.toString,j=h.hasOwnProperty,k={},l=a.document,m="2.1.1",n=function(a,b){return new n.fn.init(a,b)},o=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,p=/^-ms-/,q=/-([\da-z])/gi,r=function(a,b){return b.toUpperCase()};n.fn=n.prototype={jquery:m,constructor:n,selector:"",length:0,toArray:function(){return d.call(this)},get:function(a){return null!=a?0>a?this[a+this.length]:this[a]:d.call(this)},pushStack:function(a){var b=n.merge(this.constructor(),a);return b.prevObject=this,b.context=this.context,b},each:function(a,b){return n.each(this,a,b)},map:function(a){return this.pushStack(n.map(this,function(b,c){return a.call(b,c,b)}))},slice:function(){return this.pushStack(d.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(a){var b=this.length,c=+a+(0>a?b:0);return this.pushStack(c>=0&&b>c?[this[c]]:[])},end:function(){return this.prevObject||this.constructor(null)},push:f,sort:c.sort,splice:c.splice},n.extend=n.fn.extend=function(){var a,b,c,d,e,f,g=arguments[0]||{},h=1,i=arguments.length,j=!1;for("boolean"==typeof g&&(j=g,g=arguments[h]||{},h++),"object"==typeof g||n.isFunction(g)||(g={}),h===i&&(g=this,h--);i>h;h++)if(null!=(a=arguments[h]))for(b in a)c=g[b],d=a[b],g!==d&&(j&&d&&(n.isPlainObject(d)||(e=n.isArray(d)))?(e?(e=!1,f=c&&n.isArray(c)?c:[]):f=c&&n.isPlainObject(c)?c:{},g[b]=n.extend(j,f,d)):void 0!==d&&(g[b]=d));return g},n.extend({expando:"jQuery"+(m+Math.random()).replace(/\D/g,""),isReady:!0,error:function(a){throw new Error(a)},noop:function(){},isFunction:function(a){return"function"===n.type(a)},isArray:Array.isArray,isWindow:function(a){return null!=a&&a===a.window},isNumeric:function(a){return!n.isArray(a)&&a-parseFloat(a)>=0},isPlainObject:function(a){return"object"!==n.type(a)||a.nodeType||n.isWindow(a)?!1:a.constructor&&!j.call(a.constructor.prototype,"isPrototypeOf")?!1:!0},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},type:function(a){return null==a?a+"":"object"==typeof a||"function"==typeof a?h[i.call(a)]||"object":typeof a},globalEval:function(a){var b,c=eval;a=n.trim(a),a&&(1===a.indexOf("use strict")?(b=l.createElement("script"),b.text=a,l.head.appendChild(b).parentNode.removeChild(b)):c(a))},camelCase:function(a){return a.replace(p,"ms-").replace(q,r)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()},each:function(a,b,c){var d,e=0,f=a.length,g=s(a);if(c){if(g){for(;f>e;e++)if(d=b.apply(a[e],c),d===!1)break}else for(e in a)if(d=b.apply(a[e],c),d===!1)break}else if(g){for(;f>e;e++)if(d=b.call(a[e],e,a[e]),d===!1)break}else for(e in a)if(d=b.call(a[e],e,a[e]),d===!1)break;return a},trim:function(a){return null==a?"":(a+"").replace(o,"")},makeArray:function(a,b){var c=b||[];return null!=a&&(s(Object(a))?n.merge(c,"string"==typeof a?[a]:a):f.call(c,a)),c},inArray:function(a,b,c){return null==b?-1:g.call(b,a,c)},merge:function(a,b){for(var c=+b.length,d=0,e=a.length;c>d;d++)a[e++]=b[d];return a.length=e,a},grep:function(a,b,c){for(var d,e=[],f=0,g=a.length,h=!c;g>f;f++)d=!b(a[f],f),d!==h&&e.push(a[f]);return e},map:function(a,b,c){var d,f=0,g=a.length,h=s(a),i=[];if(h)for(;g>f;f++)d=b(a[f],f,c),null!=d&&i.push(d);else for(f in a)d=b(a[f],f,c),null!=d&&i.push(d);return e.apply([],i)},guid:1,proxy:function(a,b){var c,e,f;return"string"==typeof b&&(c=a[b],b=a,a=c),n.isFunction(a)?(e=d.call(arguments,2),f=function(){return a.apply(b||this,e.concat(d.call(arguments)))},f.guid=a.guid=a.guid||n.guid++,f):void 0},now:Date.now,support:k}),n.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(a,b){h["[object "+b+"]"]=b.toLowerCase()});function s(a){var b=a.length,c=n.type(a);return"function"===c||n.isWindow(a)?!1:1===a.nodeType&&b?!0:"array"===c||0===b||"number"==typeof b&&b>0&&b-1 in a}var t=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u="sizzle"+-new Date,v=a.document,w=0,x=0,y=gb(),z=gb(),A=gb(),B=function(a,b){return a===b&&(l=!0),0},C="undefined",D=1<<31,E={}.hasOwnProperty,F=[],G=F.pop,H=F.push,I=F.push,J=F.slice,K=F.indexOf||function(a){for(var b=0,c=this.length;c>b;b++)if(this[b]===a)return b;return-1},L="checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",M="[\\x20\\t\\r\\n\\f]",N="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",O=N.replace("w","w#"),P="\\["+M+"*("+N+")(?:"+M+"*([*^$|!~]?=)"+M+"*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|("+O+"))|)"+M+"*\\]",Q=":("+N+")(?:\\((('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|((?:\\\\.|[^\\\\()[\\]]|"+P+")*)|.*)\\)|)",R=new RegExp("^"+M+"+|((?:^|[^\\\\])(?:\\\\.)*)"+M+"+$","g"),S=new RegExp("^"+M+"*,"+M+"*"),T=new RegExp("^"+M+"*([>+~]|"+M+")"+M+"*"),U=new RegExp("="+M+"*([^\\]'\"]*?)"+M+"*\\]","g"),V=new RegExp(Q),W=new RegExp("^"+O+"$"),X={ID:new RegExp("^#("+N+")"),CLASS:new RegExp("^\\.("+N+")"),TAG:new RegExp("^("+N.replace("w","w*")+")"),ATTR:new RegExp("^"+P),PSEUDO:new RegExp("^"+Q),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+M+"*(even|odd|(([+-]|)(\\d*)n|)"+M+"*(?:([+-]|)"+M+"*(\\d+)|))"+M+"*\\)|)","i"),bool:new RegExp("^(?:"+L+")$","i"),needsContext:new RegExp("^"+M+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+M+"*((?:-\\d)?\\d*)"+M+"*\\)|)(?=[^-]|$)","i")},Y=/^(?:input|select|textarea|button)$/i,Z=/^h\d$/i,$=/^[^{]+\{\s*\[native \w/,_=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,ab=/[+~]/,bb=/'|\\/g,cb=new RegExp("\\\\([\\da-f]{1,6}"+M+"?|("+M+")|.)","ig"),db=function(a,b,c){var d="0x"+b-65536;return d!==d||c?b:0>d?String.fromCharCode(d+65536):String.fromCharCode(d>>10|55296,1023&d|56320)};try{I.apply(F=J.call(v.childNodes),v.childNodes),F[v.childNodes.length].nodeType}catch(eb){I={apply:F.length?function(a,b){H.apply(a,J.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function fb(a,b,d,e){var f,h,j,k,l,o,r,s,w,x;if((b?b.ownerDocument||b:v)!==n&&m(b),b=b||n,d=d||[],!a||"string"!=typeof a)return d;if(1!==(k=b.nodeType)&&9!==k)return[];if(p&&!e){if(f=_.exec(a))if(j=f[1]){if(9===k){if(h=b.getElementById(j),!h||!h.parentNode)return d;if(h.id===j)return d.push(h),d}else if(b.ownerDocument&&(h=b.ownerDocument.getElementById(j))&&t(b,h)&&h.id===j)return d.push(h),d}else{if(f[2])return I.apply(d,b.getElementsByTagName(a)),d;if((j=f[3])&&c.getElementsByClassName&&b.getElementsByClassName)return I.apply(d,b.getElementsByClassName(j)),d}if(c.qsa&&(!q||!q.test(a))){if(s=r=u,w=b,x=9===k&&a,1===k&&"object"!==b.nodeName.toLowerCase()){o=g(a),(r=b.getAttribute("id"))?s=r.replace(bb,"\\$&"):b.setAttribute("id",s),s="[id='"+s+"'] ",l=o.length;while(l--)o[l]=s+qb(o[l]);w=ab.test(a)&&ob(b.parentNode)||b,x=o.join(",")}if(x)try{return I.apply(d,w.querySelectorAll(x)),d}catch(y){}finally{r||b.removeAttribute("id")}}}return i(a.replace(R,"$1"),b,d,e)}function gb(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function hb(a){return a[u]=!0,a}function ib(a){var b=n.createElement("div");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function jb(a,b){var c=a.split("|"),e=a.length;while(e--)d.attrHandle[c[e]]=b}function kb(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&(~b.sourceIndex||D)-(~a.sourceIndex||D);if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function lb(a){return function(b){var c=b.nodeName.toLowerCase();return"input"===c&&b.type===a}}function mb(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}function nb(a){return hb(function(b){return b=+b,hb(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function ob(a){return a&&typeof a.getElementsByTagName!==C&&a}c=fb.support={},f=fb.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?"HTML"!==b.nodeName:!1},m=fb.setDocument=function(a){var b,e=a?a.ownerDocument||a:v,g=e.defaultView;return e!==n&&9===e.nodeType&&e.documentElement?(n=e,o=e.documentElement,p=!f(e),g&&g!==g.top&&(g.addEventListener?g.addEventListener("unload",function(){m()},!1):g.attachEvent&&g.attachEvent("onunload",function(){m()})),c.attributes=ib(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=ib(function(a){return a.appendChild(e.createComment("")),!a.getElementsByTagName("*").length}),c.getElementsByClassName=$.test(e.getElementsByClassName)&&ib(function(a){return a.innerHTML="
    ",a.firstChild.className="i",2===a.getElementsByClassName("i").length}),c.getById=ib(function(a){return o.appendChild(a).id=u,!e.getElementsByName||!e.getElementsByName(u).length}),c.getById?(d.find.ID=function(a,b){if(typeof b.getElementById!==C&&p){var c=b.getElementById(a);return c&&c.parentNode?[c]:[]}},d.filter.ID=function(a){var b=a.replace(cb,db);return function(a){return a.getAttribute("id")===b}}):(delete d.find.ID,d.filter.ID=function(a){var b=a.replace(cb,db);return function(a){var c=typeof a.getAttributeNode!==C&&a.getAttributeNode("id");return c&&c.value===b}}),d.find.TAG=c.getElementsByTagName?function(a,b){return typeof b.getElementsByTagName!==C?b.getElementsByTagName(a):void 0}:function(a,b){var c,d=[],e=0,f=b.getElementsByTagName(a);if("*"===a){while(c=f[e++])1===c.nodeType&&d.push(c);return d}return f},d.find.CLASS=c.getElementsByClassName&&function(a,b){return typeof b.getElementsByClassName!==C&&p?b.getElementsByClassName(a):void 0},r=[],q=[],(c.qsa=$.test(e.querySelectorAll))&&(ib(function(a){a.innerHTML="",a.querySelectorAll("[msallowclip^='']").length&&q.push("[*^$]="+M+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||q.push("\\["+M+"*(?:value|"+L+")"),a.querySelectorAll(":checked").length||q.push(":checked")}),ib(function(a){var b=e.createElement("input");b.setAttribute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&q.push("name"+M+"*[*^$|!~]?="),a.querySelectorAll(":enabled").length||q.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),q.push(",.*:")})),(c.matchesSelector=$.test(s=o.matches||o.webkitMatchesSelector||o.mozMatchesSelector||o.oMatchesSelector||o.msMatchesSelector))&&ib(function(a){c.disconnectedMatch=s.call(a,"div"),s.call(a,"[s!='']:x"),r.push("!=",Q)}),q=q.length&&new RegExp(q.join("|")),r=r.length&&new RegExp(r.join("|")),b=$.test(o.compareDocumentPosition),t=b||$.test(o.contains)?function(a,b){var c=9===a.nodeType?a.documentElement:a,d=b&&b.parentNode;return a===d||!(!d||1!==d.nodeType||!(c.contains?c.contains(d):a.compareDocumentPosition&&16&a.compareDocumentPosition(d)))}:function(a,b){if(b)while(b=b.parentNode)if(b===a)return!0;return!1},B=b?function(a,b){if(a===b)return l=!0,0;var d=!a.compareDocumentPosition-!b.compareDocumentPosition;return d?d:(d=(a.ownerDocument||a)===(b.ownerDocument||b)?a.compareDocumentPosition(b):1,1&d||!c.sortDetached&&b.compareDocumentPosition(a)===d?a===e||a.ownerDocument===v&&t(v,a)?-1:b===e||b.ownerDocument===v&&t(v,b)?1:k?K.call(k,a)-K.call(k,b):0:4&d?-1:1)}:function(a,b){if(a===b)return l=!0,0;var c,d=0,f=a.parentNode,g=b.parentNode,h=[a],i=[b];if(!f||!g)return a===e?-1:b===e?1:f?-1:g?1:k?K.call(k,a)-K.call(k,b):0;if(f===g)return kb(a,b);c=a;while(c=c.parentNode)h.unshift(c);c=b;while(c=c.parentNode)i.unshift(c);while(h[d]===i[d])d++;return d?kb(h[d],i[d]):h[d]===v?-1:i[d]===v?1:0},e):n},fb.matches=function(a,b){return fb(a,null,null,b)},fb.matchesSelector=function(a,b){if((a.ownerDocument||a)!==n&&m(a),b=b.replace(U,"='$1']"),!(!c.matchesSelector||!p||r&&r.test(b)||q&&q.test(b)))try{var d=s.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return fb(b,n,null,[a]).length>0},fb.contains=function(a,b){return(a.ownerDocument||a)!==n&&m(a),t(a,b)},fb.attr=function(a,b){(a.ownerDocument||a)!==n&&m(a);var e=d.attrHandle[b.toLowerCase()],f=e&&E.call(d.attrHandle,b.toLowerCase())?e(a,b,!p):void 0;return void 0!==f?f:c.attributes||!p?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},fb.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},fb.uniqueSort=function(a){var b,d=[],e=0,f=0;if(l=!c.detectDuplicates,k=!c.sortStable&&a.slice(0),a.sort(B),l){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return k=null,a},e=fb.getText=function(a){var b,c="",d=0,f=a.nodeType;if(f){if(1===f||9===f||11===f){if("string"==typeof a.textContent)return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=e(a)}else if(3===f||4===f)return a.nodeValue}else while(b=a[d++])c+=e(b);return c},d=fb.selectors={cacheLength:50,createPseudo:hb,match:X,attrHandle:{},find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(cb,db),a[3]=(a[3]||a[4]||a[5]||"").replace(cb,db),"~="===a[2]&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),"nth"===a[1].slice(0,3)?(a[3]||fb.error(a[0]),a[4]=+(a[4]?a[5]+(a[6]||1):2*("even"===a[3]||"odd"===a[3])),a[5]=+(a[7]+a[8]||"odd"===a[3])):a[3]&&fb.error(a[0]),a},PSEUDO:function(a){var b,c=!a[6]&&a[2];return X.CHILD.test(a[0])?null:(a[3]?a[2]=a[4]||a[5]||"":c&&V.test(c)&&(b=g(c,!0))&&(b=c.indexOf(")",c.length-b)-c.length)&&(a[0]=a[0].slice(0,b),a[2]=c.slice(0,b)),a.slice(0,3))}},filter:{TAG:function(a){var b=a.replace(cb,db).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=y[a+" "];return b||(b=new RegExp("(^|"+M+")"+a+"("+M+"|$)"))&&y(a,function(a){return b.test("string"==typeof a.className&&a.className||typeof a.getAttribute!==C&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=fb.attr(d,a);return null==e?"!="===b:b?(e+="","="===b?e===c:"!="===b?e!==c:"^="===b?c&&0===e.indexOf(c):"*="===b?c&&e.indexOf(c)>-1:"$="===b?c&&e.slice(-c.length)===c:"~="===b?(" "+e+" ").indexOf(c)>-1:"|="===b?e===c||e.slice(0,c.length+1)===c+"-":!1):!0}},CHILD:function(a,b,c,d,e){var f="nth"!==a.slice(0,3),g="last"!==a.slice(-4),h="of-type"===b;return 1===d&&0===e?function(a){return!!a.parentNode}:function(b,c,i){var j,k,l,m,n,o,p=f!==g?"nextSibling":"previousSibling",q=b.parentNode,r=h&&b.nodeName.toLowerCase(),s=!i&&!h;if(q){if(f){while(p){l=b;while(l=l[p])if(h?l.nodeName.toLowerCase()===r:1===l.nodeType)return!1;o=p="only"===a&&!o&&"nextSibling"}return!0}if(o=[g?q.firstChild:q.lastChild],g&&s){k=q[u]||(q[u]={}),j=k[a]||[],n=j[0]===w&&j[1],m=j[0]===w&&j[2],l=n&&q.childNodes[n];while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if(1===l.nodeType&&++m&&l===b){k[a]=[w,n,m];break}}else if(s&&(j=(b[u]||(b[u]={}))[a])&&j[0]===w)m=j[1];else while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if((h?l.nodeName.toLowerCase()===r:1===l.nodeType)&&++m&&(s&&((l[u]||(l[u]={}))[a]=[w,m]),l===b))break;return m-=e,m===d||m%d===0&&m/d>=0}}},PSEUDO:function(a,b){var c,e=d.pseudos[a]||d.setFilters[a.toLowerCase()]||fb.error("unsupported pseudo: "+a);return e[u]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?hb(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=K.call(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:hb(function(a){var b=[],c=[],d=h(a.replace(R,"$1"));return d[u]?hb(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)(f=g[h])&&(a[h]=!(b[h]=f))}):function(a,e,f){return b[0]=a,d(b,null,f,c),!c.pop()}}),has:hb(function(a){return function(b){return fb(a,b).length>0}}),contains:hb(function(a){return function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:hb(function(a){return W.test(a||"")||fb.error("unsupported lang: "+a),a=a.replace(cb,db).toLowerCase(),function(b){var c;do if(c=p?b.lang:b.getAttribute("xml:lang")||b.getAttribute("lang"))return c=c.toLowerCase(),c===a||0===c.indexOf(a+"-");while((b=b.parentNode)&&1===b.nodeType);return!1}}),target:function(b){var c=a.location&&a.location.hash;return c&&c.slice(1)===b.id},root:function(a){return a===o},focus:function(a){return a===n.activeElement&&(!n.hasFocus||n.hasFocus())&&!!(a.type||a.href||~a.tabIndex)},enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&!!a.checked||"option"===b&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},empty:function(a){for(a=a.firstChild;a;a=a.nextSibling)if(a.nodeType<6)return!1;return!0},parent:function(a){return!d.pseudos.empty(a)},header:function(a){return Z.test(a.nodeName)},input:function(a){return Y.test(a.nodeName)},button:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&"button"===a.type||"button"===b},text:function(a){var b;return"input"===a.nodeName.toLowerCase()&&"text"===a.type&&(null==(b=a.getAttribute("type"))||"text"===b.toLowerCase())},first:nb(function(){return[0]}),last:nb(function(a,b){return[b-1]}),eq:nb(function(a,b,c){return[0>c?c+b:c]}),even:nb(function(a,b){for(var c=0;b>c;c+=2)a.push(c);return a}),odd:nb(function(a,b){for(var c=1;b>c;c+=2)a.push(c);return a}),lt:nb(function(a,b,c){for(var d=0>c?c+b:c;--d>=0;)a.push(d);return a}),gt:nb(function(a,b,c){for(var d=0>c?c+b:c;++db;b++)d+=a[b].value;return d}function rb(a,b,c){var d=b.dir,e=c&&"parentNode"===d,f=x++;return b.first?function(b,c,f){while(b=b[d])if(1===b.nodeType||e)return a(b,c,f)}:function(b,c,g){var h,i,j=[w,f];if(g){while(b=b[d])if((1===b.nodeType||e)&&a(b,c,g))return!0}else while(b=b[d])if(1===b.nodeType||e){if(i=b[u]||(b[u]={}),(h=i[d])&&h[0]===w&&h[1]===f)return j[2]=h[2];if(i[d]=j,j[2]=a(b,c,g))return!0}}}function sb(a){return a.length>1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function tb(a,b,c){for(var d=0,e=b.length;e>d;d++)fb(a,b[d],c);return c}function ub(a,b,c,d,e){for(var f,g=[],h=0,i=a.length,j=null!=b;i>h;h++)(f=a[h])&&(!c||c(f,d,e))&&(g.push(f),j&&b.push(h));return g}function vb(a,b,c,d,e,f){return d&&!d[u]&&(d=vb(d)),e&&!e[u]&&(e=vb(e,f)),hb(function(f,g,h,i){var j,k,l,m=[],n=[],o=g.length,p=f||tb(b||"*",h.nodeType?[h]:h,[]),q=!a||!f&&b?p:ub(p,m,a,h,i),r=c?e||(f?a:o||d)?[]:g:q;if(c&&c(q,r,h,i),d){j=ub(r,n),d(j,[],h,i),k=j.length;while(k--)(l=j[k])&&(r[n[k]]=!(q[n[k]]=l))}if(f){if(e||a){if(e){j=[],k=r.length;while(k--)(l=r[k])&&j.push(q[k]=l);e(null,r=[],j,i)}k=r.length;while(k--)(l=r[k])&&(j=e?K.call(f,l):m[k])>-1&&(f[j]=!(g[j]=l))}}else r=ub(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):I.apply(g,r)})}function wb(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],h=g||d.relative[" "],i=g?1:0,k=rb(function(a){return a===b},h,!0),l=rb(function(a){return K.call(b,a)>-1},h,!0),m=[function(a,c,d){return!g&&(d||c!==j)||((b=c).nodeType?k(a,c,d):l(a,c,d))}];f>i;i++)if(c=d.relative[a[i].type])m=[rb(sb(m),c)];else{if(c=d.filter[a[i].type].apply(null,a[i].matches),c[u]){for(e=++i;f>e;e++)if(d.relative[a[e].type])break;return vb(i>1&&sb(m),i>1&&qb(a.slice(0,i-1).concat({value:" "===a[i-2].type?"*":""})).replace(R,"$1"),c,e>i&&wb(a.slice(i,e)),f>e&&wb(a=a.slice(e)),f>e&&qb(a))}m.push(c)}return sb(m)}function xb(a,b){var c=b.length>0,e=a.length>0,f=function(f,g,h,i,k){var l,m,o,p=0,q="0",r=f&&[],s=[],t=j,u=f||e&&d.find.TAG("*",k),v=w+=null==t?1:Math.random()||.1,x=u.length;for(k&&(j=g!==n&&g);q!==x&&null!=(l=u[q]);q++){if(e&&l){m=0;while(o=a[m++])if(o(l,g,h)){i.push(l);break}k&&(w=v)}c&&((l=!o&&l)&&p--,f&&r.push(l))}if(p+=q,c&&q!==p){m=0;while(o=b[m++])o(r,s,g,h);if(f){if(p>0)while(q--)r[q]||s[q]||(s[q]=G.call(i));s=ub(s)}I.apply(i,s),k&&!f&&s.length>0&&p+b.length>1&&fb.uniqueSort(i)}return k&&(w=v,j=t),r};return c?hb(f):f}return h=fb.compile=function(a,b){var c,d=[],e=[],f=A[a+" "];if(!f){b||(b=g(a)),c=b.length;while(c--)f=wb(b[c]),f[u]?d.push(f):e.push(f);f=A(a,xb(e,d)),f.selector=a}return f},i=fb.select=function(a,b,e,f){var i,j,k,l,m,n="function"==typeof a&&a,o=!f&&g(a=n.selector||a);if(e=e||[],1===o.length){if(j=o[0]=o[0].slice(0),j.length>2&&"ID"===(k=j[0]).type&&c.getById&&9===b.nodeType&&p&&d.relative[j[1].type]){if(b=(d.find.ID(k.matches[0].replace(cb,db),b)||[])[0],!b)return e;n&&(b=b.parentNode),a=a.slice(j.shift().value.length)}i=X.needsContext.test(a)?0:j.length;while(i--){if(k=j[i],d.relative[l=k.type])break;if((m=d.find[l])&&(f=m(k.matches[0].replace(cb,db),ab.test(j[0].type)&&ob(b.parentNode)||b))){if(j.splice(i,1),a=f.length&&qb(j),!a)return I.apply(e,f),e;break}}}return(n||h(a,o))(f,b,!p,e,ab.test(a)&&ob(b.parentNode)||b),e},c.sortStable=u.split("").sort(B).join("")===u,c.detectDuplicates=!!l,m(),c.sortDetached=ib(function(a){return 1&a.compareDocumentPosition(n.createElement("div"))}),ib(function(a){return a.innerHTML="","#"===a.firstChild.getAttribute("href")})||jb("type|href|height|width",function(a,b,c){return c?void 0:a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&ib(function(a){return a.innerHTML="",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||jb("value",function(a,b,c){return c||"input"!==a.nodeName.toLowerCase()?void 0:a.defaultValue}),ib(function(a){return null==a.getAttribute("disabled")})||jb(L,function(a,b,c){var d;return c?void 0:a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),fb}(a);n.find=t,n.expr=t.selectors,n.expr[":"]=n.expr.pseudos,n.unique=t.uniqueSort,n.text=t.getText,n.isXMLDoc=t.isXML,n.contains=t.contains;var u=n.expr.match.needsContext,v=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,w=/^.[^:#\[\.,]*$/;function x(a,b,c){if(n.isFunction(b))return n.grep(a,function(a,d){return!!b.call(a,d,a)!==c});if(b.nodeType)return n.grep(a,function(a){return a===b!==c});if("string"==typeof b){if(w.test(b))return n.filter(b,a,c);b=n.filter(b,a)}return n.grep(a,function(a){return g.call(b,a)>=0!==c})}n.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?n.find.matchesSelector(d,a)?[d]:[]:n.find.matches(a,n.grep(b,function(a){return 1===a.nodeType}))},n.fn.extend({find:function(a){var b,c=this.length,d=[],e=this;if("string"!=typeof a)return this.pushStack(n(a).filter(function(){for(b=0;c>b;b++)if(n.contains(e[b],this))return!0}));for(b=0;c>b;b++)n.find(a,e[b],d);return d=this.pushStack(c>1?n.unique(d):d),d.selector=this.selector?this.selector+" "+a:a,d},filter:function(a){return this.pushStack(x(this,a||[],!1))},not:function(a){return this.pushStack(x(this,a||[],!0))},is:function(a){return!!x(this,"string"==typeof a&&u.test(a)?n(a):a||[],!1).length}});var y,z=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,A=n.fn.init=function(a,b){var c,d;if(!a)return this;if("string"==typeof a){if(c="<"===a[0]&&">"===a[a.length-1]&&a.length>=3?[null,a,null]:z.exec(a),!c||!c[1]&&b)return!b||b.jquery?(b||y).find(a):this.constructor(b).find(a);if(c[1]){if(b=b instanceof n?b[0]:b,n.merge(this,n.parseHTML(c[1],b&&b.nodeType?b.ownerDocument||b:l,!0)),v.test(c[1])&&n.isPlainObject(b))for(c in b)n.isFunction(this[c])?this[c](b[c]):this.attr(c,b[c]);return this}return d=l.getElementById(c[2]),d&&d.parentNode&&(this.length=1,this[0]=d),this.context=l,this.selector=a,this}return a.nodeType?(this.context=this[0]=a,this.length=1,this):n.isFunction(a)?"undefined"!=typeof y.ready?y.ready(a):a(n):(void 0!==a.selector&&(this.selector=a.selector,this.context=a.context),n.makeArray(a,this))};A.prototype=n.fn,y=n(l);var B=/^(?:parents|prev(?:Until|All))/,C={children:!0,contents:!0,next:!0,prev:!0};n.extend({dir:function(a,b,c){var d=[],e=void 0!==c;while((a=a[b])&&9!==a.nodeType)if(1===a.nodeType){if(e&&n(a).is(c))break;d.push(a)}return d},sibling:function(a,b){for(var c=[];a;a=a.nextSibling)1===a.nodeType&&a!==b&&c.push(a);return c}}),n.fn.extend({has:function(a){var b=n(a,this),c=b.length;return this.filter(function(){for(var a=0;c>a;a++)if(n.contains(this,b[a]))return!0})},closest:function(a,b){for(var c,d=0,e=this.length,f=[],g=u.test(a)||"string"!=typeof a?n(a,b||this.context):0;e>d;d++)for(c=this[d];c&&c!==b;c=c.parentNode)if(c.nodeType<11&&(g?g.index(c)>-1:1===c.nodeType&&n.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?n.unique(f):f)},index:function(a){return a?"string"==typeof a?g.call(n(a),this[0]):g.call(this,a.jquery?a[0]:a):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(n.unique(n.merge(this.get(),n(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function D(a,b){while((a=a[b])&&1!==a.nodeType);return a}n.each({parent:function(a){var b=a.parentNode;return b&&11!==b.nodeType?b:null},parents:function(a){return n.dir(a,"parentNode")},parentsUntil:function(a,b,c){return n.dir(a,"parentNode",c)},next:function(a){return D(a,"nextSibling")},prev:function(a){return D(a,"previousSibling")},nextAll:function(a){return n.dir(a,"nextSibling")},prevAll:function(a){return n.dir(a,"previousSibling")},nextUntil:function(a,b,c){return n.dir(a,"nextSibling",c)},prevUntil:function(a,b,c){return n.dir(a,"previousSibling",c)},siblings:function(a){return n.sibling((a.parentNode||{}).firstChild,a)},children:function(a){return n.sibling(a.firstChild)},contents:function(a){return a.contentDocument||n.merge([],a.childNodes)}},function(a,b){n.fn[a]=function(c,d){var e=n.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=n.filter(d,e)),this.length>1&&(C[a]||n.unique(e),B.test(a)&&e.reverse()),this.pushStack(e)}});var E=/\S+/g,F={};function G(a){var b=F[a]={};return n.each(a.match(E)||[],function(a,c){b[c]=!0}),b}n.Callbacks=function(a){a="string"==typeof a?F[a]||G(a):n.extend({},a);var b,c,d,e,f,g,h=[],i=!a.once&&[],j=function(l){for(b=a.memory&&l,c=!0,g=e||0,e=0,f=h.length,d=!0;h&&f>g;g++)if(h[g].apply(l[0],l[1])===!1&&a.stopOnFalse){b=!1;break}d=!1,h&&(i?i.length&&j(i.shift()):b?h=[]:k.disable())},k={add:function(){if(h){var c=h.length;!function g(b){n.each(b,function(b,c){var d=n.type(c);"function"===d?a.unique&&k.has(c)||h.push(c):c&&c.length&&"string"!==d&&g(c)})}(arguments),d?f=h.length:b&&(e=c,j(b))}return this},remove:function(){return h&&n.each(arguments,function(a,b){var c;while((c=n.inArray(b,h,c))>-1)h.splice(c,1),d&&(f>=c&&f--,g>=c&&g--)}),this},has:function(a){return a?n.inArray(a,h)>-1:!(!h||!h.length)},empty:function(){return h=[],f=0,this},disable:function(){return h=i=b=void 0,this},disabled:function(){return!h},lock:function(){return i=void 0,b||k.disable(),this},locked:function(){return!i},fireWith:function(a,b){return!h||c&&!i||(b=b||[],b=[a,b.slice?b.slice():b],d?i.push(b):j(b)),this},fire:function(){return k.fireWith(this,arguments),this},fired:function(){return!!c}};return k},n.extend({Deferred:function(a){var b=[["resolve","done",n.Callbacks("once memory"),"resolved"],["reject","fail",n.Callbacks("once memory"),"rejected"],["notify","progress",n.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return n.Deferred(function(c){n.each(b,function(b,f){var g=n.isFunction(a[b])&&a[b];e[f[1]](function(){var a=g&&g.apply(this,arguments);a&&n.isFunction(a.promise)?a.promise().done(c.resolve).fail(c.reject).progress(c.notify):c[f[0]+"With"](this===d?c.promise():this,g?[a]:arguments)})}),a=null}).promise()},promise:function(a){return null!=a?n.extend(a,d):d}},e={};return d.pipe=d.then,n.each(b,function(a,f){var g=f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[1^a][2].disable,b[2][2].lock),e[f[0]]=function(){return e[f[0]+"With"](this===e?d:this,arguments),this},e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=d.call(arguments),e=c.length,f=1!==e||a&&n.isFunction(a.promise)?e:0,g=1===f?a:n.Deferred(),h=function(a,b,c){return function(e){b[a]=this,c[a]=arguments.length>1?d.call(arguments):e,c===i?g.notifyWith(b,c):--f||g.resolveWith(b,c)}},i,j,k;if(e>1)for(i=new Array(e),j=new Array(e),k=new Array(e);e>b;b++)c[b]&&n.isFunction(c[b].promise)?c[b].promise().done(h(b,k,c)).fail(g.reject).progress(h(b,j,i)):--f;return f||g.resolveWith(k,c),g.promise()}});var H;n.fn.ready=function(a){return n.ready.promise().done(a),this},n.extend({isReady:!1,readyWait:1,holdReady:function(a){a?n.readyWait++:n.ready(!0)},ready:function(a){(a===!0?--n.readyWait:n.isReady)||(n.isReady=!0,a!==!0&&--n.readyWait>0||(H.resolveWith(l,[n]),n.fn.triggerHandler&&(n(l).triggerHandler("ready"),n(l).off("ready"))))}});function I(){l.removeEventListener("DOMContentLoaded",I,!1),a.removeEventListener("load",I,!1),n.ready()}n.ready.promise=function(b){return H||(H=n.Deferred(),"complete"===l.readyState?setTimeout(n.ready):(l.addEventListener("DOMContentLoaded",I,!1),a.addEventListener("load",I,!1))),H.promise(b)},n.ready.promise();var J=n.access=function(a,b,c,d,e,f,g){var h=0,i=a.length,j=null==c;if("object"===n.type(c)){e=!0;for(h in c)n.access(a,b,h,c[h],!0,f,g)}else if(void 0!==d&&(e=!0,n.isFunction(d)||(g=!0),j&&(g?(b.call(a,d),b=null):(j=b,b=function(a,b,c){return j.call(n(a),c)})),b))for(;i>h;h++)b(a[h],c,g?d:d.call(a[h],h,b(a[h],c)));return e?a:j?b.call(a):i?b(a[0],c):f};n.acceptData=function(a){return 1===a.nodeType||9===a.nodeType||!+a.nodeType};function K(){Object.defineProperty(this.cache={},0,{get:function(){return{}}}),this.expando=n.expando+Math.random()}K.uid=1,K.accepts=n.acceptData,K.prototype={key:function(a){if(!K.accepts(a))return 0;var b={},c=a[this.expando];if(!c){c=K.uid++;try{b[this.expando]={value:c},Object.defineProperties(a,b)}catch(d){b[this.expando]=c,n.extend(a,b)}}return this.cache[c]||(this.cache[c]={}),c},set:function(a,b,c){var d,e=this.key(a),f=this.cache[e];if("string"==typeof b)f[b]=c;else if(n.isEmptyObject(f))n.extend(this.cache[e],b);else for(d in b)f[d]=b[d];return f},get:function(a,b){var c=this.cache[this.key(a)];return void 0===b?c:c[b]},access:function(a,b,c){var d;return void 0===b||b&&"string"==typeof b&&void 0===c?(d=this.get(a,b),void 0!==d?d:this.get(a,n.camelCase(b))):(this.set(a,b,c),void 0!==c?c:b)},remove:function(a,b){var c,d,e,f=this.key(a),g=this.cache[f];if(void 0===b)this.cache[f]={};else{n.isArray(b)?d=b.concat(b.map(n.camelCase)):(e=n.camelCase(b),b in g?d=[b,e]:(d=e,d=d in g?[d]:d.match(E)||[])),c=d.length;while(c--)delete g[d[c]]}},hasData:function(a){return!n.isEmptyObject(this.cache[a[this.expando]]||{})},discard:function(a){a[this.expando]&&delete this.cache[a[this.expando]]}};var L=new K,M=new K,N=/^(?:\{[\w\W]*\}|\[[\w\W]*\])$/,O=/([A-Z])/g;function P(a,b,c){var d;if(void 0===c&&1===a.nodeType)if(d="data-"+b.replace(O,"-$1").toLowerCase(),c=a.getAttribute(d),"string"==typeof c){try{c="true"===c?!0:"false"===c?!1:"null"===c?null:+c+""===c?+c:N.test(c)?n.parseJSON(c):c}catch(e){}M.set(a,b,c)}else c=void 0;return c}n.extend({hasData:function(a){return M.hasData(a)||L.hasData(a)},data:function(a,b,c){return M.access(a,b,c)},removeData:function(a,b){M.remove(a,b) +},_data:function(a,b,c){return L.access(a,b,c)},_removeData:function(a,b){L.remove(a,b)}}),n.fn.extend({data:function(a,b){var c,d,e,f=this[0],g=f&&f.attributes;if(void 0===a){if(this.length&&(e=M.get(f),1===f.nodeType&&!L.get(f,"hasDataAttrs"))){c=g.length;while(c--)g[c]&&(d=g[c].name,0===d.indexOf("data-")&&(d=n.camelCase(d.slice(5)),P(f,d,e[d])));L.set(f,"hasDataAttrs",!0)}return e}return"object"==typeof a?this.each(function(){M.set(this,a)}):J(this,function(b){var c,d=n.camelCase(a);if(f&&void 0===b){if(c=M.get(f,a),void 0!==c)return c;if(c=M.get(f,d),void 0!==c)return c;if(c=P(f,d,void 0),void 0!==c)return c}else this.each(function(){var c=M.get(this,d);M.set(this,d,b),-1!==a.indexOf("-")&&void 0!==c&&M.set(this,a,b)})},null,b,arguments.length>1,null,!0)},removeData:function(a){return this.each(function(){M.remove(this,a)})}}),n.extend({queue:function(a,b,c){var d;return a?(b=(b||"fx")+"queue",d=L.get(a,b),c&&(!d||n.isArray(c)?d=L.access(a,b,n.makeArray(c)):d.push(c)),d||[]):void 0},dequeue:function(a,b){b=b||"fx";var c=n.queue(a,b),d=c.length,e=c.shift(),f=n._queueHooks(a,b),g=function(){n.dequeue(a,b)};"inprogress"===e&&(e=c.shift(),d--),e&&("fx"===b&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return L.get(a,c)||L.access(a,c,{empty:n.Callbacks("once memory").add(function(){L.remove(a,[b+"queue",c])})})}}),n.fn.extend({queue:function(a,b){var c=2;return"string"!=typeof a&&(b=a,a="fx",c--),arguments.lengthx",k.noCloneChecked=!!b.cloneNode(!0).lastChild.defaultValue}();var U="undefined";k.focusinBubbles="onfocusin"in a;var V=/^key/,W=/^(?:mouse|pointer|contextmenu)|click/,X=/^(?:focusinfocus|focusoutblur)$/,Y=/^([^.]*)(?:\.(.+)|)$/;function Z(){return!0}function $(){return!1}function _(){try{return l.activeElement}catch(a){}}n.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,o,p,q,r=L.get(a);if(r){c.handler&&(f=c,c=f.handler,e=f.selector),c.guid||(c.guid=n.guid++),(i=r.events)||(i=r.events={}),(g=r.handle)||(g=r.handle=function(b){return typeof n!==U&&n.event.triggered!==b.type?n.event.dispatch.apply(a,arguments):void 0}),b=(b||"").match(E)||[""],j=b.length;while(j--)h=Y.exec(b[j])||[],o=q=h[1],p=(h[2]||"").split(".").sort(),o&&(l=n.event.special[o]||{},o=(e?l.delegateType:l.bindType)||o,l=n.event.special[o]||{},k=n.extend({type:o,origType:q,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&n.expr.match.needsContext.test(e),namespace:p.join(".")},f),(m=i[o])||(m=i[o]=[],m.delegateCount=0,l.setup&&l.setup.call(a,d,p,g)!==!1||a.addEventListener&&a.addEventListener(o,g,!1)),l.add&&(l.add.call(a,k),k.handler.guid||(k.handler.guid=c.guid)),e?m.splice(m.delegateCount++,0,k):m.push(k),n.event.global[o]=!0)}},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,o,p,q,r=L.hasData(a)&&L.get(a);if(r&&(i=r.events)){b=(b||"").match(E)||[""],j=b.length;while(j--)if(h=Y.exec(b[j])||[],o=q=h[1],p=(h[2]||"").split(".").sort(),o){l=n.event.special[o]||{},o=(d?l.delegateType:l.bindType)||o,m=i[o]||[],h=h[2]&&new RegExp("(^|\\.)"+p.join("\\.(?:.*\\.|)")+"(\\.|$)"),g=f=m.length;while(f--)k=m[f],!e&&q!==k.origType||c&&c.guid!==k.guid||h&&!h.test(k.namespace)||d&&d!==k.selector&&("**"!==d||!k.selector)||(m.splice(f,1),k.selector&&m.delegateCount--,l.remove&&l.remove.call(a,k));g&&!m.length&&(l.teardown&&l.teardown.call(a,p,r.handle)!==!1||n.removeEvent(a,o,r.handle),delete i[o])}else for(o in i)n.event.remove(a,o+b[j],c,d,!0);n.isEmptyObject(i)&&(delete r.handle,L.remove(a,"events"))}},trigger:function(b,c,d,e){var f,g,h,i,k,m,o,p=[d||l],q=j.call(b,"type")?b.type:b,r=j.call(b,"namespace")?b.namespace.split("."):[];if(g=h=d=d||l,3!==d.nodeType&&8!==d.nodeType&&!X.test(q+n.event.triggered)&&(q.indexOf(".")>=0&&(r=q.split("."),q=r.shift(),r.sort()),k=q.indexOf(":")<0&&"on"+q,b=b[n.expando]?b:new n.Event(q,"object"==typeof b&&b),b.isTrigger=e?2:3,b.namespace=r.join("."),b.namespace_re=b.namespace?new RegExp("(^|\\.)"+r.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,b.result=void 0,b.target||(b.target=d),c=null==c?[b]:n.makeArray(c,[b]),o=n.event.special[q]||{},e||!o.trigger||o.trigger.apply(d,c)!==!1)){if(!e&&!o.noBubble&&!n.isWindow(d)){for(i=o.delegateType||q,X.test(i+q)||(g=g.parentNode);g;g=g.parentNode)p.push(g),h=g;h===(d.ownerDocument||l)&&p.push(h.defaultView||h.parentWindow||a)}f=0;while((g=p[f++])&&!b.isPropagationStopped())b.type=f>1?i:o.bindType||q,m=(L.get(g,"events")||{})[b.type]&&L.get(g,"handle"),m&&m.apply(g,c),m=k&&g[k],m&&m.apply&&n.acceptData(g)&&(b.result=m.apply(g,c),b.result===!1&&b.preventDefault());return b.type=q,e||b.isDefaultPrevented()||o._default&&o._default.apply(p.pop(),c)!==!1||!n.acceptData(d)||k&&n.isFunction(d[q])&&!n.isWindow(d)&&(h=d[k],h&&(d[k]=null),n.event.triggered=q,d[q](),n.event.triggered=void 0,h&&(d[k]=h)),b.result}},dispatch:function(a){a=n.event.fix(a);var b,c,e,f,g,h=[],i=d.call(arguments),j=(L.get(this,"events")||{})[a.type]||[],k=n.event.special[a.type]||{};if(i[0]=a,a.delegateTarget=this,!k.preDispatch||k.preDispatch.call(this,a)!==!1){h=n.event.handlers.call(this,a,j),b=0;while((f=h[b++])&&!a.isPropagationStopped()){a.currentTarget=f.elem,c=0;while((g=f.handlers[c++])&&!a.isImmediatePropagationStopped())(!a.namespace_re||a.namespace_re.test(g.namespace))&&(a.handleObj=g,a.data=g.data,e=((n.event.special[g.origType]||{}).handle||g.handler).apply(f.elem,i),void 0!==e&&(a.result=e)===!1&&(a.preventDefault(),a.stopPropagation()))}return k.postDispatch&&k.postDispatch.call(this,a),a.result}},handlers:function(a,b){var c,d,e,f,g=[],h=b.delegateCount,i=a.target;if(h&&i.nodeType&&(!a.button||"click"!==a.type))for(;i!==this;i=i.parentNode||this)if(i.disabled!==!0||"click"!==a.type){for(d=[],c=0;h>c;c++)f=b[c],e=f.selector+" ",void 0===d[e]&&(d[e]=f.needsContext?n(e,this).index(i)>=0:n.find(e,this,null,[i]).length),d[e]&&d.push(f);d.length&&g.push({elem:i,handlers:d})}return h]*)\/>/gi,bb=/<([\w:]+)/,cb=/<|&#?\w+;/,db=/<(?:script|style|link)/i,eb=/checked\s*(?:[^=]|=\s*.checked.)/i,fb=/^$|\/(?:java|ecma)script/i,gb=/^true\/(.*)/,hb=/^\s*\s*$/g,ib={option:[1,""],thead:[1,"","
    "],col:[2,"","
    "],tr:[2,"","
    "],td:[3,"","
    "],_default:[0,"",""]};ib.optgroup=ib.option,ib.tbody=ib.tfoot=ib.colgroup=ib.caption=ib.thead,ib.th=ib.td;function jb(a,b){return n.nodeName(a,"table")&&n.nodeName(11!==b.nodeType?b:b.firstChild,"tr")?a.getElementsByTagName("tbody")[0]||a.appendChild(a.ownerDocument.createElement("tbody")):a}function kb(a){return a.type=(null!==a.getAttribute("type"))+"/"+a.type,a}function lb(a){var b=gb.exec(a.type);return b?a.type=b[1]:a.removeAttribute("type"),a}function mb(a,b){for(var c=0,d=a.length;d>c;c++)L.set(a[c],"globalEval",!b||L.get(b[c],"globalEval"))}function nb(a,b){var c,d,e,f,g,h,i,j;if(1===b.nodeType){if(L.hasData(a)&&(f=L.access(a),g=L.set(b,f),j=f.events)){delete g.handle,g.events={};for(e in j)for(c=0,d=j[e].length;d>c;c++)n.event.add(b,e,j[e][c])}M.hasData(a)&&(h=M.access(a),i=n.extend({},h),M.set(b,i))}}function ob(a,b){var c=a.getElementsByTagName?a.getElementsByTagName(b||"*"):a.querySelectorAll?a.querySelectorAll(b||"*"):[];return void 0===b||b&&n.nodeName(a,b)?n.merge([a],c):c}function pb(a,b){var c=b.nodeName.toLowerCase();"input"===c&&T.test(a.type)?b.checked=a.checked:("input"===c||"textarea"===c)&&(b.defaultValue=a.defaultValue)}n.extend({clone:function(a,b,c){var d,e,f,g,h=a.cloneNode(!0),i=n.contains(a.ownerDocument,a);if(!(k.noCloneChecked||1!==a.nodeType&&11!==a.nodeType||n.isXMLDoc(a)))for(g=ob(h),f=ob(a),d=0,e=f.length;e>d;d++)pb(f[d],g[d]);if(b)if(c)for(f=f||ob(a),g=g||ob(h),d=0,e=f.length;e>d;d++)nb(f[d],g[d]);else nb(a,h);return g=ob(h,"script"),g.length>0&&mb(g,!i&&ob(a,"script")),h},buildFragment:function(a,b,c,d){for(var e,f,g,h,i,j,k=b.createDocumentFragment(),l=[],m=0,o=a.length;o>m;m++)if(e=a[m],e||0===e)if("object"===n.type(e))n.merge(l,e.nodeType?[e]:e);else if(cb.test(e)){f=f||k.appendChild(b.createElement("div")),g=(bb.exec(e)||["",""])[1].toLowerCase(),h=ib[g]||ib._default,f.innerHTML=h[1]+e.replace(ab,"<$1>")+h[2],j=h[0];while(j--)f=f.lastChild;n.merge(l,f.childNodes),f=k.firstChild,f.textContent=""}else l.push(b.createTextNode(e));k.textContent="",m=0;while(e=l[m++])if((!d||-1===n.inArray(e,d))&&(i=n.contains(e.ownerDocument,e),f=ob(k.appendChild(e),"script"),i&&mb(f),c)){j=0;while(e=f[j++])fb.test(e.type||"")&&c.push(e)}return k},cleanData:function(a){for(var b,c,d,e,f=n.event.special,g=0;void 0!==(c=a[g]);g++){if(n.acceptData(c)&&(e=c[L.expando],e&&(b=L.cache[e]))){if(b.events)for(d in b.events)f[d]?n.event.remove(c,d):n.removeEvent(c,d,b.handle);L.cache[e]&&delete L.cache[e]}delete M.cache[c[M.expando]]}}}),n.fn.extend({text:function(a){return J(this,function(a){return void 0===a?n.text(this):this.empty().each(function(){(1===this.nodeType||11===this.nodeType||9===this.nodeType)&&(this.textContent=a)})},null,a,arguments.length)},append:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=jb(this,a);b.appendChild(a)}})},prepend:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=jb(this,a);b.insertBefore(a,b.firstChild)}})},before:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this)})},after:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this.nextSibling)})},remove:function(a,b){for(var c,d=a?n.filter(a,this):this,e=0;null!=(c=d[e]);e++)b||1!==c.nodeType||n.cleanData(ob(c)),c.parentNode&&(b&&n.contains(c.ownerDocument,c)&&mb(ob(c,"script")),c.parentNode.removeChild(c));return this},empty:function(){for(var a,b=0;null!=(a=this[b]);b++)1===a.nodeType&&(n.cleanData(ob(a,!1)),a.textContent="");return this},clone:function(a,b){return a=null==a?!1:a,b=null==b?a:b,this.map(function(){return n.clone(this,a,b)})},html:function(a){return J(this,function(a){var b=this[0]||{},c=0,d=this.length;if(void 0===a&&1===b.nodeType)return b.innerHTML;if("string"==typeof a&&!db.test(a)&&!ib[(bb.exec(a)||["",""])[1].toLowerCase()]){a=a.replace(ab,"<$1>");try{for(;d>c;c++)b=this[c]||{},1===b.nodeType&&(n.cleanData(ob(b,!1)),b.innerHTML=a);b=0}catch(e){}}b&&this.empty().append(a)},null,a,arguments.length)},replaceWith:function(){var a=arguments[0];return this.domManip(arguments,function(b){a=this.parentNode,n.cleanData(ob(this)),a&&a.replaceChild(b,this)}),a&&(a.length||a.nodeType)?this:this.remove()},detach:function(a){return this.remove(a,!0)},domManip:function(a,b){a=e.apply([],a);var c,d,f,g,h,i,j=0,l=this.length,m=this,o=l-1,p=a[0],q=n.isFunction(p);if(q||l>1&&"string"==typeof p&&!k.checkClone&&eb.test(p))return this.each(function(c){var d=m.eq(c);q&&(a[0]=p.call(this,c,d.html())),d.domManip(a,b)});if(l&&(c=n.buildFragment(a,this[0].ownerDocument,!1,this),d=c.firstChild,1===c.childNodes.length&&(c=d),d)){for(f=n.map(ob(c,"script"),kb),g=f.length;l>j;j++)h=c,j!==o&&(h=n.clone(h,!0,!0),g&&n.merge(f,ob(h,"script"))),b.call(this[j],h,j);if(g)for(i=f[f.length-1].ownerDocument,n.map(f,lb),j=0;g>j;j++)h=f[j],fb.test(h.type||"")&&!L.access(h,"globalEval")&&n.contains(i,h)&&(h.src?n._evalUrl&&n._evalUrl(h.src):n.globalEval(h.textContent.replace(hb,"")))}return this}}),n.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(a,b){n.fn[a]=function(a){for(var c,d=[],e=n(a),g=e.length-1,h=0;g>=h;h++)c=h===g?this:this.clone(!0),n(e[h])[b](c),f.apply(d,c.get());return this.pushStack(d)}});var qb,rb={};function sb(b,c){var d,e=n(c.createElement(b)).appendTo(c.body),f=a.getDefaultComputedStyle&&(d=a.getDefaultComputedStyle(e[0]))?d.display:n.css(e[0],"display");return e.detach(),f}function tb(a){var b=l,c=rb[a];return c||(c=sb(a,b),"none"!==c&&c||(qb=(qb||n("