From 8340a1b0a1eb6d4a95a71397ada7571b3183276a Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 3 Jun 2014 10:34:38 -0700 Subject: [PATCH 001/107] Properties module can now take multiple properties files --- .../druid/initialization/Initialization.java | 3 +- .../initialization/PropertiesModule.java | 49 ++++++++++--------- 2 files changed, 28 insertions(+), 24 deletions(-) diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index 292aa3c95cf..70b810892f8 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -85,6 +85,7 @@ import java.net.URI; import java.net.URISyntaxException; import java.net.URL; import java.net.URLClassLoader; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -351,7 +352,7 @@ public class Initialization return Guice.createInjector( new DruidGuiceExtensions(), new JacksonModule(), - new PropertiesModule("runtime.properties"), + new PropertiesModule(Arrays.asList("global.runtime.properties", "runtime.properties")), new ConfigModule(), new Module() { diff --git a/server/src/main/java/io/druid/server/initialization/PropertiesModule.java b/server/src/main/java/io/druid/server/initialization/PropertiesModule.java index 3aad005dd68..c609504155f 100644 --- a/server/src/main/java/io/druid/server/initialization/PropertiesModule.java +++ b/server/src/main/java/io/druid/server/initialization/PropertiesModule.java @@ -33,6 +33,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; +import java.util.List; import java.util.Properties; /** @@ -41,11 +42,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 @@ -57,30 +58,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 { - fileProps.load(new InputStreamReader(stream, Charsets.UTF_8)); - } - 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 { - Closeables.closeQuietly(stream); + catch (FileNotFoundException e) { + log.wtf(e, "This can only happen if the .exists() call lied. That's f'd up."); + } + finally { + Closeables.closeQuietly(stream); + } } binder.bind(Properties.class).toInstance(props); From 01e8a713b6060ad0ba987fe13690ba694647817c Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Thu, 5 Jun 2014 17:42:53 +0530 Subject: [PATCH 002/107] unit tests passing with offheap-indexing --- .../io/druid/indexer/IndexGeneratorJob.java | 8 +- .../io/druid/indexing/common/TaskToolbox.java | 11 +- .../indexing/common/TaskToolboxFactory.java | 12 +- .../common/index/YeOldePlumberSchool.java | 12 +- .../indexing/common/task/DeleteTask.java | 60 ++-- .../druid/indexing/common/task/IndexTask.java | 3 +- .../common/task/RealtimeIndexTask.java | 1 + .../indexing/overlord/TaskLifecycleTest.java | 4 +- .../worker/WorkerTaskMonitorTest.java | 4 +- .../query/GroupByParallelQueryRunner.java | 15 +- .../query/groupby/GroupByQueryHelper.java | 9 +- .../groupby/GroupByQueryQueryToolChest.java | 12 +- .../groupby/GroupByQueryRunnerFactory.java | 10 +- .../segment/incremental/IncrementalIndex.java | 284 ++++++++++-------- .../incremental/IncrementalIndexAdapter.java | 16 +- .../IncrementalIndexStorageAdapter.java | 34 ++- .../java/io/druid/query/TestQueryRunners.java | 3 +- .../HyperUniqueBufferAggregatorTest.java | 93 ++++++ .../query/groupby/GroupByQueryRunnerTest.java | 10 +- .../GroupByTimeseriesQueryRunnerTest.java | 4 +- .../TimeseriesQueryRunnerBonusTest.java | 4 +- .../java/io/druid/segment/EmptyIndexTest.java | 3 +- .../io/druid/segment/IndexMergerTest.java | 7 +- .../test/java/io/druid/segment/TestIndex.java | 4 +- .../segment/data/IncrementalIndexTest.java | 9 +- .../filter/SpatialFilterBonusTest.java | 13 +- .../segment/filter/SpatialFilterTest.java | 13 +- .../IncrementalIndexStorageAdapterTest.java | 13 +- .../io/druid/guice/DruidProcessingModule.java | 26 +- .../io/druid/offheap/OffheapBufferPool.java | 53 ++++ .../realtime/plumber/FlushingPlumber.java | 8 +- .../plumber/FlushingPlumberSchool.java | 11 +- .../realtime/plumber/RealtimePlumber.java | 14 +- .../plumber/RealtimePlumberSchool.java | 14 +- .../druid/segment/realtime/plumber/Sink.java | 17 +- .../segment/realtime/FireDepartmentTest.java | 3 +- .../segment/realtime/RealtimeManagerTest.java | 3 +- .../plumber/RealtimePlumberSchoolTest.java | 2 + .../segment/realtime/plumber/SinkTest.java | 3 +- 39 files changed, 567 insertions(+), 258 deletions(-) create mode 100644 processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniqueBufferAggregatorTest.java create mode 100644 server/src/main/java/io/druid/offheap/OffheapBufferPool.java 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 b2f1dfda129..db0c9568dac 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -34,6 +34,7 @@ import com.metamx.common.ISE; 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; @@ -320,6 +321,7 @@ public class IndexGeneratorJob implements Jobby } } ); + index.close(); index = makeIncrementalIndex(bucket, aggs); startTime = System.currentTimeMillis(); @@ -378,7 +380,7 @@ public class IndexGeneratorJob implements Jobby } ); } - + index.close(); serializeOutIndex(context, bucket, mergedBase, Lists.newArrayList(allDimensionNames)); for (File file : toMerge) { @@ -610,13 +612,15 @@ public class IndexGeneratorJob implements Jobby private IncrementalIndex makeIncrementalIndex(Bucket theBucket, AggregatorFactory[] aggs) { + //TODO: review this, add a config for batch ingestion return new IncrementalIndex( new IncrementalIndexSchema.Builder() .withMinTimestamp(theBucket.time.getMillis()) .withSpatialDimensions(config.getSchema().getDataSchema().getParser()) .withQueryGranularity(config.getSchema().getDataSchema().getGranularitySpec().getQueryGranularity()) .withMetrics(aggs) - .build() + .build(), + new OffheapBufferPool(1024 * 1024 * 1024) ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java index 5eb50b622c6..944c5b19ea1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java @@ -28,6 +28,7 @@ import com.google.common.collect.Multimaps; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; import io.druid.client.FilteredServerView; +import io.druid.collections.StupidPool; import io.druid.indexing.common.actions.SegmentInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.actions.TaskActionClientFactory; @@ -46,6 +47,7 @@ import org.joda.time.Interval; import java.io.File; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Collection; import java.util.List; import java.util.Map; @@ -73,6 +75,7 @@ public class TaskToolbox private final SegmentLoader segmentLoader; private final ObjectMapper objectMapper; private final File taskWorkDir; + private final StupidPool indexPool; public TaskToolbox( TaskConfig config, @@ -90,7 +93,8 @@ public class TaskToolbox MonitorScheduler monitorScheduler, SegmentLoader segmentLoader, ObjectMapper objectMapper, - final File taskWorkDir + final File taskWorkDir, + StupidPool indexPool ) { this.config = config; @@ -109,6 +113,7 @@ public class TaskToolbox this.segmentLoader = segmentLoader; this.objectMapper = objectMapper; this.taskWorkDir = taskWorkDir; + this.indexPool = indexPool; } public TaskConfig getConfig() @@ -210,4 +215,8 @@ public class TaskToolbox { return taskWorkDir; } + + public StupidPool getIndexPool(){ + return indexPool; + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java index 89d275b9a7f..b950959b473 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java @@ -24,6 +24,8 @@ import com.google.inject.Inject; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; import io.druid.client.FilteredServerView; +import io.druid.collections.StupidPool; +import io.druid.guice.annotations.Global; import io.druid.guice.annotations.Processing; import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.config.TaskConfig; @@ -36,6 +38,7 @@ import io.druid.segment.loading.DataSegmentPusher; import io.druid.server.coordination.DataSegmentAnnouncer; import java.io.File; +import java.nio.ByteBuffer; import java.util.concurrent.ExecutorService; /** @@ -57,6 +60,7 @@ public class TaskToolboxFactory private final MonitorScheduler monitorScheduler; private final SegmentLoaderFactory segmentLoaderFactory; private final ObjectMapper objectMapper; + private final StupidPool bufferPool; @Inject public TaskToolboxFactory( @@ -73,7 +77,9 @@ public class TaskToolboxFactory @Processing ExecutorService queryExecutorService, MonitorScheduler monitorScheduler, SegmentLoaderFactory segmentLoaderFactory, - ObjectMapper objectMapper + ObjectMapper objectMapper, + //TODO: have a separate index pool + @Global StupidPool bufferPool ) { this.config = config; @@ -90,6 +96,7 @@ public class TaskToolboxFactory this.monitorScheduler = monitorScheduler; this.segmentLoaderFactory = segmentLoaderFactory; this.objectMapper = objectMapper; + this.bufferPool = bufferPool; } public TaskToolbox build(Task task) @@ -112,7 +119,8 @@ public class TaskToolboxFactory monitorScheduler, segmentLoaderFactory.manufacturate(taskWorkDir), objectMapper, - taskWorkDir + taskWorkDir, + bufferPool ); } } 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..ff9f1549607 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 @@ -31,7 +31,9 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.Granularity; import com.metamx.common.logger.Logger; +import io.druid.collections.StupidPool; import io.druid.data.input.InputRow; +import io.druid.guice.annotations.Global; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.segment.IndexIO; @@ -49,9 +51,11 @@ import io.druid.segment.realtime.plumber.Sink; import io.druid.timeline.DataSegment; import org.apache.commons.io.FileUtils; import org.joda.time.Interval; +import sun.misc.JavaNioAccess; import java.io.File; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.List; import java.util.Set; @@ -65,6 +69,7 @@ public class YeOldePlumberSchool implements PlumberSchool private final String version; private final DataSegmentPusher dataSegmentPusher; private final File tmpSegmentDir; + private final StupidPool bufferPool; private static final Logger log = new Logger(YeOldePlumberSchool.class); @@ -73,13 +78,16 @@ public class YeOldePlumberSchool implements PlumberSchool @JsonProperty("interval") Interval interval, @JsonProperty("version") String version, @JacksonInject("segmentPusher") DataSegmentPusher dataSegmentPusher, - @JacksonInject("tmpSegmentDir") File tmpSegmentDir + @JacksonInject("tmpSegmentDir") File tmpSegmentDir, + //TODO: review this global annotation + @JacksonInject @Global StupidPool bufferPool ) { this.interval = interval; this.version = version; this.dataSegmentPusher = dataSegmentPusher; this.tmpSegmentDir = tmpSegmentDir; + this.bufferPool = bufferPool; } @Override @@ -96,7 +104,7 @@ public class YeOldePlumberSchool implements PlumberSchool ) { // There can be only one. - final Sink theSink = new Sink(interval, schema, config, version); + final Sink theSink = new Sink(interval, schema, config, version, bufferPool); // Temporary directory to hold spilled segments. final File persistDir = new File(tmpSegmentDir, theSink.getSegment().getIdentifier()); 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 index 970818a6e9d..90d6eff2775 100644 --- 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 @@ -79,33 +79,43 @@ public class DeleteTask extends AbstractFixedIntervalTask { // 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() + final IncrementalIndex empty = new IncrementalIndex( + 0, + QueryGranularity.NONE, + new AggregatorFactory[0], + toolbox.getIndexPool() ); + try { + final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(getInterval(), empty); - toolbox.pushSegments(ImmutableList.of(uploadedSegment)); + // Create DataSegment + final DataSegment segment = + DataSegment.builder() + .dataSource(this.getDataSource()) + .interval(getInterval()) + .version(myLock.getVersion()) + .shardSpec(new NoneShardSpec()) + .build(); - return TaskStatus.success(getId()); + 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()); + } + finally { + empty.close(); + } } } 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 32565f1d51a..d4cd8ea0bb2 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 @@ -385,7 +385,8 @@ public class IndexTask extends AbstractFixedIntervalTask interval, version, wrappedDataSegmentPusher, - tmpDir + tmpDir, + toolbox.getIndexPool() ).findPlumber(schema, new RealtimeTuningConfig(null, null, null, null, null, null, null, shardSpec), metrics); // rowFlushBoundary for this job 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 e0fc20d1216..bfc6b35fc3f 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 @@ -308,6 +308,7 @@ public class RealtimeIndexTask extends AbstractTask segmentPublisher, toolbox.getNewSegmentServerView(), toolbox.getQueryExecutorService(), + toolbox.getIndexPool(), null, null, null, diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 30325c9d398..9068535f38a 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -43,6 +43,7 @@ import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.InputRowParser; import io.druid.granularity.QueryGranularity; +import io.druid.offheap.OffheapBufferPool; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskLock; @@ -205,7 +206,8 @@ public class TaskLifecycleTest } ) ), - new DefaultObjectMapper() + new DefaultObjectMapper(), + new OffheapBufferPool(1024 * 1024) ); tr = new ThreadPoolTaskRunner(tb); tq = new TaskQueue(tqc, ts, tr, tac, tl, emitter); diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index 8d4bf32b870..4f88488e436 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -38,6 +38,7 @@ import io.druid.indexing.overlord.TestRemoteTaskRunnerConfig; import io.druid.indexing.overlord.ThreadPoolTaskRunner; import io.druid.indexing.worker.config.WorkerConfig; import io.druid.jackson.DefaultObjectMapper; +import io.druid.offheap.OffheapBufferPool; import io.druid.segment.loading.DataSegmentPuller; import io.druid.segment.loading.LocalDataSegmentPuller; import io.druid.segment.loading.OmniSegmentLoader; @@ -138,7 +139,8 @@ public class WorkerTaskMonitorTest } } ) - ), jsonMapper + ), jsonMapper, + new OffheapBufferPool(1024 * 1024) ) ), new WorkerConfig().setCapacity(1) diff --git a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java index 51c663c6a2e..43e65b9e49a 100644 --- a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java @@ -31,12 +31,14 @@ 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.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.Arrays; import java.util.List; import java.util.concurrent.ExecutionException; @@ -51,20 +53,25 @@ public class GroupByParallelQueryRunner implements QueryRunner private final ExecutorService exec; private final Ordering ordering; private final Supplier configSupplier; + private final StupidPool bufferPool; + public GroupByParallelQueryRunner( ExecutorService exec, Ordering ordering, Supplier configSupplier, + StupidPool bufferPool, QueryRunner... queryables ) { - this(exec, ordering, configSupplier, Arrays.asList(queryables)); + this(exec, ordering, configSupplier, bufferPool, Arrays.asList(queryables)); } public GroupByParallelQueryRunner( ExecutorService exec, - Ordering ordering, Supplier configSupplier, + Ordering ordering, + Supplier configSupplier, + StupidPool bufferPool, Iterable> queryables ) { @@ -72,6 +79,7 @@ public class GroupByParallelQueryRunner implements QueryRunner this.ordering = ordering; this.queryables = Iterables.unmodifiableIterable(Iterables.filter(queryables, Predicates.notNull())); this.configSupplier = configSupplier; + this.bufferPool = bufferPool; } @Override @@ -81,7 +89,8 @@ public class GroupByParallelQueryRunner implements QueryRunner final GroupByQuery query = (GroupByQuery) queryParam; final Pair> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair( query, - configSupplier.get() + configSupplier.get(), + bufferPool ); final int priority = query.getContextPriority(0); 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 00298f18ba0..e7f1fdea912 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java @@ -24,6 +24,7 @@ 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; @@ -32,13 +33,16 @@ import io.druid.query.dimension.DimensionSpec; import io.druid.segment.incremental.IncrementalIndex; import javax.annotation.Nullable; +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(); @@ -75,7 +79,8 @@ public class GroupByQueryHelper // since incoming truncated timestamps may precede timeStart granTimeStart, gran, - aggs.toArray(new AggregatorFactory[aggs.size()]) + aggs.toArray(new AggregatorFactory[aggs.size()]), + bufferPool ); Accumulator accumulator = new Accumulator() 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 0e00ceae46d..1cec9a993c7 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -32,8 +32,10 @@ import com.metamx.common.guava.ConcatSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.collections.StupidPool; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; +import io.druid.guice.annotations.Global; import io.druid.query.DataSource; import io.druid.query.DataSourceUtil; import io.druid.query.IntervalChunkingQueryRunner; @@ -49,6 +51,7 @@ import io.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.joda.time.Interval; import org.joda.time.Minutes; +import java.nio.ByteBuffer; import java.util.Map; /** @@ -62,15 +65,19 @@ public class GroupByQueryQueryToolChest extends QueryToolChest NO_MERGE_CONTEXT = ImmutableMap.of(GROUP_BY_MERGE_KEY, "false"); private final Supplier configSupplier; private GroupByQueryEngine engine; // For running the outer query around a subquery + private final StupidPool bufferPool; + @Inject public GroupByQueryQueryToolChest( Supplier configSupplier, - GroupByQueryEngine engine + GroupByQueryEngine engine, + @Global StupidPool bufferPool ) { this.configSupplier = configSupplier; this.engine = engine; + this.bufferPool = bufferPool; } @Override @@ -142,7 +149,8 @@ public class GroupByQueryQueryToolChest extends QueryToolChest> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair( query, - config + config, + bufferPool ); return rows.accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); 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 714aad37925..a498a162965 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java @@ -29,7 +29,9 @@ import com.metamx.common.ISE; import com.metamx.common.guava.ExecutorExecutingSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import io.druid.collections.StupidPool; import io.druid.data.input.Row; +import io.druid.guice.annotations.Global; import io.druid.query.ConcatQueryRunner; import io.druid.query.GroupByParallelQueryRunner; import io.druid.query.Query; @@ -39,6 +41,7 @@ import io.druid.query.QueryToolChest; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; +import java.nio.ByteBuffer; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -51,17 +54,20 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory config; private final GroupByQueryQueryToolChest toolChest; + private final StupidPool bufferPool; @Inject public GroupByQueryRunnerFactory( GroupByQueryEngine engine, Supplier config, - GroupByQueryQueryToolChest toolChest + GroupByQueryQueryToolChest toolChest, + @Global StupidPool bufferPool ) { this.engine = engine; this.config = config; this.toolChest = toolChest; + this.bufferPool = bufferPool; } @Override @@ -117,7 +123,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory +public class IncrementalIndex implements Iterable, Closeable { private static final Logger log = new Logger(IncrementalIndex.class); private static final Joiner JOINER = Joiner.on(","); @@ -76,17 +81,21 @@ public class IncrementalIndex implements Iterable 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; private final DimensionHolder dimValues; - private final ConcurrentSkipListMap facts; + private final ConcurrentSkipListMap 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) + public IncrementalIndex(IncrementalIndexSchema incrementalIndexSchema, StupidPool bufferPool) { this.minTimestamp = incrementalIndexSchema.getMinTimestamp(); this.gran = incrementalIndexSchema.getGran(); @@ -95,7 +104,100 @@ public class IncrementalIndex implements Iterable 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 TimestampColumnSelector makeTimestampColumnSelector() + { + return new TimestampColumnSelector() + { + @Override + public long getTimestamp() + { + return in.get().getTimestampFromEpoch(); + } + }; + } + + @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(); + + if (typeName.equals("float")) { + return new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return Float.TYPE; + } + + @Override + public Float get() + { + return in.get().getFloatMetric(columnName); + } + }; + } + + 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(String dimension) + { + // we should implement this, but this is going to be rewritten soon anyways + throw new UnsupportedOperationException( + "Incremental index aggregation does not support dimension selectors" + ); + } + } + ); + aggPositionOffsets[i] = currAggSize; + currAggSize += agg.getMaxIntermediateSize(); final String metricName = metrics[i].getName().toLowerCase(); metricNamesBuilder.add(metricName); metricIndexesBuilder.put(metricName, i); @@ -105,6 +207,8 @@ public class IncrementalIndex implements Iterable metricIndexes = metricIndexesBuilder.build(); metricTypes = metricTypesBuilder.build(); + this.totalAggSize = currAggSize; + this.dimensionOrder = Maps.newLinkedHashMap(); this.dimensions = new CopyOnWriteArrayList(); int index = 0; @@ -114,22 +218,24 @@ public class IncrementalIndex implements Iterable } this.spatialDimensions = incrementalIndexSchema.getSpatialDimensions(); this.spatialDimensionRowFormatter = new SpatialDimensionRowFormatter(spatialDimensions); - + this.bufferHolder = bufferPool.take(); this.dimValues = new DimensionHolder(); - this.facts = new ConcurrentSkipListMap(); + this.facts = 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 ); } @@ -137,7 +243,7 @@ public class IncrementalIndex implements Iterable * 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. *

- * This is *not* thread-safe. Calls to add() should always happen on the same thread. + * This method is thread-safe. * * @param row the row of data to add * @@ -188,118 +294,23 @@ public class IncrementalIndex implements Iterable 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(); - - if (typeName.equals("float")) { - return new ObjectColumnSelector() - { - @Override - public Class classOfObject() - { - return Float.TYPE; - } - - @Override - public Float get() - { - return in.getFloatMetric(columnName); - } - }; - } - - 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(String dimension) - { - // we should implement this, but this is going to be rewritten soon anyways - throw new UnsupportedOperationException( - "Incremental index aggregation does not support dimension selectors" - ); - } - } - - ); - } - - Aggregator[] prev = facts.putIfAbsent(key, aggs); - if (prev != null) { - aggs = prev; - } else { - numEntries.incrementAndGet(); - } - } - synchronized (this) { - in = row; - for (Aggregator agg : aggs) { - agg.aggregate(); + if (!facts.containsKey(key)) { + int rowOffset = totalAggSize * numEntries.getAndIncrement(); + for (int i = 0; i < aggs.length; i++) { + aggs[i].init(bufferHolder.get(), getMetricPosition(rowOffset, i)); + } + facts.put(key, rowOffset); } - in = null; } + in.set(row); + int rowOffset = facts.get(key); + 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(); } @@ -413,12 +424,27 @@ 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]; + } + + ConcurrentSkipListMap getFacts() { return facts; } - ConcurrentNavigableMap getSubMap(TimeAndDims start, TimeAndDims end) + ConcurrentNavigableMap getSubMap(TimeAndDims start, TimeAndDims end) { return facts.subMap(start, end); } @@ -438,13 +464,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(); @@ -456,8 +482,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) { @@ -474,6 +500,12 @@ public class IncrementalIndex implements Iterable }; } + @Override + public void close() throws IOException + { + bufferHolder.close(); + } + static class DimensionHolder { private final Map dimensions; 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..bf44a8b25ac 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java @@ -24,7 +24,6 @@ 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.data.EmptyIndexedInts; @@ -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( @@ -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,9 +202,10 @@ 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(); 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 0eddf59ac98..3e12d0726c0 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -29,7 +29,7 @@ import com.metamx.collections.spatial.search.Bound; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.granularity.QueryGranularity; -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; @@ -61,7 +61,6 @@ import java.util.concurrent.ConcurrentNavigableMap; public class IncrementalIndexStorageAdapter implements StorageAdapter { private static final Splitter SPLITTER = Splitter.on(","); - private final IncrementalIndex index; public IncrementalIndexStorageAdapter( @@ -165,8 +164,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; @@ -355,13 +354,17 @@ 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) + ); } }; } @@ -376,7 +379,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 @@ -388,7 +391,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) + ); } }; } @@ -411,11 +417,9 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter final String[] dimVals = currEntry.getKey().getDims()[dimensionIndex]; if (dimVals.length == 1) { return dimVals[0]; - } - else if (dimVals.length == 0) { + } else if (dimVals.length == 0) { return null; - } - else { + } else { return dimVals; } } @@ -439,14 +443,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; @@ -457,7 +461,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter return currEntry.getKey(); } - public Aggregator[] getValue() + public Integer getValue() { return currEntry.getValue(); } diff --git a/processing/src/test/java/io/druid/query/TestQueryRunners.java b/processing/src/test/java/io/druid/query/TestQueryRunners.java index c4767c1c6f9..94d263f7cdf 100644 --- a/processing/src/test/java/io/druid/query/TestQueryRunners.java +++ b/processing/src/test/java/io/druid/query/TestQueryRunners.java @@ -24,11 +24,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/aggregation/hyperloglog/HyperUniqueBufferAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniqueBufferAggregatorTest.java new file mode 100644 index 00000000000..d2d291ac3b9 --- /dev/null +++ b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniqueBufferAggregatorTest.java @@ -0,0 +1,93 @@ +package io.druid.query.aggregation.hyperloglog; + +import com.google.common.hash.HashFunction; +import com.google.common.hash.Hashing; +import io.druid.segment.ObjectColumnSelector; +import org.junit.Test; + +import java.nio.ByteBuffer; + +/** + * Created with IntelliJ IDEA. + * User: neo + * Date: 05/06/14 + * Time: 3:14 PM + * To change this template use File | Settings | File Templates. + */ +public class HyperUniqueBufferAggregatorTest +{ + private final HashFunction fn = Hashing.murmur3_128(); + private volatile HyperLogLogCollector collector; + + @Test + public void testAggregation() + { + final HyperUniquesBufferAggregator agg = new HyperUniquesBufferAggregator( + new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return HyperLogLogCollector.class; + } + + @Override + public Object get() + { + return collector; + } + } + ); + ByteBuffer byteBuffer = ByteBuffer.allocate(HyperLogLogCollector.getLatestNumBytesForDenseStorage()); + + for (int i = 0; i < 1000; i++) { + collector = HyperLogLogCollector.makeLatestCollector(); + collector.add(fn.hashInt(i).asBytes()); + agg.aggregate(byteBuffer, 0); + } + + final HyperLogLogCollector collector = HyperLogLogCollector.makeCollector( + ((HyperLogLogCollector) agg.get( + byteBuffer, + 0 + )).toByteBuffer() + ); + System.out.println(collector.estimateCardinality()); + + } + + @Test + public void testAggregation2() + { + final HyperUniquesAggregator agg = new HyperUniquesAggregator( + "abc", + new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return HyperLogLogCollector.class; + } + + @Override + public Object get() + { + return collector; + } + } + ); + + for (int i = 0; i < 1000; i++) { + collector = HyperLogLogCollector.makeLatestCollector(); + collector.add(fn.hashInt(i).asBytes()); + agg.aggregate(); + } + + final HyperLogLogCollector collector = HyperLogLogCollector.makeCollector( + ((HyperLogLogCollector) agg.get( + )).toByteBuffer() + ); + System.out.println(collector.estimateCardinality()); + + } +} 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 97e64a0ec0c..396c8a5ef33 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -40,6 +40,7 @@ import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryToolChest; +import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; @@ -114,7 +115,8 @@ public class GroupByQueryRunnerTest final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( engine, configSupplier, - new GroupByQueryQueryToolChest(configSupplier, engine) + new GroupByQueryQueryToolChest(configSupplier, engine, TestQueryRunners.pool), + TestQueryRunners.pool ); return Lists.newArrayList( @@ -758,7 +760,7 @@ public class GroupByQueryRunnerTest ) ); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner); + QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine, TestQueryRunners.pool).mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); } @@ -807,7 +809,7 @@ public class GroupByQueryRunnerTest ) ); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner); + QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine, TestQueryRunners.pool).mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); } @@ -856,7 +858,7 @@ public class GroupByQueryRunnerTest ) ); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner); + QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine, TestQueryRunners.pool).mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); } 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 2538e91bc76..b5077cb2dc6 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -32,6 +32,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; @@ -73,7 +74,8 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( engine, configSupplier, - new GroupByQueryQueryToolChest(configSupplier, engine) + new GroupByQueryQueryToolChest(configSupplier, engine, TestQueryRunners.pool), + TestQueryRunners.pool ); final Collection objects = QueryRunnerTestHelper.makeQueryRunners(factory); 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 d1497a19026..36eba162090 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java @@ -31,6 +31,7 @@ import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; 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; @@ -49,7 +50,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; diff --git a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java index d4e835566a9..04db1387a37 100644 --- a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java +++ b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java @@ -22,6 +22,7 @@ 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.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexAdapter; @@ -45,7 +46,7 @@ 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); diff --git a/processing/src/test/java/io/druid/segment/IndexMergerTest.java b/processing/src/test/java/io/druid/segment/IndexMergerTest.java index a21eb92c718..ace0ae7c33e 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerTest.java @@ -25,6 +25,7 @@ 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.data.IncrementalIndexTest; import io.druid.segment.incremental.IncrementalIndex; @@ -65,7 +66,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( @@ -117,8 +118,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(); diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index 95823210549..14eb6c9db3c 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -31,6 +31,7 @@ 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; @@ -155,7 +156,8 @@ public class TestIndex 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 + new DateTime("2011-01-12T00:00:00.000Z").getMillis(), QueryGranularity.NONE, METRIC_AGGS, + TestQueryRunners.pool ); final AtomicLong startTime = new AtomicLong(); 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 0eb327972ee..3a7d3d39e7c 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java @@ -31,6 +31,7 @@ import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryRunner; 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; @@ -108,7 +109,8 @@ public class SpatialFilterBonusTest Lists.newArrayList() ) ) - ).build() + ).build(), + TestQueryRunners.pool ); theIndex.add( new MapBasedInputRow( @@ -233,7 +235,8 @@ public class SpatialFilterBonusTest Lists.newArrayList() ) ) - ).build() + ).build(), + TestQueryRunners.pool ); IncrementalIndex second = new IncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -246,7 +249,8 @@ public class SpatialFilterBonusTest Lists.newArrayList() ) ) - ).build() + ).build(), + TestQueryRunners.pool ); IncrementalIndex third = new IncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -259,7 +263,8 @@ public class SpatialFilterBonusTest Lists.newArrayList() ) ) - ).build() + ).build(), + TestQueryRunners.pool ); 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 d342c12c577..f69166da60d 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java @@ -31,6 +31,7 @@ import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryRunner; 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; @@ -108,7 +109,8 @@ public class SpatialFilterTest Arrays.asList("lat", "long") ) ) - ).build() + ).build(), + TestQueryRunners.pool ); theIndex.add( new MapBasedInputRow( @@ -248,7 +250,8 @@ public class SpatialFilterTest Arrays.asList("lat", "long") ) ) - ).build() + ).build(), + TestQueryRunners.pool ); IncrementalIndex second = new IncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -261,7 +264,8 @@ public class SpatialFilterTest Arrays.asList("lat", "long") ) ) - ).build() + ).build(), + TestQueryRunners.pool ); IncrementalIndex third = new IncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -274,7 +278,8 @@ public class SpatialFilterTest Arrays.asList("lat", "long") ) ) - ).build() + ).build(), + TestQueryRunners.pool ); 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 db6fc2f909d..1eee18fe782 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.LongSumAggregatorFactory; @@ -62,7 +63,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( @@ -127,7 +129,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 ); @@ -179,7 +182,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(); @@ -234,7 +238,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/server/src/main/java/io/druid/guice/DruidProcessingModule.java b/server/src/main/java/io/druid/guice/DruidProcessingModule.java index 402aded1f74..ccedcc6e110 100644 --- a/server/src/main/java/io/druid/guice/DruidProcessingModule.java +++ b/server/src/main/java/io/druid/guice/DruidProcessingModule.java @@ -33,6 +33,7 @@ import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.collections.StupidPool; import io.druid.guice.annotations.Global; import io.druid.guice.annotations.Processing; +import io.druid.offheap.OffheapBufferPool; import io.druid.query.MetricsEmittingExecutorService; import io.druid.query.PrioritizedExecutorService; import io.druid.server.DruidProcessingConfig; @@ -102,31 +103,8 @@ public class DruidProcessingModule implements Module log.warn(e, 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/realtime/plumber/FlushingPlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java index 083c290c4cc..30babfd57ec 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java @@ -25,6 +25,7 @@ import com.metamx.common.Granularity; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; +import io.druid.collections.StupidPool; import io.druid.common.guava.ThreadRenamingCallable; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.indexing.DataSchema; @@ -34,6 +35,7 @@ import io.druid.server.coordination.DataSegmentAnnouncer; import org.joda.time.DateTime; import org.joda.time.Duration; +import java.nio.ByteBuffer; import java.util.List; import java.util.Map; import java.util.concurrent.Callable; @@ -62,7 +64,8 @@ public class FlushingPlumber extends RealtimePlumber ServiceEmitter emitter, QueryRunnerFactoryConglomerate conglomerate, DataSegmentAnnouncer segmentAnnouncer, - ExecutorService queryExecutorService + ExecutorService queryExecutorService, + StupidPool bufferPool ) { super( @@ -75,7 +78,8 @@ public class FlushingPlumber extends RealtimePlumber queryExecutorService, null, null, - null + null, + bufferPool ); this.flushDuration = flushDuration; diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java index 026af6f43c8..f7de3ff21bd 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java @@ -25,6 +25,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.metamx.common.Granularity; import com.metamx.emitter.service.ServiceEmitter; +import io.druid.collections.StupidPool; +import io.druid.guice.annotations.Global; import io.druid.guice.annotations.Processing; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.indexing.DataSchema; @@ -35,6 +37,7 @@ import org.joda.time.Duration; import org.joda.time.Period; import java.io.File; +import java.nio.ByteBuffer; import java.util.concurrent.ExecutorService; /** @@ -51,6 +54,7 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool private final QueryRunnerFactoryConglomerate conglomerate; private final DataSegmentAnnouncer segmentAnnouncer; private final ExecutorService queryExecutorService; + private final StupidPool bufferPool; @JsonCreator public FlushingPlumberSchool( @@ -59,6 +63,8 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool @JacksonInject QueryRunnerFactoryConglomerate conglomerate, @JacksonInject DataSegmentAnnouncer segmentAnnouncer, @JacksonInject @Processing ExecutorService queryExecutorService, + //TODO: define separate index pool + @JacksonInject @Global StupidPool bufferPool, // Backwards compatible @JsonProperty("windowPeriod") Period windowPeriod, @JsonProperty("basePersistDirectory") File basePersistDirectory, @@ -76,6 +82,7 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool null, null, queryExecutorService, + bufferPool, windowPeriod, basePersistDirectory, segmentGranularity, @@ -89,6 +96,7 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool this.conglomerate = conglomerate; this.segmentAnnouncer = segmentAnnouncer; this.queryExecutorService = queryExecutorService; + this.bufferPool = bufferPool; } @Override @@ -108,7 +116,8 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool emitter, conglomerate, segmentAnnouncer, - queryExecutorService + queryExecutorService, + bufferPool ); } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 306350aaaa3..cbe8b597b9f 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -19,10 +19,12 @@ import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.client.FilteredServerView; import io.druid.client.ServerView; +import io.druid.collections.StupidPool; import io.druid.common.guava.ThreadRenamingCallable; import io.druid.common.guava.ThreadRenamingRunnable; import io.druid.concurrent.Execs; import io.druid.data.input.InputRow; +import io.druid.guice.annotations.Global; import io.druid.query.MetricsEmittingQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; @@ -59,6 +61,7 @@ import javax.annotation.Nullable; import java.io.File; import java.io.FilenameFilter; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Comparator; import java.util.List; @@ -95,6 +98,8 @@ public class RealtimePlumber implements Plumber private volatile ExecutorService persistExecutor = null; private volatile ExecutorService mergeExecutor = null; private volatile ScheduledExecutorService scheduledExecutor = null; + private final StupidPool bufferPool; + public RealtimePlumber( DataSchema schema, @@ -106,7 +111,9 @@ public class RealtimePlumber implements Plumber ExecutorService queryExecutorService, DataSegmentPusher dataSegmentPusher, SegmentPublisher segmentPublisher, - FilteredServerView serverView + FilteredServerView serverView, + StupidPool bufferPool + ) { this.schema = schema; @@ -120,6 +127,7 @@ public class RealtimePlumber implements Plumber this.dataSegmentPusher = dataSegmentPusher; this.segmentPublisher = segmentPublisher; this.serverView = serverView; + this.bufferPool = bufferPool; log.info("Creating plumber using rejectionPolicy[%s]", getRejectionPolicy()); } @@ -184,7 +192,7 @@ public class RealtimePlumber implements Plumber segmentGranularity.increment(new DateTime(truncatedTime)) ); - retVal = new Sink(sinkInterval, schema, config, versioningPolicy.getVersion(sinkInterval)); + retVal = new Sink(sinkInterval, schema, config, versioningPolicy.getVersion(sinkInterval), bufferPool); try { segmentAnnouncer.announceSegment(retVal.getSegment()); @@ -535,7 +543,7 @@ public class RealtimePlumber implements Plumber ); } - Sink currSink = new Sink(sinkInterval, schema, config, versioningPolicy.getVersion(sinkInterval), hydrants); + Sink currSink = new Sink(sinkInterval, schema, config, versioningPolicy.getVersion(sinkInterval), hydrants, bufferPool); sinks.put(sinkInterval.getStartMillis(), currSink); sinkTimeline.add( currSink.getInterval(), diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java index eb52a30ba31..2b24fcf4279 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java @@ -23,10 +23,14 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; import com.metamx.common.Granularity; +import com.metamx.common.logger.Logger; import com.metamx.emitter.service.ServiceEmitter; import io.druid.client.FilteredServerView; import io.druid.client.ServerView; +import io.druid.collections.StupidPool; +import io.druid.guice.annotations.Global; import io.druid.guice.annotations.Processing; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.indexing.DataSchema; @@ -38,7 +42,9 @@ import io.druid.server.coordination.DataSegmentAnnouncer; import org.joda.time.Period; import java.io.File; +import java.nio.ByteBuffer; import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicLong; /** */ @@ -51,6 +57,7 @@ public class RealtimePlumberSchool implements PlumberSchool private final SegmentPublisher segmentPublisher; private final FilteredServerView serverView; private final ExecutorService queryExecutorService; + private final StupidPool bufferPool; // Backwards compatible private final Period windowPeriod; @@ -69,6 +76,8 @@ public class RealtimePlumberSchool implements PlumberSchool @JacksonInject SegmentPublisher segmentPublisher, @JacksonInject FilteredServerView serverView, @JacksonInject @Processing ExecutorService executorService, + //TODO: define separate index pool + @JacksonInject @Global StupidPool bufferPool, // Backwards compatible @JsonProperty("windowPeriod") Period windowPeriod, @JsonProperty("basePersistDirectory") File basePersistDirectory, @@ -91,6 +100,7 @@ public class RealtimePlumberSchool implements PlumberSchool this.versioningPolicy = versioningPolicy; this.rejectionPolicyFactory = rejectionPolicyFactory; this.maxPendingPersists = maxPendingPersists; + this.bufferPool = bufferPool; } @Deprecated @@ -149,7 +159,8 @@ public class RealtimePlumberSchool implements PlumberSchool queryExecutorService, dataSegmentPusher, segmentPublisher, - serverView + serverView, + bufferPool ); } @@ -162,4 +173,5 @@ public class RealtimePlumberSchool implements PlumberSchool Preconditions.checkNotNull(serverView, "must specify a serverView to do this action."); Preconditions.checkNotNull(emitter, "must specify a serviceEmitter to do this action."); } + } 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 f3ff140193b..4dc1edf61b8 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 @@ -27,8 +27,10 @@ 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.collections.StupidPool; import io.druid.data.input.InputRow; import io.druid.data.input.impl.SpatialDimensionSchema; +import io.druid.guice.annotations.Global; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; @@ -39,6 +41,7 @@ import io.druid.timeline.DataSegment; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Iterator; import java.util.List; @@ -57,18 +60,23 @@ public class Sink implements Iterable private final RealtimeTuningConfig config; private final String version; private final CopyOnWriteArrayList hydrants = new CopyOnWriteArrayList(); + private final StupidPool bufferPool; + public Sink( Interval interval, DataSchema schema, RealtimeTuningConfig config, - String version + String version, + StupidPool bufferPool + ) { this.schema = schema; this.config = config; this.interval = interval; this.version = version; + this.bufferPool = bufferPool; makeNewCurrIndex(interval.getStartMillis(), schema); } @@ -78,13 +86,15 @@ public class Sink implements Iterable DataSchema schema, RealtimeTuningConfig config, String version, - List hydrants + List hydrants, + StupidPool bufferPool ) { this.schema = schema; this.config = config; this.interval = interval; this.version = version; + this.bufferPool = bufferPool; for (int i = 0; i < hydrants.size(); ++i) { final FireHydrant hydrant = hydrants.get(i); @@ -183,7 +193,8 @@ public class Sink implements Iterable .withQueryGranularity(schema.getGranularitySpec().getQueryGranularity()) .withSpatialDimensions(schema.getParser()) .withMetrics(schema.getAggregators()) - .build() + .build(), + bufferPool ); FireHydrant old; 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 19c104a4ff0..1566013194c 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -27,6 +27,7 @@ import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; 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.segment.indexing.DataSchema; @@ -73,7 +74,7 @@ public class FireDepartmentTest new RealtimeIOConfig( null, new RealtimePlumberSchool( - null, null, null, null, null, null, null, null, null, null, null, null, 0 + null, null, null, null, null, null, null, TestQueryRunners.pool, null, null, null, null, null, 0 ) ), new RealtimeTuningConfig( diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index e29dd187b4b..4a1f692a3f4 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -31,6 +31,7 @@ import io.druid.data.input.impl.InputRowParser; import io.druid.granularity.QueryGranularity; import io.druid.query.Query; import io.druid.query.QueryRunner; +import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.indexing.DataSchema; @@ -117,7 +118,7 @@ public class RealtimeManagerTest null, null ); - plumber = new TestPlumber(new Sink(new Interval("0/P5000Y"), schema, tuningConfig, new DateTime().toString())); + plumber = new TestPlumber(new Sink(new Interval("0/P5000Y"), schema, tuningConfig, new DateTime().toString(), TestQueryRunners.pool)); realtimeManager = new RealtimeManager( Arrays.asList( 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 ddcb503af58..f2a060ea1cc 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 @@ -40,6 +40,7 @@ import io.druid.granularity.QueryGranularity; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.Query; import io.druid.query.QueryRunnerFactory; +import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.indexing.DataSchema; @@ -144,6 +145,7 @@ public class RealtimePlumberSchoolTest segmentPublisher, serverView, MoreExecutors.sameThreadExecutor(), + TestQueryRunners.pool, new Period("PT10m"), tmpDir, Granularity.HOUR, diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index b068a994cb7..e50ac4403dc 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.Lists; import com.metamx.common.Granularity; import io.druid.data.input.InputRow; 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.indexing.DataSchema; @@ -64,7 +65,7 @@ public class SinkTest null, null ); - final Sink sink = new Sink(interval, schema, tuningConfig, version); + final Sink sink = new Sink(interval, schema, tuningConfig, version, TestQueryRunners.pool); sink.add( new InputRow() From 5bdc4a761ae8fafa17262f1573001e5eee2d5c7c Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Thu, 5 Jun 2014 19:41:43 +0530 Subject: [PATCH 003/107] close IncrementalIndex properly and free up buffer --- .../common/index/YeOldePlumberSchool.java | 5 +++-- .../query/GroupByParallelQueryRunner.java | 8 ++++++- .../groupby/GroupByQueryQueryToolChest.java | 21 ++++++++++--------- .../realtime/plumber/RealtimePlumber.java | 6 ++++-- 4 files changed, 25 insertions(+), 15 deletions(-) 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 ff9f1549607..de3aa720219 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 @@ -40,6 +40,7 @@ import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; import io.druid.segment.SegmentUtils; +import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.loading.DataSegmentPusher; @@ -219,14 +220,14 @@ public class YeOldePlumberSchool implements PlumberSchool log.info("Spilling index[%d] with rows[%d] to: %s", indexToPersist.getCount(), rowsToPersist, dirToPersist); try { - + final IncrementalIndex index = indexToPersist.getIndex(); IndexMerger.persist( indexToPersist.getIndex(), dirToPersist ); indexToPersist.swapSegment(null); - + index.close(); metrics.incrementRowOutputCount(rowsToPersist); spilled.add(dirToPersist); diff --git a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java index 43e65b9e49a..98b6ae7d9f5 100644 --- a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java @@ -28,6 +28,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; 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; @@ -140,7 +141,12 @@ public class GroupByParallelQueryRunner implements QueryRunner } } - return Sequences.simple(indexAccumulatorPair.lhs.iterableWithPostAggregations(null)); + return new ResourceClosingSequence( + Sequences.simple( + indexAccumulatorPair.lhs + .iterableWithPostAggregations(null) + ), indexAccumulatorPair.lhs + ); } } 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 1cec9a993c7..6a5845b3ef1 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -29,6 +29,7 @@ import com.google.inject.Inject; import com.metamx.common.Pair; import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.ConcatSequence; +import com.metamx.common.guava.ResourceClosingSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.emitter.service.ServiceMetricEvent; @@ -62,10 +63,13 @@ public class GroupByQueryQueryToolChest extends QueryToolChest NO_MERGE_CONTEXT = ImmutableMap.of(GROUP_BY_MERGE_KEY, "false"); + private static final Map NO_MERGE_CONTEXT = ImmutableMap.of( + GROUP_BY_MERGE_KEY, + "false" + ); private final Supplier configSupplier; - private GroupByQueryEngine engine; // For running the outer query around a subquery private final StupidPool bufferPool; + private GroupByQueryEngine engine; // For running the outer query around a subquery @Inject @@ -99,16 +103,15 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeGroupByResults(final GroupByQuery query, QueryRunner runner) { - Sequence result; - // If there's a subquery, merge subquery results and then apply the aggregator DataSource dataSource = query.getDataSource(); if (dataSource instanceof QueryDataSource) { GroupByQuery subquery; try { subquery = (GroupByQuery) ((QueryDataSource) dataSource).getQuery(); - } catch (ClassCastException e) { + } + catch (ClassCastException e) { throw new UnsupportedOperationException("Subqueries must be of type 'group by'"); } Sequence subqueryResult = mergeGroupByResults(subquery, runner); @@ -118,11 +121,10 @@ public class GroupByQueryQueryToolChest extends QueryToolChest(postAggregate(query, index), index); } - private Sequence postAggregate(final GroupByQuery query, IncrementalIndex index) { Sequence sequence = Sequences.map( @@ -135,7 +137,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeSequences(Sequence> seqOfSequences) { diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index cbe8b597b9f..26b4cc45509 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -39,6 +39,7 @@ import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; +import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.loading.DataSegmentPusher; @@ -713,14 +714,15 @@ public class RealtimePlumber implements Plumber indexToPersist.getIndex(), new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())) ); - + IncrementalIndex index = indexToPersist.getIndex(); indexToPersist.swapSegment( new QueryableIndexSegment( indexToPersist.getSegment().getIdentifier(), IndexIO.loadIndex(persistedFile) ) ); - + //TODO: can there be some races here ? + index.close(); return numRows; } catch (IOException e) { From 7233fce50cc7ad37e52d4626d2f1d97940b6cbd9 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 9 Jun 2014 15:59:44 -0700 Subject: [PATCH 004/107] refactor examples to use global runtime --- examples/config/broker/runtime.properties | 4 +-- .../config/coordinator/runtime.properties | 6 ---- examples/config/global.runtime.properties | 24 +++++++++++++++ examples/config/historical/runtime.properties | 2 -- examples/config/overlord/runtime.properties | 28 +++++------------- examples/config/realtime/runtime.properties | 11 ------- pom.xml | 2 +- .../server/initialization/EmitterModule.java | 5 ++-- .../initialization/NoopEmitterModule.java | 29 +++++++++++++++++++ 9 files changed, 65 insertions(+), 46 deletions(-) create mode 100644 examples/config/global.runtime.properties create mode 100644 server/src/main/java/io/druid/server/initialization/NoopEmitterModule.java diff --git a/examples/config/broker/runtime.properties b/examples/config/broker/runtime.properties index 8afae982654..31799a7caf9 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 +# Add more threads or larger buffer for faster groupBys 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/global.runtime.properties b/examples/config/global.runtime.properties new file mode 100644 index 00000000000..bc9058849d8 --- /dev/null +++ b/examples/config/global.runtime.properties @@ -0,0 +1,24 @@ +# Extensions +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.116","io.druid.extensions:druid-kafka-seven:0.6.116","io.druid.extensions:druid-rabbitmq:0.6.116", "io.druid.extensions:druid-s3-extensions:0.6.116"] + +# 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 +druid.emitter=noop \ No newline at end of file diff --git a/examples/config/historical/runtime.properties b/examples/config/historical/runtime.properties index 51d19e89af6..dc927f4de1e 100644 --- a/examples/config/historical/runtime.properties +++ b/examples/config/historical/runtime.properties @@ -2,8 +2,6 @@ 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.116"] # Dummy read only AWS account (used to download example data) diff --git a/examples/config/overlord/runtime.properties b/examples/config/overlord/runtime.properties index 24745c3147f..dc67d60adaf 100644 --- a/examples/config/overlord/runtime.properties +++ b/examples/config/overlord/runtime.properties @@ -1,22 +1,8 @@ --server --Xmx256m --Duser.timezone=UTC --Dfile.encoding=UTF-8 +druid.host=localhost +druid.port=8083 +druid.service=overlord --Ddruid.host=localhost --Ddruid.port=8080 --Ddruid.service=overlord - --Ddruid.zk.service.host=localhost - --Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.116"] - --Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid --Ddruid.db.connector.user=druid --Ddruid.db.connector.password=diurd - --Ddruid.selectors.indexing.serviceName=overlord --Ddruid.indexer.queue.startDelay=PT0M --Ddruid.indexer.runner.javaOpts="-server -Xmx256m" --Ddruid.indexer.fork.property.druid.processing.numThreads=1 --Ddruid.indexer.fork.property.druid.computation.buffer.size=100000000 \ No newline at end of file +druid.indexer.queue.startDelay=PT0M +druid.indexer.runner.javaOpts="-server -Xmx256m" +druid.indexer.fork.property.druid.processing.numThreads=1 +druid.indexer.fork.property.druid.computation.buffer.size=100000000 \ No newline at end of file diff --git a/examples/config/realtime/runtime.properties b/examples/config/realtime/runtime.properties index b78547fe8ab..def7470a99b 100644 --- a/examples/config/realtime/runtime.properties +++ b/examples/config/realtime/runtime.properties @@ -2,19 +2,8 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.zk.service.host=localhost - -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.116","io.druid.extensions:druid-kafka-seven:0.6.116","io.druid.extensions:druid-rabbitmq:0.6.116"] - # 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"] diff --git a/pom.xml b/pom.xml index fd02b96820e..30832917f0c 100644 --- a/pom.xml +++ b/pom.xml @@ -73,7 +73,7 @@ com.metamx emitter - 0.2.11 + 0.2.12 com.metamx 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..721e65d10ef --- /dev/null +++ b/server/src/main/java/io/druid/server/initialization/NoopEmitterModule.java @@ -0,0 +1,29 @@ +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(); + } +} From 5380f14905093f1e20e47df741129c342d6f3704 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 9 Jun 2014 16:13:57 -0700 Subject: [PATCH 005/107] add header --- .../initialization/NoopEmitterModule.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/server/src/main/java/io/druid/server/initialization/NoopEmitterModule.java b/server/src/main/java/io/druid/server/initialization/NoopEmitterModule.java index 721e65d10ef..f874d27021e 100644 --- a/server/src/main/java/io/druid/server/initialization/NoopEmitterModule.java +++ b/server/src/main/java/io/druid/server/initialization/NoopEmitterModule.java @@ -1,3 +1,22 @@ +/* + * 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; From a7e19ad892884996b77afc35e85cdc3ea7bfe9ae Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Thu, 12 Jun 2014 19:32:37 +0530 Subject: [PATCH 006/107] configure buffer sizes --- .../io/druid/indexer/IndexGeneratorJob.java | 8 +++++-- .../io/druid/indexing/common/TaskToolbox.java | 8 +------ .../indexing/common/TaskToolboxFactory.java | 5 +--- .../common/index/YeOldePlumberSchool.java | 10 +++----- .../indexing/common/task/DeleteTask.java | 3 ++- .../druid/indexing/common/task/IndexTask.java | 3 +-- .../common/task/RealtimeIndexTask.java | 1 - .../groupby/GroupByQueryRunnerFactory.java | 15 ++++++++---- .../groupby/having/GreaterThanHavingSpec.java | 1 - .../segment/incremental/IncrementalIndex.java | 3 +++ .../realtime/plumber/FlushingPlumber.java | 6 ++--- .../plumber/FlushingPlumberSchool.java | 8 +------ .../realtime/plumber/RealtimePlumber.java | 10 +++----- .../plumber/RealtimePlumberSchool.java | 15 +----------- .../druid/segment/realtime/plumber/Sink.java | 24 ++++++++----------- .../segment/realtime/FireDepartmentTest.java | 2 +- .../segment/realtime/RealtimeManagerTest.java | 2 +- .../plumber/RealtimePlumberSchoolTest.java | 1 - .../segment/realtime/plumber/SinkTest.java | 4 ++-- 19 files changed, 49 insertions(+), 80 deletions(-) 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 d867dc8fe9a..cb15480da03 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -617,7 +617,11 @@ public class IndexGeneratorJob implements Jobby private IncrementalIndex makeIncrementalIndex(Bucket theBucket, AggregatorFactory[] aggs) { - //TODO: review this, add a config for batch ingestion + int aggsSize = 0; + for (AggregatorFactory agg : aggs) { + aggsSize += agg.getMaxIntermediateSize(); + } + int bufferSize = aggsSize * config.getSchema().getTuningConfig().getRowFlushBoundary(); return new IncrementalIndex( new IncrementalIndexSchema.Builder() .withMinTimestamp(theBucket.time.getMillis()) @@ -625,7 +629,7 @@ public class IndexGeneratorJob implements Jobby .withQueryGranularity(config.getSchema().getDataSchema().getGranularitySpec().getQueryGranularity()) .withMetrics(aggs) .build(), - new OffheapBufferPool(1024 * 1024 * 1024) + new OffheapBufferPool(bufferSize) ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java index 944c5b19ea1..0b19d9fd5e4 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java @@ -75,7 +75,6 @@ public class TaskToolbox private final SegmentLoader segmentLoader; private final ObjectMapper objectMapper; private final File taskWorkDir; - private final StupidPool indexPool; public TaskToolbox( TaskConfig config, @@ -93,8 +92,7 @@ public class TaskToolbox MonitorScheduler monitorScheduler, SegmentLoader segmentLoader, ObjectMapper objectMapper, - final File taskWorkDir, - StupidPool indexPool + final File taskWorkDir ) { this.config = config; @@ -113,7 +111,6 @@ public class TaskToolbox this.segmentLoader = segmentLoader; this.objectMapper = objectMapper; this.taskWorkDir = taskWorkDir; - this.indexPool = indexPool; } public TaskConfig getConfig() @@ -216,7 +213,4 @@ public class TaskToolbox return taskWorkDir; } - public StupidPool getIndexPool(){ - return indexPool; - } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java index b950959b473..fd9365dc05d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java @@ -60,7 +60,6 @@ public class TaskToolboxFactory private final MonitorScheduler monitorScheduler; private final SegmentLoaderFactory segmentLoaderFactory; private final ObjectMapper objectMapper; - private final StupidPool bufferPool; @Inject public TaskToolboxFactory( @@ -96,7 +95,6 @@ public class TaskToolboxFactory this.monitorScheduler = monitorScheduler; this.segmentLoaderFactory = segmentLoaderFactory; this.objectMapper = objectMapper; - this.bufferPool = bufferPool; } public TaskToolbox build(Task task) @@ -119,8 +117,7 @@ public class TaskToolboxFactory monitorScheduler, segmentLoaderFactory.manufacturate(taskWorkDir), objectMapper, - taskWorkDir, - bufferPool + taskWorkDir ); } } 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 de3aa720219..9ed0fc1e062 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 @@ -70,7 +70,6 @@ public class YeOldePlumberSchool implements PlumberSchool private final String version; private final DataSegmentPusher dataSegmentPusher; private final File tmpSegmentDir; - private final StupidPool bufferPool; private static final Logger log = new Logger(YeOldePlumberSchool.class); @@ -79,16 +78,13 @@ public class YeOldePlumberSchool implements PlumberSchool @JsonProperty("interval") Interval interval, @JsonProperty("version") String version, @JacksonInject("segmentPusher") DataSegmentPusher dataSegmentPusher, - @JacksonInject("tmpSegmentDir") File tmpSegmentDir, - //TODO: review this global annotation - @JacksonInject @Global StupidPool bufferPool - ) + @JacksonInject("tmpSegmentDir") File tmpSegmentDir + ) { this.interval = interval; this.version = version; this.dataSegmentPusher = dataSegmentPusher; this.tmpSegmentDir = tmpSegmentDir; - this.bufferPool = bufferPool; } @Override @@ -105,7 +101,7 @@ public class YeOldePlumberSchool implements PlumberSchool ) { // There can be only one. - final Sink theSink = new Sink(interval, schema, config, version, bufferPool); + final Sink theSink = new Sink(interval, schema, config, version); // Temporary directory to hold spilled segments. final File persistDir = new File(tmpSegmentDir, theSink.getSegment().getIdentifier()); 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 index 90d6eff2775..d5995f9756f 100644 --- 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 @@ -32,6 +32,7 @@ import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.SegmentInsertAction; +import io.druid.offheap.OffheapBufferPool; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.IndexMerger; import io.druid.segment.IndexableAdapter; @@ -83,7 +84,7 @@ public class DeleteTask extends AbstractFixedIntervalTask 0, QueryGranularity.NONE, new AggregatorFactory[0], - toolbox.getIndexPool() + new OffheapBufferPool(0) ); try { final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(getInterval(), empty); 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 d4cd8ea0bb2..32565f1d51a 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 @@ -385,8 +385,7 @@ public class IndexTask extends AbstractFixedIntervalTask interval, version, wrappedDataSegmentPusher, - tmpDir, - toolbox.getIndexPool() + tmpDir ).findPlumber(schema, new RealtimeTuningConfig(null, null, null, null, null, null, null, shardSpec), metrics); // rowFlushBoundary for this job 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 bfc6b35fc3f..e0fc20d1216 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 @@ -308,7 +308,6 @@ public class RealtimeIndexTask extends AbstractTask segmentPublisher, toolbox.getNewSegmentServerView(), toolbox.getQueryExecutorService(), - toolbox.getIndexPool(), null, null, null, 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 a498a162965..e603a1997f4 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java @@ -54,20 +54,21 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory config; private final GroupByQueryQueryToolChest toolChest; - private final StupidPool bufferPool; + @Global + StupidPool computationBufferPool; @Inject public GroupByQueryRunnerFactory( GroupByQueryEngine engine, Supplier config, GroupByQueryQueryToolChest toolChest, - @Global StupidPool bufferPool + @Global StupidPool computationBufferPool ) { this.engine = engine; this.config = config; this.toolChest = toolChest; - this.bufferPool = bufferPool; + this.computationBufferPool = computationBufferPool; } @Override @@ -123,7 +124,13 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory 0; } 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 52106ae8a0c..fdf772db72a 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -297,6 +297,9 @@ public class IncrementalIndex implements Iterable, Closeable synchronized (this) { if (!facts.containsKey(key)) { int rowOffset = totalAggSize * numEntries.getAndIncrement(); + if (rowOffset + totalAggSize > bufferHolder.get().limit()) { + throw new ISE("Buffer Full cannot add more rows current rowSize : %d", numEntries.get()); + } for (int i = 0; i < aggs.length; i++) { aggs[i].init(bufferHolder.get(), getMetricPosition(rowOffset, i)); } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java index 30babfd57ec..9fc23be052f 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java @@ -64,8 +64,7 @@ public class FlushingPlumber extends RealtimePlumber ServiceEmitter emitter, QueryRunnerFactoryConglomerate conglomerate, DataSegmentAnnouncer segmentAnnouncer, - ExecutorService queryExecutorService, - StupidPool bufferPool + ExecutorService queryExecutorService ) { super( @@ -78,8 +77,7 @@ public class FlushingPlumber extends RealtimePlumber queryExecutorService, null, null, - null, - bufferPool + null ); this.flushDuration = flushDuration; diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java index f7de3ff21bd..e5a57461f97 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java @@ -54,7 +54,6 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool private final QueryRunnerFactoryConglomerate conglomerate; private final DataSegmentAnnouncer segmentAnnouncer; private final ExecutorService queryExecutorService; - private final StupidPool bufferPool; @JsonCreator public FlushingPlumberSchool( @@ -63,8 +62,6 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool @JacksonInject QueryRunnerFactoryConglomerate conglomerate, @JacksonInject DataSegmentAnnouncer segmentAnnouncer, @JacksonInject @Processing ExecutorService queryExecutorService, - //TODO: define separate index pool - @JacksonInject @Global StupidPool bufferPool, // Backwards compatible @JsonProperty("windowPeriod") Period windowPeriod, @JsonProperty("basePersistDirectory") File basePersistDirectory, @@ -82,7 +79,6 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool null, null, queryExecutorService, - bufferPool, windowPeriod, basePersistDirectory, segmentGranularity, @@ -96,7 +92,6 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool this.conglomerate = conglomerate; this.segmentAnnouncer = segmentAnnouncer; this.queryExecutorService = queryExecutorService; - this.bufferPool = bufferPool; } @Override @@ -116,8 +111,7 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool emitter, conglomerate, segmentAnnouncer, - queryExecutorService, - bufferPool + queryExecutorService ); } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 26b4cc45509..337149aadac 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -99,7 +99,6 @@ public class RealtimePlumber implements Plumber private volatile ExecutorService persistExecutor = null; private volatile ExecutorService mergeExecutor = null; private volatile ScheduledExecutorService scheduledExecutor = null; - private final StupidPool bufferPool; public RealtimePlumber( @@ -112,9 +111,7 @@ public class RealtimePlumber implements Plumber ExecutorService queryExecutorService, DataSegmentPusher dataSegmentPusher, SegmentPublisher segmentPublisher, - FilteredServerView serverView, - StupidPool bufferPool - + FilteredServerView serverView ) { this.schema = schema; @@ -128,7 +125,6 @@ public class RealtimePlumber implements Plumber this.dataSegmentPusher = dataSegmentPusher; this.segmentPublisher = segmentPublisher; this.serverView = serverView; - this.bufferPool = bufferPool; log.info("Creating plumber using rejectionPolicy[%s]", getRejectionPolicy()); } @@ -193,7 +189,7 @@ public class RealtimePlumber implements Plumber segmentGranularity.increment(new DateTime(truncatedTime)) ); - retVal = new Sink(sinkInterval, schema, config, versioningPolicy.getVersion(sinkInterval), bufferPool); + retVal = new Sink(sinkInterval, schema, config, versioningPolicy.getVersion(sinkInterval)); try { segmentAnnouncer.announceSegment(retVal.getSegment()); @@ -544,7 +540,7 @@ public class RealtimePlumber implements Plumber ); } - Sink currSink = new Sink(sinkInterval, schema, config, versioningPolicy.getVersion(sinkInterval), hydrants, bufferPool); + Sink currSink = new Sink(sinkInterval, schema, config, versioningPolicy.getVersion(sinkInterval), hydrants); sinks.put(sinkInterval.getStartMillis(), currSink); sinkTimeline.add( currSink.getInterval(), diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java index 2b24fcf4279..11ebf015b11 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java @@ -23,14 +23,9 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import com.google.common.base.Supplier; import com.metamx.common.Granularity; -import com.metamx.common.logger.Logger; import com.metamx.emitter.service.ServiceEmitter; import io.druid.client.FilteredServerView; -import io.druid.client.ServerView; -import io.druid.collections.StupidPool; -import io.druid.guice.annotations.Global; import io.druid.guice.annotations.Processing; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.indexing.DataSchema; @@ -42,9 +37,7 @@ import io.druid.server.coordination.DataSegmentAnnouncer; import org.joda.time.Period; import java.io.File; -import java.nio.ByteBuffer; import java.util.concurrent.ExecutorService; -import java.util.concurrent.atomic.AtomicLong; /** */ @@ -57,8 +50,6 @@ public class RealtimePlumberSchool implements PlumberSchool private final SegmentPublisher segmentPublisher; private final FilteredServerView serverView; private final ExecutorService queryExecutorService; - private final StupidPool bufferPool; - // Backwards compatible private final Period windowPeriod; private final File basePersistDirectory; @@ -76,8 +67,6 @@ public class RealtimePlumberSchool implements PlumberSchool @JacksonInject SegmentPublisher segmentPublisher, @JacksonInject FilteredServerView serverView, @JacksonInject @Processing ExecutorService executorService, - //TODO: define separate index pool - @JacksonInject @Global StupidPool bufferPool, // Backwards compatible @JsonProperty("windowPeriod") Period windowPeriod, @JsonProperty("basePersistDirectory") File basePersistDirectory, @@ -100,7 +89,6 @@ public class RealtimePlumberSchool implements PlumberSchool this.versioningPolicy = versioningPolicy; this.rejectionPolicyFactory = rejectionPolicyFactory; this.maxPendingPersists = maxPendingPersists; - this.bufferPool = bufferPool; } @Deprecated @@ -159,8 +147,7 @@ public class RealtimePlumberSchool implements PlumberSchool queryExecutorService, dataSegmentPusher, segmentPublisher, - serverView, - bufferPool + serverView ); } 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 4dc1edf61b8..57db2201d87 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 @@ -29,8 +29,7 @@ import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import io.druid.collections.StupidPool; import io.druid.data.input.InputRow; -import io.druid.data.input.impl.SpatialDimensionSchema; -import io.druid.guice.annotations.Global; +import io.druid.offheap.OffheapBufferPool; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; @@ -52,31 +51,25 @@ 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 Interval interval; private final DataSchema schema; private final RealtimeTuningConfig config; private final String version; private final CopyOnWriteArrayList hydrants = new CopyOnWriteArrayList(); - private final StupidPool bufferPool; + private volatile FireHydrant currHydrant; public Sink( Interval interval, DataSchema schema, RealtimeTuningConfig config, - String version, - StupidPool bufferPool - + String version ) { this.schema = schema; this.config = config; this.interval = interval; this.version = version; - this.bufferPool = bufferPool; makeNewCurrIndex(interval.getStartMillis(), schema); } @@ -86,15 +79,13 @@ public class Sink implements Iterable DataSchema schema, RealtimeTuningConfig config, String version, - List hydrants, - StupidPool bufferPool + List hydrants ) { this.schema = schema; this.config = config; this.interval = interval; this.version = version; - this.bufferPool = bufferPool; for (int i = 0; i < hydrants.size(); ++i) { final FireHydrant hydrant = hydrants.get(i); @@ -187,6 +178,11 @@ public class Sink implements Iterable private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) { + int aggsSize = 0; + for (AggregatorFactory agg : schema.getAggregators()) { + aggsSize += agg.getMaxIntermediateSize(); + } + int bufferSize = aggsSize * config.getMaxRowsInMemory(); IncrementalIndex newIndex = new IncrementalIndex( new IncrementalIndexSchema.Builder() .withMinTimestamp(minTimestamp) @@ -194,7 +190,7 @@ public class Sink implements Iterable .withSpatialDimensions(schema.getParser()) .withMetrics(schema.getAggregators()) .build(), - bufferPool + new OffheapBufferPool(bufferSize) ); FireHydrant old; 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 1566013194c..5817ed6a9ac 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -74,7 +74,7 @@ public class FireDepartmentTest new RealtimeIOConfig( null, new RealtimePlumberSchool( - null, null, null, null, null, null, null, TestQueryRunners.pool, null, null, null, null, null, 0 + null, null, null, null, null, null, null, null, null, null, null, null, 0 ) ), new RealtimeTuningConfig( diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index 4a1f692a3f4..7abe823594e 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -118,7 +118,7 @@ public class RealtimeManagerTest null, null ); - plumber = new TestPlumber(new Sink(new Interval("0/P5000Y"), schema, tuningConfig, new DateTime().toString(), TestQueryRunners.pool)); + plumber = new TestPlumber(new Sink(new Interval("0/P5000Y"), schema, tuningConfig, new DateTime().toString())); realtimeManager = new RealtimeManager( Arrays.asList( 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 f2a060ea1cc..bab7a549b52 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 @@ -145,7 +145,6 @@ public class RealtimePlumberSchoolTest segmentPublisher, serverView, MoreExecutors.sameThreadExecutor(), - TestQueryRunners.pool, new Period("PT10m"), tmpDir, Granularity.HOUR, diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index e50ac4403dc..0e84660f682 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -56,7 +56,7 @@ public class SinkTest final Interval interval = new Interval("2013-01-01/2013-01-02"); final String version = new DateTime().toString(); RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( - 1, + 100, new Period("P1Y"), null, null, @@ -65,7 +65,7 @@ public class SinkTest null, null ); - final Sink sink = new Sink(interval, schema, tuningConfig, version, TestQueryRunners.pool); + final Sink sink = new Sink(interval, schema, tuningConfig, version); sink.add( new InputRow() From 0ddaf4c307156783dd521098ad869f473e675890 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Fri, 13 Jun 2014 15:36:57 +0530 Subject: [PATCH 007/107] fix race while index closing and querying --- .../common/index/YeOldePlumberSchool.java | 1 - .../query/GroupByParallelQueryRunner.java | 8 +++-- .../segment/IncrementalIndexSegment.java | 2 +- .../druid/segment/realtime/FireHydrant.java | 26 ++++++++++---- .../realtime/plumber/RealtimePlumber.java | 36 +++++++++++-------- 5 files changed, 47 insertions(+), 26 deletions(-) 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 9ed0fc1e062..a34a24b4550 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 @@ -223,7 +223,6 @@ public class YeOldePlumberSchool implements PlumberSchool ); indexToPersist.swapSegment(null); - index.close(); metrics.incrementRowOutputCount(rowsToPersist); spilled.add(dirToPersist); diff --git a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java index 070f54dace5..705e8eabc4f 100644 --- a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java @@ -63,7 +63,6 @@ public class GroupByParallelQueryRunner implements QueryRunner private final QueryWatcher queryWatcher; private final StupidPool bufferPool; - public GroupByParallelQueryRunner( ExecutorService exec, Ordering ordering, @@ -155,17 +154,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()); } @@ -173,7 +176,8 @@ public class GroupByParallelQueryRunner implements QueryRunner Sequences.simple( indexAccumulatorPair.lhs .iterableWithPostAggregations(null) - ), indexAccumulatorPair.lhs + ), + indexAccumulatorPair.lhs ); } 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/server/src/main/java/io/druid/segment/realtime/FireHydrant.java b/server/src/main/java/io/druid/segment/realtime/FireHydrant.java index db99ff45719..282ef63b624 100644 --- a/server/src/main/java/io/druid/segment/realtime/FireHydrant.java +++ b/server/src/main/java/io/druid/segment/realtime/FireHydrant.java @@ -19,17 +19,21 @@ package io.druid.segment.realtime; +import com.google.api.client.repackaged.com.google.common.base.Throwables; import io.druid.segment.IncrementalIndexSegment; +import io.druid.segment.ReferenceCountingSegment; import io.druid.segment.Segment; import io.druid.segment.incremental.IncrementalIndex; +import java.io.IOException; + /** -*/ + */ public class FireHydrant { - private volatile IncrementalIndex index; - private volatile Segment adapter; private final int count; + private volatile IncrementalIndex index; + private volatile ReferenceCountingSegment adapter; public FireHydrant( IncrementalIndex index, @@ -38,7 +42,7 @@ public class FireHydrant ) { this.index = index; - this.adapter = new IncrementalIndexSegment(index, segmentIdentifier); + this.adapter = new ReferenceCountingSegment(new IncrementalIndexSegment(index, segmentIdentifier)); this.count = count; } @@ -48,7 +52,7 @@ public class FireHydrant ) { this.index = null; - this.adapter = adapter; + this.adapter = new ReferenceCountingSegment(adapter); this.count = count; } @@ -57,7 +61,7 @@ public class FireHydrant return index; } - public Segment getSegment() + public ReferenceCountingSegment getSegment() { return adapter; } @@ -74,7 +78,15 @@ public class FireHydrant public void swapSegment(Segment adapter) { - this.adapter = adapter; + if (this.adapter != null) { + try { + this.adapter.close(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + this.adapter = new ReferenceCountingSegment(adapter); this.index = null; } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 337149aadac..0b46de25de2 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -19,12 +19,10 @@ import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.client.FilteredServerView; import io.druid.client.ServerView; -import io.druid.collections.StupidPool; import io.druid.common.guava.ThreadRenamingCallable; import io.druid.common.guava.ThreadRenamingRunnable; import io.druid.concurrent.Execs; import io.druid.data.input.InputRow; -import io.druid.guice.annotations.Global; import io.druid.query.MetricsEmittingQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; @@ -59,10 +57,10 @@ import org.joda.time.Interval; import org.joda.time.Period; import javax.annotation.Nullable; +import java.io.Closeable; import java.io.File; import java.io.FilenameFilter; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Comparator; import java.util.List; @@ -76,10 +74,8 @@ import java.util.concurrent.ScheduledExecutorService; public class RealtimePlumber implements Plumber { private static final EmittingLogger log = new EmittingLogger(RealtimePlumber.class); - private final DataSchema schema; private final RealtimeTuningConfig config; - private final RejectionPolicy rejectionPolicy; private final FireDepartmentMetrics metrics; private final ServiceEmitter emitter; @@ -253,7 +249,19 @@ public class RealtimePlumber implements Plumber @Override public QueryRunner 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); + } + } } } ) @@ -717,8 +725,6 @@ public class RealtimePlumber implements Plumber IndexIO.loadIndex(persistedFile) ) ); - //TODO: can there be some races here ? - index.close(); return numRows; } catch (IOException e) { @@ -785,13 +791,13 @@ public class RealtimePlumber implements Plumber && config.getShardSpec().getPartitionNum() == segment.getShardSpec().getPartitionNum() && Iterables.any( sinks.keySet(), new Predicate() - { - @Override - public boolean apply(Long sinkKey) - { - return segment.getInterval().contains(sinkKey); - } - } + { + @Override + public boolean apply(Long sinkKey) + { + return segment.getInterval().contains(sinkKey); + } + } ); } } From a9c09ec8a71ce543e76cc15e8ed5ac56e55698e2 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Mon, 16 Jun 2014 17:55:43 +0530 Subject: [PATCH 008/107] fix compilation --- .../io/druid/segment/incremental/IncrementalIndex.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) 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 fdf772db72a..0c16ea7d81e 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -21,6 +21,7 @@ package io.druid.segment.incremental; import com.google.common.base.Function; import com.google.common.base.Joiner; +import com.google.common.base.Throwables; import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; import com.google.common.collect.ImmutableList; @@ -504,9 +505,14 @@ public class IncrementalIndex implements Iterable, Closeable } @Override - public void close() throws IOException + public void close() { - bufferHolder.close(); + try { + bufferHolder.close(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } } static class DimensionHolder From 0629be622cd2378af8fd22b631c833f961a908e1 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Mon, 16 Jun 2014 18:50:49 +0530 Subject: [PATCH 009/107] remove unnecessary changes & fix index closing subquery --- .../io/druid/indexing/common/TaskToolbox.java | 3 - .../indexing/common/TaskToolboxFactory.java | 7 +- .../common/index/YeOldePlumberSchool.java | 10 +- .../indexing/overlord/TaskLifecycleTest.java | 4 +- .../worker/WorkerTaskMonitorTest.java | 4 +- .../GroupByParallelQueryRunner.java.orig | 195 ------------------ .../groupby/GroupByQueryQueryToolChest.java | 14 +- .../groupby/having/GreaterThanHavingSpec.java | 1 + .../HyperUniqueBufferAggregatorTest.java | 93 --------- .../realtime/plumber/FlushingPlumber.java | 2 - .../plumber/FlushingPlumberSchool.java | 3 - .../realtime/plumber/RealtimePlumber.java | 1 - .../plumber/RealtimePlumberSchool.java | 3 +- .../segment/realtime/FireDepartmentTest.java | 3 +- .../segment/realtime/RealtimeManagerTest.java | 3 +- .../plumber/RealtimePlumberSchoolTest.java | 3 +- .../segment/realtime/plumber/SinkTest.java | 1 - 17 files changed, 20 insertions(+), 330 deletions(-) delete mode 100644 processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java.orig delete mode 100644 processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniqueBufferAggregatorTest.java diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java index 0b19d9fd5e4..5eb50b622c6 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java @@ -28,7 +28,6 @@ import com.google.common.collect.Multimaps; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; import io.druid.client.FilteredServerView; -import io.druid.collections.StupidPool; import io.druid.indexing.common.actions.SegmentInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.actions.TaskActionClientFactory; @@ -47,7 +46,6 @@ import org.joda.time.Interval; import java.io.File; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.Collection; import java.util.List; import java.util.Map; @@ -212,5 +210,4 @@ public class TaskToolbox { return taskWorkDir; } - } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java index fd9365dc05d..89d275b9a7f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java @@ -24,8 +24,6 @@ import com.google.inject.Inject; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; import io.druid.client.FilteredServerView; -import io.druid.collections.StupidPool; -import io.druid.guice.annotations.Global; import io.druid.guice.annotations.Processing; import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.config.TaskConfig; @@ -38,7 +36,6 @@ import io.druid.segment.loading.DataSegmentPusher; import io.druid.server.coordination.DataSegmentAnnouncer; import java.io.File; -import java.nio.ByteBuffer; import java.util.concurrent.ExecutorService; /** @@ -76,9 +73,7 @@ public class TaskToolboxFactory @Processing ExecutorService queryExecutorService, MonitorScheduler monitorScheduler, SegmentLoaderFactory segmentLoaderFactory, - ObjectMapper objectMapper, - //TODO: have a separate index pool - @Global StupidPool bufferPool + ObjectMapper objectMapper ) { this.config = config; 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 a34a24b4550..11f6bb2264d 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 @@ -31,16 +31,13 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.Granularity; import com.metamx.common.logger.Logger; -import io.druid.collections.StupidPool; import io.druid.data.input.InputRow; -import io.druid.guice.annotations.Global; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; import io.druid.segment.SegmentUtils; -import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.loading.DataSegmentPusher; @@ -52,11 +49,9 @@ import io.druid.segment.realtime.plumber.Sink; import io.druid.timeline.DataSegment; import org.apache.commons.io.FileUtils; import org.joda.time.Interval; -import sun.misc.JavaNioAccess; import java.io.File; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.List; import java.util.Set; @@ -79,7 +74,7 @@ public class YeOldePlumberSchool implements PlumberSchool @JsonProperty("version") String version, @JacksonInject("segmentPusher") DataSegmentPusher dataSegmentPusher, @JacksonInject("tmpSegmentDir") File tmpSegmentDir - ) + ) { this.interval = interval; this.version = version; @@ -216,13 +211,14 @@ public class YeOldePlumberSchool implements PlumberSchool log.info("Spilling index[%d] with rows[%d] to: %s", indexToPersist.getCount(), rowsToPersist, dirToPersist); try { - final IncrementalIndex index = indexToPersist.getIndex(); + IndexMerger.persist( indexToPersist.getIndex(), dirToPersist ); indexToPersist.swapSegment(null); + metrics.incrementRowOutputCount(rowsToPersist); spilled.add(dirToPersist); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 9068535f38a..30325c9d398 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -43,7 +43,6 @@ import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.InputRowParser; import io.druid.granularity.QueryGranularity; -import io.druid.offheap.OffheapBufferPool; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskLock; @@ -206,8 +205,7 @@ public class TaskLifecycleTest } ) ), - new DefaultObjectMapper(), - new OffheapBufferPool(1024 * 1024) + new DefaultObjectMapper() ); tr = new ThreadPoolTaskRunner(tb); tq = new TaskQueue(tqc, ts, tr, tac, tl, emitter); diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index 4f88488e436..8d4bf32b870 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -38,7 +38,6 @@ import io.druid.indexing.overlord.TestRemoteTaskRunnerConfig; import io.druid.indexing.overlord.ThreadPoolTaskRunner; import io.druid.indexing.worker.config.WorkerConfig; import io.druid.jackson.DefaultObjectMapper; -import io.druid.offheap.OffheapBufferPool; import io.druid.segment.loading.DataSegmentPuller; import io.druid.segment.loading.LocalDataSegmentPuller; import io.druid.segment.loading.OmniSegmentLoader; @@ -139,8 +138,7 @@ public class WorkerTaskMonitorTest } } ) - ), jsonMapper, - new OffheapBufferPool(1024 * 1024) + ), jsonMapper ) ), new WorkerConfig().setCapacity(1) diff --git a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java.orig b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java.orig deleted file mode 100644 index b2cf4965b22..00000000000 --- a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java.orig +++ /dev/null @@ -1,195 +0,0 @@ -/* - * 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; - -import com.google.common.base.Function; -import com.google.common.base.Predicates; -import com.google.common.base.Supplier; -import com.google.common.base.Throwables; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.collect.Ordering; -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.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.Arrays; -import java.util.List; -import java.util.concurrent.CancellationException; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - - -public class GroupByParallelQueryRunner implements QueryRunner -{ - private static final Logger log = new Logger(GroupByParallelQueryRunner.class); - private final Iterable> queryables; - private final ListeningExecutorService exec; - private final Ordering ordering; - private final Supplier configSupplier; -<<<<<<< HEAD - private final StupidPool bufferPool; -======= - private final QueryWatcher queryWatcher; ->>>>>>> master - - - public GroupByParallelQueryRunner( - ExecutorService exec, - Ordering ordering, - Supplier configSupplier, -<<<<<<< HEAD - StupidPool bufferPool, - QueryRunner... queryables - ) - { - this(exec, ordering, configSupplier, bufferPool, Arrays.asList(queryables)); -======= - QueryWatcher queryWatcher, - QueryRunner... queryables - ) - { - this(exec, ordering, configSupplier, queryWatcher, Arrays.asList(queryables)); ->>>>>>> master - } - - public GroupByParallelQueryRunner( - ExecutorService exec, -<<<<<<< HEAD - Ordering ordering, - Supplier configSupplier, - StupidPool bufferPool, -======= - Ordering ordering, Supplier configSupplier, - QueryWatcher queryWatcher, ->>>>>>> master - Iterable> queryables - ) - { - this.exec = MoreExecutors.listeningDecorator(exec); - this.ordering = ordering; - 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) - { - - final GroupByQuery query = (GroupByQuery) queryParam; - final Pair> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair( - query, - configSupplier.get(), - bufferPool - ); - final int priority = query.getContextPriority(0); - - if (Iterables.isEmpty(queryables)) { - log.warn("No queryables found."); - } - ListenableFuture> futures = Futures.allAsList( - Lists.newArrayList( - Iterables.transform( - queryables, - new Function, ListenableFuture>() - { - @Override - public ListenableFuture apply(final QueryRunner input) - { - return exec.submit( - new AbstractPrioritizedCallable(priority) - { - @Override - public Boolean call() throws Exception - { - try { - input.run(queryParam).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); - return true; - } - catch (QueryInterruptedException e) { - throw Throwables.propagate(e); - } - catch (Exception e) { - log.error(e, "Exception with one of the sequences!"); - throw Throwables.propagate(e); - } - } - } - ); - } - } - ) - ) - ); - - // Let the runners complete - try { - queryWatcher.registerQuery(query, futures); - final Number timeout = query.getContextValue("timeout", (Number) null); - if(timeout == null) { - futures.get(); - } else { - futures.get(timeout.longValue(), TimeUnit.MILLISECONDS); - } - } - catch (InterruptedException e) { - log.warn(e, "Query interrupted, cancelling pending results, query id [%s]", query.getId()); - futures.cancel(true); - throw new QueryInterruptedException("Query interrupted"); - } - catch(CancellationException e) { - throw new QueryInterruptedException("Query cancelled"); - } - catch(TimeoutException e) { - log.info("Query timeout, cancelling pending results for query id [%s]", query.getId()); - futures.cancel(true); - throw new QueryInterruptedException("Query timeout"); - } - catch (ExecutionException e) { - throw Throwables.propagate(e.getCause()); - } - - return new ResourceClosingSequence( - Sequences.simple( - indexAccumulatorPair.lhs - .iterableWithPostAggregations(null) - ), indexAccumulatorPair.lhs - ); - } - -} 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 6a5845b3ef1..ff000dd0a0a 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -103,9 +103,9 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeGroupByResults(final GroupByQuery query, QueryRunner runner) { - Sequence result; // If there's a subquery, merge subquery results and then apply the aggregator DataSource dataSource = query.getDataSource(); + final IncrementalIndex index; if (dataSource instanceof QueryDataSource) { GroupByQuery subquery; try { @@ -115,13 +115,15 @@ public class GroupByQueryQueryToolChest extends QueryToolChest subqueryResult = mergeGroupByResults(subquery, runner); - IncrementalIndexStorageAdapter adapter - = new IncrementalIndexStorageAdapter(makeIncrementalIndex(subquery, subqueryResult)); - result = engine.process(query, adapter); + final IncrementalIndex subQueryResultIndex = makeIncrementalIndex(subquery, subqueryResult); + + Sequence result = engine.process(query, new IncrementalIndexStorageAdapter(subQueryResultIndex)); + index = makeIncrementalIndex(query, result); + subQueryResultIndex.close(); } else { - result = runner.run(query); + index = makeIncrementalIndex(query, runner.run(query)); + } - final IncrementalIndex index = makeIncrementalIndex(query, result); return new ResourceClosingSequence(postAggregate(query, index), index); } diff --git a/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java index ba3f18c9dcd..0c2c9e7810b 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java @@ -58,6 +58,7 @@ public class GreaterThanHavingSpec implements HavingSpec public boolean eval(Row row) { float metricValue = row.getFloatMetric(aggregationName); + return Float.compare(metricValue, value.floatValue()) > 0; } diff --git a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniqueBufferAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniqueBufferAggregatorTest.java deleted file mode 100644 index d2d291ac3b9..00000000000 --- a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniqueBufferAggregatorTest.java +++ /dev/null @@ -1,93 +0,0 @@ -package io.druid.query.aggregation.hyperloglog; - -import com.google.common.hash.HashFunction; -import com.google.common.hash.Hashing; -import io.druid.segment.ObjectColumnSelector; -import org.junit.Test; - -import java.nio.ByteBuffer; - -/** - * Created with IntelliJ IDEA. - * User: neo - * Date: 05/06/14 - * Time: 3:14 PM - * To change this template use File | Settings | File Templates. - */ -public class HyperUniqueBufferAggregatorTest -{ - private final HashFunction fn = Hashing.murmur3_128(); - private volatile HyperLogLogCollector collector; - - @Test - public void testAggregation() - { - final HyperUniquesBufferAggregator agg = new HyperUniquesBufferAggregator( - new ObjectColumnSelector() - { - @Override - public Class classOfObject() - { - return HyperLogLogCollector.class; - } - - @Override - public Object get() - { - return collector; - } - } - ); - ByteBuffer byteBuffer = ByteBuffer.allocate(HyperLogLogCollector.getLatestNumBytesForDenseStorage()); - - for (int i = 0; i < 1000; i++) { - collector = HyperLogLogCollector.makeLatestCollector(); - collector.add(fn.hashInt(i).asBytes()); - agg.aggregate(byteBuffer, 0); - } - - final HyperLogLogCollector collector = HyperLogLogCollector.makeCollector( - ((HyperLogLogCollector) agg.get( - byteBuffer, - 0 - )).toByteBuffer() - ); - System.out.println(collector.estimateCardinality()); - - } - - @Test - public void testAggregation2() - { - final HyperUniquesAggregator agg = new HyperUniquesAggregator( - "abc", - new ObjectColumnSelector() - { - @Override - public Class classOfObject() - { - return HyperLogLogCollector.class; - } - - @Override - public Object get() - { - return collector; - } - } - ); - - for (int i = 0; i < 1000; i++) { - collector = HyperLogLogCollector.makeLatestCollector(); - collector.add(fn.hashInt(i).asBytes()); - agg.aggregate(); - } - - final HyperLogLogCollector collector = HyperLogLogCollector.makeCollector( - ((HyperLogLogCollector) agg.get( - )).toByteBuffer() - ); - System.out.println(collector.estimateCardinality()); - - } -} diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java index 9fc23be052f..083c290c4cc 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java @@ -25,7 +25,6 @@ import com.metamx.common.Granularity; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; -import io.druid.collections.StupidPool; import io.druid.common.guava.ThreadRenamingCallable; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.indexing.DataSchema; @@ -35,7 +34,6 @@ import io.druid.server.coordination.DataSegmentAnnouncer; import org.joda.time.DateTime; import org.joda.time.Duration; -import java.nio.ByteBuffer; import java.util.List; import java.util.Map; import java.util.concurrent.Callable; diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java index e5a57461f97..026af6f43c8 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java @@ -25,8 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.metamx.common.Granularity; import com.metamx.emitter.service.ServiceEmitter; -import io.druid.collections.StupidPool; -import io.druid.guice.annotations.Global; import io.druid.guice.annotations.Processing; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.indexing.DataSchema; @@ -37,7 +35,6 @@ import org.joda.time.Duration; import org.joda.time.Period; import java.io.File; -import java.nio.ByteBuffer; import java.util.concurrent.ExecutorService; /** diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 0b46de25de2..4a24d0a0e83 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -718,7 +718,6 @@ public class RealtimePlumber implements Plumber indexToPersist.getIndex(), new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())) ); - IncrementalIndex index = indexToPersist.getIndex(); indexToPersist.swapSegment( new QueryableIndexSegment( indexToPersist.getSegment().getIdentifier(), diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java index 11ebf015b11..eb52a30ba31 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java @@ -26,6 +26,7 @@ import com.google.common.base.Preconditions; import com.metamx.common.Granularity; import com.metamx.emitter.service.ServiceEmitter; import io.druid.client.FilteredServerView; +import io.druid.client.ServerView; import io.druid.guice.annotations.Processing; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.indexing.DataSchema; @@ -50,6 +51,7 @@ public class RealtimePlumberSchool implements PlumberSchool private final SegmentPublisher segmentPublisher; private final FilteredServerView serverView; private final ExecutorService queryExecutorService; + // Backwards compatible private final Period windowPeriod; private final File basePersistDirectory; @@ -160,5 +162,4 @@ public class RealtimePlumberSchool implements PlumberSchool Preconditions.checkNotNull(serverView, "must specify a serverView to do this action."); Preconditions.checkNotNull(emitter, "must specify a serviceEmitter to do this action."); } - } 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 5817ed6a9ac..b4d87260150 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -27,12 +27,11 @@ import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; 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.segment.indexing.DataSchema; -import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.RealtimeIOConfig; +import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.realtime.plumber.RealtimePlumberSchool; import junit.framework.Assert; diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index 7abe823594e..39cfc481968 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -31,12 +31,11 @@ import io.druid.data.input.impl.InputRowParser; import io.druid.granularity.QueryGranularity; import io.druid.query.Query; import io.druid.query.QueryRunner; -import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.indexing.DataSchema; -import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.RealtimeIOConfig; +import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.realtime.plumber.Plumber; import io.druid.segment.realtime.plumber.PlumberSchool; 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 bab7a549b52..eef42cad4b3 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 @@ -31,16 +31,15 @@ import com.metamx.emitter.service.ServiceEmitter; import io.druid.client.FilteredServerView; import io.druid.client.ServerView; import io.druid.data.input.InputRow; -import io.druid.data.input.impl.ParseSpec; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.InputRowParser; import io.druid.data.input.impl.JSONParseSpec; +import io.druid.data.input.impl.ParseSpec; import io.druid.data.input.impl.TimestampSpec; import io.druid.granularity.QueryGranularity; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.Query; import io.druid.query.QueryRunnerFactory; -import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.indexing.DataSchema; diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index 0e84660f682..c6615128410 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -24,7 +24,6 @@ import com.google.common.collect.Lists; import com.metamx.common.Granularity; import io.druid.data.input.InputRow; 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.indexing.DataSchema; From 00856f0feca2a9bc9c00183c83eccac3b1558893 Mon Sep 17 00:00:00 2001 From: jisookim0513 Date: Wed, 18 Jun 2014 16:55:33 -0700 Subject: [PATCH 010/107] add retry feature on broker side; modify QueryRunner inferface and tests --- pom.xml | 2 +- .../main/java/io/druid/query/BaseQuery.java | 8 +- .../io/druid/query/BySegmentQueryRunner.java | 8 +- .../query/BySegmentSkippingQueryRunner.java | 11 +- .../query/ChainedExecutionQueryRunner.java | 5 +- .../io/druid/query/ConcatQueryRunner.java | 7 +- .../query/FinalizeResultsQueryRunner.java | 6 +- .../query/GroupByParallelQueryRunner.java | 5 +- .../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 | 7 +- .../ReferenceCountingSegmentQueryRunner.java | 6 +- .../druid/query/ResultMergeQueryRunner.java | 7 +- .../java/io/druid/query/RetryQueryRunner.java | 66 +++++++++ .../druid/query/RetryQueryRunnerConfig.java | 30 ++++ .../io/druid/query/SubqueryQueryRunner.java | 9 +- .../java/io/druid/query/UnionQueryRunner.java | 10 +- .../groupby/GroupByQueryQueryToolChest.java | 13 +- .../groupby/GroupByQueryRunnerFactory.java | 8 +- .../SegmentMetadataQueryRunnerFactory.java | 10 +- .../search/SearchQueryQueryToolChest.java | 9 +- .../druid/query/search/SearchQueryRunner.java | 7 +- .../select/SelectQueryRunnerFactory.java | 10 +- .../spec/SpecificSegmentQueryRunner.java | 15 +- .../druid/query/spec/SpecificSegmentSpec.java | 2 + .../TimeBoundaryQueryQueryToolChest.java | 5 +- .../TimeBoundaryQueryRunnerFactory.java | 7 +- .../timeseries/TimeseriesQueryEngine.java | 4 +- .../TimeseriesQueryRunnerFactory.java | 9 +- .../io/druid/query/topn/TopNQueryEngine.java | 6 +- .../query/topn/TopNQueryQueryToolChest.java | 9 +- .../query/topn/TopNQueryRunnerFactory.java | 9 +- .../segment/IncrementalIndexSegment.java | 1 + .../segment/NullStorageAdapterException.java | 27 ++++ .../druid/segment/QueryableIndexSegment.java | 1 + .../ChainedExecutionQueryRunnerTest.java | 15 +- .../query/groupby/GroupByQueryRunnerTest.java | 65 +++++---- .../GroupByTimeseriesQueryRunnerTest.java | 6 +- .../query/metadata/SegmentAnalyzerTest.java | 5 +- .../metadata/SegmentMetadataQueryTest.java | 6 +- .../query/search/SearchQueryRunnerTest.java | 7 +- .../query/select/SelectQueryRunnerTest.java | 20 ++- .../TimeBoundaryQueryRunnerTest.java | 9 +- .../TimeSeriesUnionQueryRunnerTest.java | 5 +- .../TimeseriesQueryRunnerBonusTest.java | 5 +- .../timeseries/TimeseriesQueryRunnerTest.java | 125 ++++++++-------- .../druid/query/topn/TopNQueryRunnerTest.java | 134 +++++++++--------- .../druid/query/topn/TopNUnionQueryTest.java | 5 +- .../filter/SpatialFilterBonusTest.java | 9 +- .../segment/filter/SpatialFilterTest.java | 9 +- .../druid/client/CachingClusteredClient.java | 6 +- .../io/druid/client/CachingQueryRunner.java | 7 +- .../io/druid/client/DirectDruidClient.java | 19 ++- .../server/ClientQuerySegmentWalker.java | 4 +- .../java/io/druid/server/QueryResource.java | 22 +-- .../bridge/BridgeQuerySegmentWalker.java | 3 +- .../client/CachingClusteredClientTest.java | 45 +++--- .../druid/client/CachingQueryRunnerTest.java | 13 +- .../druid/client/DirectDruidClientTest.java | 23 ++- .../coordination/ServerManagerTest.java | 10 +- .../src/main/java/io/druid/cli/CliBroker.java | 2 + 63 files changed, 604 insertions(+), 339 deletions(-) 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/NullStorageAdapterException.java diff --git a/pom.xml b/pom.xml index 6717674b368..46e6a3feb98 100644 --- a/pom.xml +++ b/pom.xml @@ -244,7 +244,7 @@ com.fasterxml.jackson.datatype jackson-datatype-joda - 2.2.3 + 2.4.0 com.fasterxml.jackson.dataformat diff --git a/processing/src/main/java/io/druid/query/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java index 32d9c3256f4..3dac2fa0cfd 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 metadata) { - return run(querySegmentSpec.lookup(this, walker)); + return run(querySegmentSpec.lookup(this, walker), metadata); } - public Sequence run(QueryRunner runner) + public Sequence run(QueryRunner runner, Map metadata) { - return runner.run(this); + return runner.run(this, metadata); } @Override diff --git a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java index 44094d0216a..1d41a58b58b 100644 --- a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java @@ -29,6 +29,7 @@ import com.metamx.common.guava.YieldingAccumulator; import org.joda.time.DateTime; import java.util.List; +import java.util.Map; /** */ @@ -51,10 +52,10 @@ public class BySegmentQueryRunner implements QueryRunner @Override @SuppressWarnings("unchecked") - public Sequence run(final Query query) + public Sequence run(final Query query, Map metadata) { if (query.getContextBySegment(false)) { - final Sequence baseSequence = base.run(query); + final Sequence baseSequence = base.run(query, metadata); return new Sequence() { @Override @@ -96,7 +97,6 @@ public class BySegmentQueryRunner implements QueryRunner } }; } - - return base.run(query); + return base.run(query, metadata); } } diff --git a/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java b/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java index 13ca4dd75df..09cf38ee40b 100644 --- a/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java @@ -21,6 +21,9 @@ package io.druid.query; import com.metamx.common.guava.Sequence; +import java.util.List; +import java.util.Map; + /** */ public abstract class BySegmentSkippingQueryRunner implements QueryRunner @@ -35,14 +38,14 @@ public abstract class BySegmentSkippingQueryRunner implements QueryRunner } @Override - public Sequence run(Query query) + public Sequence run(Query query, Map metadata) { if (query.getContextBySegment(false)) { - return baseRunner.run(query); + return baseRunner.run(query, metadata); } - return doRun(baseRunner, query); + return doRun(baseRunner, query, metadata); } - protected abstract Sequence doRun(QueryRunner baseRunner, Query query); + protected abstract Sequence doRun(QueryRunner baseRunner, Query query, Map metadata); } diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index 8a5ed51a4df..45a9f5518d8 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 metadata) { final int priority = query.getContextPriority(0); @@ -124,7 +125,7 @@ public class ChainedExecutionQueryRunner implements QueryRunner throw new ISE("Input is null?! How is this possible?!"); } - Sequence result = input.run(query); + Sequence result = input.run(query, metadata); 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..482216fe818 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 metadata) { 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, metadata); } } ) diff --git a/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java index eb5bf466daf..565141aad7d 100644 --- a/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java +++ b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java @@ -28,6 +28,8 @@ import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.aggregation.MetricManipulatorFns; import javax.annotation.Nullable; +import java.util.List; +import java.util.Map; /** */ @@ -46,7 +48,7 @@ public class FinalizeResultsQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query) + public Sequence run(final Query query, Map metadata) { final boolean isBySegment = query.getContextBySegment(false); final boolean shouldFinalize = query.getContextFinalize(true); @@ -94,7 +96,7 @@ public class FinalizeResultsQueryRunner implements QueryRunner return Sequences.map( - baseRunner.run(queryToRun), + baseRunner.run(queryToRun, metadata), finalizerFn ); diff --git a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java index c9b14b6314b..ef1bd009523 100644 --- a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java @@ -43,6 +43,7 @@ import io.druid.segment.incremental.IncrementalIndex; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -86,7 +87,7 @@ public class GroupByParallelQueryRunner implements QueryRunner } @Override - public Sequence run(final Query queryParam) + public Sequence run(final Query queryParam, final Map metadata) { final GroupByQuery query = (GroupByQuery) queryParam; @@ -115,7 +116,7 @@ public class GroupByParallelQueryRunner implements QueryRunner public Boolean call() throws Exception { try { - input.run(queryParam).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); + input.run(queryParam, metadata).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); return true; } catch (QueryInterruptedException e) { diff --git a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java index 0bf4c3928b0..42f5093468f 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 metadata) { if (period.getMillis() == 0) { - return baseRunner.run(query); + return baseRunner.run(query, metadata); } 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))), + metadata ); } } diff --git a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java index dbad443cb36..110aadf4ce5 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; /** */ @@ -66,7 +68,7 @@ public class MetricsEmittingQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query) + public Sequence run(final Query query, final Map metadata) { final ServiceMetricEvent.Builder builder = builderFn.apply(query); String queryId = query.getId(); @@ -84,7 +86,7 @@ public class MetricsEmittingQueryRunner implements QueryRunner long startTime = System.currentTimeMillis(); try { - retVal = queryRunner.run(query).accumulate(outType, accumulator); + retVal = queryRunner.run(query, metadata).accumulate(outType, accumulator); } catch (RuntimeException e) { builder.setUser10("failed"); @@ -114,7 +116,7 @@ public class MetricsEmittingQueryRunner implements QueryRunner long startTime = System.currentTimeMillis(); try { - retVal = queryRunner.run(query).toYielder(initValue, accumulator); + retVal = queryRunner.run(query, metadata).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..0f659d01793 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 metadata) { 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 04c581152ad..74484b7ab96 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 metadata); - public Sequence run(QueryRunner runner); + public Sequence run(QueryRunner runner, Map metadata); 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..81b5f4b39ad 100644 --- a/processing/src/main/java/io/druid/query/QueryRunner.java +++ b/processing/src/main/java/io/druid/query/QueryRunner.java @@ -21,9 +21,12 @@ package io.druid.query; import com.metamx.common.guava.Sequence; +import java.util.List; +import java.util.Map; + /** */ public interface QueryRunner { - public Sequence run(Query query); -} + public Sequence run(Query query, Map metadata); +} \ 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 c366c287ac5..457b645544b 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 metadata) { final Closeable closeable = adapter.increment(); try { - final Sequence baseSequence = factory.createRunner(adapter).run(query); + final Sequence baseSequence = factory.createRunner(adapter).run(query, metadata); 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..c33cd032161 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 metadata) { - return CombiningSequence.create(baseRunner.run(query), makeOrdering(query), createMergeFn(query)); + return CombiningSequence.create(baseRunner.run(query, metadata), 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..2bd8f34a46a --- /dev/null +++ b/processing/src/main/java/io/druid/query/RetryQueryRunner.java @@ -0,0 +1,66 @@ +/* + * 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.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import io.druid.query.spec.QuerySegmentSpec; +import io.druid.query.spec.SpecificSegmentSpec; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +public class RetryQueryRunner implements QueryRunner +{ + private final QueryRunner baseRunner; + private final QueryToolChest> toolChest; + + public RetryQueryRunner(QueryRunner baseRunner, QueryToolChest> toolChest) + { + this.baseRunner = baseRunner; + this.toolChest = toolChest; + } + + @Override + public Sequence run(final Query query, Map metadata) + { + Sequence returningSeq = baseRunner.run(query, metadata); + + for (int i = RetryQueryRunnerConfig.numTries(); i > 0; i--) { + for (int j = metadata.get("missingSegments").size(); j > 0; j--) { + QuerySegmentSpec segmentSpec = new SpecificSegmentSpec((SegmentDescriptor)metadata.get("missingSegments").remove(0)); + returningSeq = toolChest.mergeSequences( + Sequences.simple( + Arrays.asList( + returningSeq, + baseRunner.run( + query.withQuerySegmentSpec(segmentSpec), + metadata + ) + ) + ) + ); + } + } + + return returningSeq; + } +} 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..f917c18c1b5 --- /dev/null +++ b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java @@ -0,0 +1,30 @@ +/* + * 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 static int numTries = 1; + + public static int numTries() { return numTries; } +} diff --git a/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java b/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java index 8e13d9219e9..dc7994cd01e 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 metadata) { DataSource dataSource = query.getDataSource(); if (dataSource instanceof QueryDataSource) { - return run((Query) ((QueryDataSource) dataSource).getQuery()); + return run((Query) ((QueryDataSource) dataSource).getQuery(), metadata); } else { - return baseRunner.run(query); + return baseRunner.run(query, metadata); } } } diff --git a/processing/src/main/java/io/druid/query/UnionQueryRunner.java b/processing/src/main/java/io/druid/query/UnionQueryRunner.java index 6679e6d7c77..d081d20cafe 100644 --- a/processing/src/main/java/io/druid/query/UnionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/UnionQueryRunner.java @@ -26,7 +26,8 @@ import com.google.common.collect.Lists; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import java.util.ArrayList; +import java.util.List; +import java.util.Map; public class UnionQueryRunner implements QueryRunner { @@ -43,7 +44,7 @@ public class UnionQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query) + public Sequence run(final Query query, final Map metadata) { DataSource dataSource = query.getDataSource(); if (dataSource instanceof UnionDataSource) { @@ -57,7 +58,8 @@ public class UnionQueryRunner implements QueryRunner public Sequence apply(DataSource singleSource) { return baseRunner.run( - query.withDataSource(singleSource) + query.withDataSource(singleSource), + metadata ); } } @@ -65,7 +67,7 @@ public class UnionQueryRunner implements QueryRunner ) ); } else { - return baseRunner.run(query); + return baseRunner.run(query, metadata); } } 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 0e00ceae46d..6ce42b14b53 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -49,6 +49,7 @@ import io.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.joda.time.Interval; import org.joda.time.Minutes; +import java.util.List; import java.util.Map; /** @@ -79,18 +80,18 @@ public class GroupByQueryQueryToolChest extends QueryToolChest() { @Override - public Sequence run(Query input) + public Sequence run(Query input, Map metadata) { if (Boolean.valueOf((String) 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, metadata); } else { - return runner.run(input); + return runner.run(input, metadata); } } }; } - private Sequence mergeGroupByResults(final GroupByQuery query, QueryRunner runner) + private Sequence mergeGroupByResults(final GroupByQuery query, QueryRunner runner, Map metadata) { Sequence result; @@ -104,12 +105,12 @@ public class GroupByQueryQueryToolChest extends QueryToolChest subqueryResult = mergeGroupByResults(subquery, runner); + Sequence subqueryResult = mergeGroupByResults(subquery, runner, metadata); IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(makeIncrementalIndex(subquery, subqueryResult)); result = engine.process(query, adapter); } else { - result = runner.run(query); + result = runner.run(query, metadata); } return postAggregate(query, makeIncrementalIndex(query, result)); 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 e8634089c2f..f9bd43c20dd 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java @@ -45,6 +45,8 @@ import io.druid.query.QueryWatcher; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; +import java.util.List; +import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; @@ -100,7 +102,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory() { @Override - public Sequence run(final Query query) + public Sequence run(final Query query, final Map metadata) { ListenableFuture> future = queryExecutor.submit( @@ -110,7 +112,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory call() throws Exception { return new ExecutorExecutingSequence( - input.run(query), + input.run(query, metadata), queryExecutor ); } @@ -166,7 +168,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory run(Query input) + public Sequence run(Query input, Map metadata) { 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/SegmentMetadataQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java index c8e7208638c..395db8908fb 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java @@ -45,6 +45,7 @@ import io.druid.segment.QueryableIndex; import io.druid.segment.Segment; import java.util.Arrays; +import java.util.List; import java.util.Map; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; @@ -74,7 +75,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory() { @Override - public Sequence run(Query inQ) + public Sequence run(Query inQ, Map metadata) { SegmentMetadataQuery query = (SegmentMetadataQuery) inQ; @@ -133,7 +134,10 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory() { @Override - public Sequence run(final Query query) + public Sequence run( + final Query query, + final Map metadata + ) { final int priority = query.getContextPriority(0); ListenableFuture> future = queryExecutor.submit( @@ -142,7 +146,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory call() throws Exception { - return input.run(query); + return input.run(query, metadata); } } ); 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 a0a2487cca4..229101adf95 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -284,7 +284,10 @@ public class SearchQueryQueryToolChest extends QueryToolChest> run(Query> input) + public Sequence> run( + Query> input, + Map metadata + ) { if (!(input instanceof SearchQuery)) { throw new ISE("Can only handle [%s], got [%s]", SearchQuery.class, input.getClass()); @@ -292,13 +295,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 53fe74e104f..d95d63887b5 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java @@ -19,7 +19,6 @@ package io.druid.query.search; -import com.google.common.base.Function; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -52,7 +51,6 @@ import io.druid.segment.filter.Filters; import it.uniroma3.mat.extendedset.intset.ConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; -import javax.annotation.Nullable; import java.util.List; import java.util.Map; import java.util.TreeSet; @@ -71,7 +69,10 @@ public class SearchQueryRunner implements QueryRunner> } @Override - public Sequence> run(final Query> input) + public Sequence> run( + final Query> input, + Map metadata + ) { 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/SelectQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java index 72cce700a6d..702d6295c57 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java @@ -19,14 +19,11 @@ package io.druid.query.select; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; import io.druid.query.ChainedExecutionQueryRunner; import io.druid.query.Query; -import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -34,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; /** @@ -91,7 +90,10 @@ public class SelectQueryRunnerFactory } @Override - public Sequence> run(Query> input) + public Sequence> run( + Query> input, + Map metadata + ) { 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..cd8134cb2ee 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java @@ -22,12 +22,16 @@ package io.druid.query.spec; import com.google.common.base.Throwables; 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.segment.NullStorageAdapterException; import java.io.IOException; +import java.util.List; +import java.util.Map; import java.util.concurrent.Callable; /** @@ -47,7 +51,7 @@ public class SpecificSegmentQueryRunner implements QueryRunner } @Override - public Sequence run(final Query input) + public Sequence run(final Query input, final Map metadata) { final Query query = input.withQuerySegmentSpec(specificSpec); @@ -60,7 +64,14 @@ public class SpecificSegmentQueryRunner implements QueryRunner @Override public Sequence call() throws Exception { - return base.run(query); + Sequence returningSeq; + try { + returningSeq = base.run(query, metadata); + } catch (NullStorageAdapterException e) { + metadata.get("missingSegments").add(((SpecificSegmentSpec) specificSpec).getDescriptor()); + returningSeq = Sequences.empty(); + } + return returningSeq; } }); 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 cc4aaa46579..b9a5216bedf 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -45,6 +45,7 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.List; +import java.util.Map; /** */ @@ -95,13 +96,13 @@ public class TimeBoundaryQueryQueryToolChest { @Override protected Sequence> doRun( - QueryRunner> baseRunner, Query> input + QueryRunner> baseRunner, Query> input, Map metadata ) { TimeBoundaryQuery query = (TimeBoundaryQuery) input; return Sequences.simple( query.mergeResults( - Sequences.toList(baseRunner.run(query), Lists.>newArrayList()) + Sequences.toList(baseRunner.run(query, metadata), 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 1f78429ead3..3858c0d8cc4 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -34,6 +34,8 @@ import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; import java.util.Iterator; +import java.util.List; +import java.util.Map; import java.util.concurrent.ExecutorService; /** @@ -82,7 +84,10 @@ public class TimeBoundaryQueryRunnerFactory } @Override - public Sequence> run(Query> input) + public Sequence> run( + Query> input, + Map metadata + ) { 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 ad290536b30..ee239c2c572 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java @@ -20,7 +20,6 @@ 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; @@ -28,6 +27,7 @@ import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.segment.Cursor; +import io.druid.segment.NullStorageAdapterException; import io.druid.segment.StorageAdapter; import io.druid.segment.filter.Filters; @@ -40,7 +40,7 @@ public class TimeseriesQueryEngine public Sequence> process(final TimeseriesQuery query, final StorageAdapter adapter) { if (adapter == null) { - throw new ISE( + throw new NullStorageAdapterException( "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 724d4818226..1706378ae38 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java @@ -19,13 +19,11 @@ package io.druid.query.timeseries; -import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; import io.druid.query.ChainedExecutionQueryRunner; import io.druid.query.Query; -import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -34,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; /** @@ -91,7 +91,10 @@ public class TimeseriesQueryRunnerFactory } @Override - public Sequence> run(Query> input) + public Sequence> run( + Query> input, + Map metadata + ) { 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 f0716a60407..1faf1fb7699 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java @@ -22,8 +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.FunctionalIterable; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.logger.Logger; @@ -34,11 +32,11 @@ 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.NullStorageAdapterException; import io.druid.segment.StorageAdapter; import io.druid.segment.filter.Filters; import org.joda.time.Interval; -import javax.sql.rowset.Predicate; import java.nio.ByteBuffer; import java.util.List; @@ -58,7 +56,7 @@ public class TopNQueryEngine public Sequence> query(final TopNQuery query, final StorageAdapter adapter) { if (adapter == null) { - throw new ISE( + throw new NullStorageAdapterException( "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 f275651383f..43fc6b9f303 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -410,7 +410,10 @@ public class TopNQueryQueryToolChest extends QueryToolChest> run(Query> input) + public Sequence> run( + Query> input, + Map metadata + ) { if (!(input instanceof TopNQuery)) { throw new ISE("Can only handle [%s], got [%s]", TopNQuery.class, input.getClass()); @@ -418,13 +421,13 @@ public class TopNQueryQueryToolChest extends QueryToolChest minTopNThreshold) { - return runner.run(query); + return runner.run(query, metadata); } final boolean isBySegment = query.getContextBySegment(false); return Sequences.map( - runner.run(query.withThreshold(minTopNThreshold)), + runner.run(query.withThreshold(minTopNThreshold), metadata), 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 524f9ace6a5..7b3c5cb07a9 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java @@ -21,7 +21,6 @@ package io.druid.query.topn; import com.google.inject.Inject; import com.metamx.common.ISE; -import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; import io.druid.collections.StupidPool; import io.druid.guice.annotations.Global; @@ -35,7 +34,8 @@ import io.druid.query.Result; import io.druid.segment.Segment; import java.nio.ByteBuffer; -import java.util.Iterator; +import java.util.List; +import java.util.Map; import java.util.concurrent.ExecutorService; /** @@ -65,7 +65,10 @@ public class TopNQueryRunnerFactory implements QueryRunnerFactory>() { @Override - public Sequence> run(Query> input) + public Sequence> run( + Query> input, + Map metadata + ) { 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/IncrementalIndexSegment.java b/processing/src/main/java/io/druid/segment/IncrementalIndexSegment.java index f21f7f1fa09..ab5d7974dcb 100644 --- a/processing/src/main/java/io/druid/segment/IncrementalIndexSegment.java +++ b/processing/src/main/java/io/druid/segment/IncrementalIndexSegment.java @@ -31,6 +31,7 @@ public class IncrementalIndexSegment implements Segment { private final IncrementalIndex index; private final String segmentIdentifier; + private boolean nullStorage = false; public IncrementalIndexSegment( IncrementalIndex index, diff --git a/processing/src/main/java/io/druid/segment/NullStorageAdapterException.java b/processing/src/main/java/io/druid/segment/NullStorageAdapterException.java new file mode 100644 index 00000000000..8d2b967afff --- /dev/null +++ b/processing/src/main/java/io/druid/segment/NullStorageAdapterException.java @@ -0,0 +1,27 @@ +/* + * 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 NullStorageAdapterException extends IllegalStateException +{ + public NullStorageAdapterException(String formatText, Object... arguments) { + super(String.format(formatText, arguments)); + } +} diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexSegment.java b/processing/src/main/java/io/druid/segment/QueryableIndexSegment.java index c2a7ee18aa6..e5075fc80ab 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexSegment.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexSegment.java @@ -29,6 +29,7 @@ public class QueryableIndexSegment implements Segment { private final QueryableIndex index; private final String identifier; + private boolean nullStorage = false; public QueryableIndexSegment(final String segmentIdentifier, QueryableIndex index) { diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index f2555dd7214..4c1b7351c0d 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -35,6 +35,9 @@ import org.easymock.IAnswer; import org.junit.Assert; import org.junit.Test; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -99,13 +102,14 @@ public class ChainedExecutionQueryRunnerTest runner3 ) ); - + HashMap metadata = new HashMap(); final Sequence seq = chainedRunner.run( Druids.newTimeseriesQueryBuilder() .dataSource("test") .intervals("2014/2015") .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) - .build() + .build(), + metadata ); Future resultFuture = Executors.newFixedThreadPool(1).submit( @@ -202,14 +206,15 @@ public class ChainedExecutionQueryRunnerTest runner3 ) ); - + HashMap metadata = 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(), + metadata ); Future resultFuture = Executors.newFixedThreadPool(1).submit( @@ -263,7 +268,7 @@ public class ChainedExecutionQueryRunnerTest } @Override - public Sequence run(Query query) + public Sequence run(Query query, Map metadata) { hasStarted = true; latch.countDown(); 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 be8fe086d78..ac073b6f723 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -74,6 +74,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collection; import java.util.Comparator; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -343,7 +344,7 @@ public class GroupByQueryRunnerTest new QueryRunner() { @Override - public Sequence run(Query query) + public Sequence run(Query query, Map metadata) { // simulate two daily segments final Query query1 = query.withQuerySegmentSpec( @@ -352,7 +353,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, metadata), runner.run(query2, metadata)); } } ); @@ -369,8 +370,9 @@ public class GroupByQueryRunnerTest createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) ); - TestHelper.assertExpectedObjects(expectedResults, runner.run(fullQuery), "direct"); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery), "merged"); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedObjects(expectedResults, runner.run(fullQuery, metadata), "direct"); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, metadata), "merged"); List allGranExpectedResults = Arrays.asList( createExpectedRow("2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L), @@ -384,8 +386,8 @@ public class GroupByQueryRunnerTest 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, metadata), "direct"); + TestHelper.assertExpectedObjects(allGranExpectedResults, mergedRunner.run(allGranQuery, metadata), "merged"); } @Test @@ -427,9 +429,9 @@ public class GroupByQueryRunnerTest ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - + HashMap metadata = new HashMap(); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery), String.format("limit: %d", limit) + Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery, metadata), String.format("limit: %d", limit) ); } @@ -535,7 +537,7 @@ public class GroupByQueryRunnerTest new QueryRunner() { @Override - public Sequence run(Query query) + public Sequence run(Query query, Map metadata) { // simulate two daily segments final Query query1 = query.withQuerySegmentSpec( @@ -544,12 +546,12 @@ 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, metadata), runner.run(query2, metadata)); } } ); - - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery), "merged"); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, metadata), "merged"); } @Test @@ -585,10 +587,11 @@ public class GroupByQueryRunnerTest ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "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(), metadata), "limited" ); } @@ -625,9 +628,10 @@ public class GroupByQueryRunnerTest ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "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(), metadata), "limited" ); } @@ -664,9 +668,10 @@ public class GroupByQueryRunnerTest ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "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(), metadata), "limited" ); } @@ -706,7 +711,7 @@ public class GroupByQueryRunnerTest new QueryRunner() { @Override - public Sequence run(Query query) + public Sequence run(Query query, Map metadata) { // simulate two daily segments final Query query1 = query.withQuerySegmentSpec( @@ -715,12 +720,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, metadata), runner.run(query2, metadata)); } } ); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery), "merged"); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, metadata), "merged"); } @Test @@ -759,8 +765,9 @@ public class GroupByQueryRunnerTest ) ); + HashMap metadata = new HashMap(); QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "no-limit"); } @Test @@ -792,8 +799,8 @@ public class GroupByQueryRunnerTest createExpectedRow("2011-04-01", "quality", "technology", "rows", 2L), createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L) ); - - TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedObjects(expectedResults, runner.run(query, metadata), "normal"); final GroupByQueryEngine engine = new GroupByQueryEngine( configSupplier, new StupidPool( @@ -809,7 +816,7 @@ public class GroupByQueryRunnerTest ); QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "no-limit"); } @Test @@ -842,7 +849,8 @@ public class GroupByQueryRunnerTest createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L) ); - TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedObjects(expectedResults, runner.run(query, metadata), "normal"); final GroupByQueryEngine engine = new GroupByQueryEngine( configSupplier, new StupidPool( @@ -858,7 +866,7 @@ public class GroupByQueryRunnerTest ); QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "no-limit"); } // A subquery identical to the query should yield identical results @@ -1038,7 +1046,8 @@ public class GroupByQueryRunnerTest toolChest ); - Sequence queryResult = theRunner.run(query); + HashMap metadata = new HashMap(); + Sequence queryResult = theRunner.run(query, metadata); 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 a9fb506ca0b..dcb6b76e575 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -42,6 +42,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collection; +import java.util.Map; /** */ @@ -91,7 +92,7 @@ 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; @@ -104,7 +105,8 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest .setDimFilter(tsQuery.getDimensionsFilter()) .setAggregatorSpecs(tsQuery.getAggregatorSpecs()) .setPostAggregatorSpecs(tsQuery.getPostAggregatorSpecs()) - .build() + .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 70c65f8da88..bb2dcbdd85d 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java @@ -25,7 +25,6 @@ import io.druid.query.LegacyDataSource; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; -import io.druid.query.QueryWatcher; import io.druid.query.metadata.metadata.ColumnAnalysis; import io.druid.query.metadata.metadata.SegmentAnalysis; import io.druid.query.metadata.metadata.SegmentMetadataQuery; @@ -38,6 +37,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; @@ -102,6 +102,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 metadata = new HashMap(); + return Sequences.toList(query.run(runner, metadata), 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 ed1740460f8..3e627bdca49 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java @@ -40,6 +40,8 @@ import org.junit.Assert; import org.junit.Test; import java.util.Arrays; +import java.util.HashMap; +import java.util.List; public class SegmentMetadataQueryTest { @@ -70,9 +72,9 @@ public class SegmentMetadataQueryTest .toInclude(new ListColumnIncluderator(Arrays.asList("placement"))) .merge(true) .build(); - + HashMap metadata = new HashMap(); Iterable results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), 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 c69ee1c5a27..1e64c284b84 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java @@ -23,13 +23,10 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.guava.Sequences; import io.druid.query.Druids; -import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; -import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.query.filter.DimFilter; import io.druid.query.search.search.FragmentSearchQuerySpec; @@ -47,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; @@ -371,8 +369,9 @@ public class SearchQueryRunnerTest private void checkSearchQuery(SearchQuery searchQuery, Map> expectedResults) { + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(searchQuery), + runner.run(searchQuery, metadata), 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 07f99165873..88a983be6d6 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java @@ -22,15 +22,12 @@ package io.druid.query.select; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.ISE; import com.metamx.common.guava.Sequences; import io.druid.jackson.DefaultObjectMapper; -import io.druid.query.Query; import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; -import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.query.TableDataSource; import io.druid.query.filter.SelectorDimFilter; @@ -45,6 +42,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; @@ -90,9 +88,9 @@ public class SelectQueryRunnerTest new PagingSpec(null, 3), null ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); @@ -159,9 +157,9 @@ public class SelectQueryRunnerTest new PagingSpec(null, 3), null ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); @@ -219,9 +217,9 @@ public class SelectQueryRunnerTest new PagingSpec(Maps.newLinkedHashMap(ImmutableMap.of(QueryRunnerTestHelper.segmentId, 3)), 3), null ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); @@ -279,9 +277,9 @@ public class SelectQueryRunnerTest new PagingSpec(Maps.newLinkedHashMap(ImmutableMap.of(QueryRunnerTestHelper.segmentId, 3)), 3), null ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), 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 7bc499dca80..28b378e08c9 100644 --- a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java @@ -20,13 +20,10 @@ package io.druid.query.timeboundary; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.guava.Sequences; import io.druid.query.Druids; -import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; -import io.druid.query.QueryWatcher; import io.druid.query.Result; import org.joda.time.DateTime; import org.junit.Assert; @@ -36,6 +33,8 @@ import org.junit.runners.Parameterized; import java.io.IOException; import java.util.Collection; +import java.util.HashMap; +import java.util.List; /** */ @@ -66,9 +65,9 @@ public class TimeBoundaryQueryRunnerTest TimeBoundaryQuery timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder() .dataSource("testing") .build(); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(timeBoundaryQuery), + runner.run(timeBoundaryQuery, metadata), 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 17d61908c3c..bedc433cc69 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java @@ -38,6 +38,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; @RunWith(Parameterized.class) @@ -97,9 +98,9 @@ public class TimeSeriesUnionQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), 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..92821c4260c 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java @@ -43,6 +43,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 @@ -110,9 +111,9 @@ public class TimeseriesQueryRunnerBonusTest ) ) .build(); - + HashMap metadata = new HashMap(); return Sequences.toList( - runner.run(query), + runner.run(query, metadata), 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 708a7de1054..17ee38449dd 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -52,6 +52,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; /** @@ -100,9 +101,10 @@ public class TimeseriesQueryRunnerTest DateTime expectedEarliest = new DateTime("2011-01-12"); DateTime expectedLast = new DateTime("2011-04-15"); + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); @@ -166,8 +168,9 @@ public class TimeseriesQueryRunnerTest DateTime expectedEarliest = new DateTime("2011-01-12"); DateTime expectedLast = new DateTime("2011-04-15"); + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); Result result = results.iterator().next(); @@ -212,9 +215,9 @@ public class TimeseriesQueryRunnerTest DateTime expectedEarliest = new DateTime("2011-01-12"); DateTime expectedLast = new DateTime("2011-04-15"); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); @@ -278,9 +281,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); @@ -325,9 +328,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); @@ -367,9 +370,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results1 = Sequences.toList( - runner.run(query1), + runner.run(query1, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults1, results1); @@ -406,7 +409,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results2 = Sequences.toList( - runner.run(query2), + runner.run(query2, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults2, results2); @@ -457,9 +460,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results1 = Sequences.toList( - runner.run(query1), + runner.run(query1, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults1, results1); @@ -499,9 +502,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results1 = Sequences.toList( - runner.run(query1), + runner.run(query1, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults1, results1); @@ -539,7 +542,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results2 = Sequences.toList( - runner.run(query2), + runner.run(query2, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults2, results2); @@ -572,9 +575,9 @@ public class TimeseriesQueryRunnerTest .build(); List> expectedResults = Arrays.asList(); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -622,9 +625,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -672,9 +675,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -722,9 +725,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -772,9 +775,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -822,9 +825,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -880,9 +883,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -938,9 +941,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -996,9 +999,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1052,9 +1055,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1114,9 +1117,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1158,9 +1161,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1202,9 +1205,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1260,9 +1263,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - + HashMap metadata = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1300,8 +1303,9 @@ public class TimeseriesQueryRunnerTest ) ) ); + HashMap metadata = new HashMap(); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1340,8 +1344,9 @@ public class TimeseriesQueryRunnerTest ) ) ); + HashMap metadata = new HashMap(); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1358,7 +1363,7 @@ public class TimeseriesQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - + HashMap metadata = new HashMap(); Iterable> expectedResults = Sequences.toList( runner.run( Druids.newTimeseriesQueryBuilder() @@ -1367,12 +1372,13 @@ public class TimeseriesQueryRunnerTest .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .build() + .build(), + metadata ), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1389,7 +1395,7 @@ public class TimeseriesQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - + HashMap metadata = new HashMap(); Iterable> expectedResults = Sequences.toList( runner.run( Druids.newTimeseriesQueryBuilder() @@ -1399,12 +1405,13 @@ public class TimeseriesQueryRunnerTest .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .build() + .build(), + metadata ), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1450,7 +1457,7 @@ public class TimeseriesQueryRunnerTest ) ) .build(); - + HashMap metadata = new HashMap(); Iterable> expectedResults = Sequences.toList( runner.run( Druids.newTimeseriesQueryBuilder() @@ -1460,12 +1467,13 @@ public class TimeseriesQueryRunnerTest .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .build() + .build(), + metadata ), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, metadata), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1513,7 +1521,7 @@ public class TimeseriesQueryRunnerTest ) ) .build(); - + HashMap metadata = new HashMap(); Iterable> expectedResults = Sequences.toList( runner.run( Druids.newTimeseriesQueryBuilder() @@ -1523,12 +1531,13 @@ public class TimeseriesQueryRunnerTest .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .build() + .build(), + metadata ), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query), + runner.run(query, metadata), 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 09d383168cf..44a166aa503 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 metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -230,8 +231,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @@ -295,8 +296,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @@ -345,8 +346,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -394,8 +395,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -443,8 +444,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -485,8 +486,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -520,8 +521,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -569,8 +570,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -622,8 +623,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -664,8 +665,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -682,7 +683,7 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - + HashMap metadata = 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, metadata) ); } @@ -721,7 +722,7 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - + HashMap metadata = 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, metadata) ); } @@ -747,7 +748,7 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - + HashMap metadata = 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(), + metadata ), Lists.>newArrayList() - ), runner.run(query) + ), runner.run(query, metadata) ); } @@ -781,7 +783,7 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - + HashMap metadata = 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(), + metadata ), Lists.>newArrayList() ) - , runner.run(query) + , runner.run(query, metadata) ); } @@ -840,8 +843,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -889,8 +892,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -945,8 +948,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -993,8 +996,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -1034,8 +1037,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -1075,8 +1078,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -1116,8 +1119,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -1157,8 +1160,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -1209,8 +1212,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -1261,8 +1264,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -1313,8 +1316,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -1358,8 +1361,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @@ -1404,8 +1407,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -1449,8 +1452,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -1498,8 +1501,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -1583,8 +1586,8 @@ public class TopNQueryRunnerTest ) ) ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } @Test @@ -1666,6 +1669,7 @@ public class TopNQueryRunnerTest ) ) ); - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } } 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..4df1aa50818 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 metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } 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..7b76e2d0f7d 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java @@ -61,6 +61,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; @@ -448,8 +449,8 @@ public class SpatialFilterBonusTest factory.createRunner(segment), factory.getToolchest() ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } catch (Exception e) { throw Throwables.propagate(e); @@ -535,8 +536,8 @@ public class SpatialFilterBonusTest factory.createRunner(segment), factory.getToolchest() ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } 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 84df58a260d..631d06c94d2 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java @@ -61,6 +61,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; @@ -478,8 +479,8 @@ public class SpatialFilterTest factory.createRunner(segment), factory.getToolchest() ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } catch (Exception e) { throw Throwables.propagate(e); @@ -565,8 +566,8 @@ public class SpatialFilterTest factory.createRunner(segment), factory.getToolchest() ); - - TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + HashMap metadata = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); } catch (Exception e) { throw Throwables.propagate(e); diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index cf5f09228f6..339849347ad 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -115,7 +115,7 @@ public class CachingClusteredClient implements QueryRunner } @Override - public Sequence run(final Query query) + public Sequence run(final Query query, final Map metadata) { final QueryToolChest> toolChest = warehouse.getToolChest(query); final CacheStrategy> strategy = toolChest.getCacheStrategy(query); @@ -327,11 +327,11 @@ 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), metadata); } else { resultSeqToAdd = toolChest.mergeSequences( Sequences.map( - clientQueryable.run(rewrittenQuery.withQuerySegmentSpec(segmentSpec)), + clientQueryable.run(rewrittenQuery.withQuerySegmentSpec(segmentSpec), metadata), 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 ad2718e6a45..2a5c932e73c 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 metadata) { final CacheStrategy strategy = toolChest.getCacheStrategy(query); @@ -140,7 +141,7 @@ public class CachingQueryRunner implements QueryRunner return Sequences.withEffect( Sequences.map( - base.run(query), + base.run(query, metadata), new Function() { @Override @@ -162,7 +163,7 @@ public class CachingQueryRunner implements QueryRunner MoreExecutors.sameThreadExecutor() ); } else { - return base.run(query); + return base.run(query, metadata); } } diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index b6030f9755b..5bf0747a0cf 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -54,6 +54,7 @@ import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChestWarehouse; import io.druid.query.QueryWatcher; import io.druid.query.Result; +import io.druid.query.SegmentDescriptor; import io.druid.query.aggregation.MetricManipulatorFns; import org.jboss.netty.handler.codec.http.HttpChunk; import org.jboss.netty.handler.codec.http.HttpHeaders; @@ -64,6 +65,8 @@ import java.io.IOException; import java.io.InputStream; import java.net.URL; import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; import java.util.Map; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; @@ -111,7 +114,7 @@ public class DirectDruidClient implements QueryRunner } @Override - public Sequence run(final Query query) + public Sequence run(final Query query, final Map metadata) { QueryToolChest> toolChest = warehouse.getToolChest(query); boolean isBySegment = query.getContextBySegment(false); @@ -156,6 +159,20 @@ public class DirectDruidClient implements QueryRunner log.debug("Initial response from url[%s]", url); startTime = System.currentTimeMillis(); byteCount += response.getContent().readableBytes(); + + if (!response.getHeader("Missing-Segments").equals("")) { + LinkedList missingSegments = new LinkedList(); + try { + missingSegments = objectMapper.readValue(response.getHeader("Missing-Segments"), LinkedList.class); + for (int i = missingSegments.size(); i > 0; i--) { + missingSegments.add(objectMapper.convertValue(missingSegments.remove(0), SegmentDescriptor.class)); + } + } + catch (IOException e) { + } + metadata.get("missingSegments").addAll(missingSegments); + } + return super.handleResponse(response); } diff --git a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java index f0e8aa01153..4a5e6ef40ac 100644 --- a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java @@ -31,6 +31,7 @@ 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.SegmentDescriptor; import io.druid.query.UnionQueryRunner; import org.joda.time.Interval; @@ -86,7 +87,8 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker return toolChest.makeMetricBuilder(query); } }, - toolChest.preMergeQueryDecoration(baseClient) + toolChest.preMergeQueryDecoration(new RetryQueryRunner(baseClient, toolChest) + ) ).withWaitMeasuredFromNow(), toolChest ) diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 33bdd519c83..2d13ae3cfa9 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -22,20 +22,15 @@ package io.druid.server; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; -import com.google.api.client.repackaged.com.google.common.base.Throwables; import com.google.common.base.Charsets; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; -import com.google.common.io.Closeables; import com.google.inject.Inject; -import com.metamx.common.guava.Accumulator; -import com.metamx.common.guava.Accumulators; 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 com.metamx.common.guava.YieldingAccumulators; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; @@ -51,7 +46,6 @@ import org.joda.time.DateTime; import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; -import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; import javax.ws.rs.POST; import javax.ws.rs.Path; @@ -59,12 +53,13 @@ import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.Context; -import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import javax.ws.rs.core.StreamingOutput; import java.io.IOException; import java.io.OutputStream; -import java.nio.charset.Charset; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; import java.util.UUID; /** @@ -147,7 +142,9 @@ public class QueryResource log.debug("Got query [%s]", query); } - Sequence results = query.run(texasRanger); + HashMap metadata = new HashMap(); + metadata.put("missingSegments", new LinkedList()); + Sequence results = query.run(texasRanger, metadata); if (results == null) { results = Sequences.empty(); @@ -167,6 +164,12 @@ public class QueryResource } ) ) { + + String missingSegments = ""; + if (!metadata.get("missingSegments").isEmpty()) { + missingSegments = jsonMapper.writeValueAsString(metadata.get("missingSegments")); + } + long requestTime = System.currentTimeMillis() - start; emitter.emit( @@ -209,6 +212,7 @@ public class QueryResource isSmile ? APPLICATION_JSON : APPLICATION_SMILE ) .header("X-Druid-Query-Id", queryId) + .header("Missing-Segments", missingSegments) .build(); } } 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..5590fd420b8 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 metadata) { try { Server instance = brokerSelector.pick(); diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 7e151f3818d..9c75067f678 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -106,6 +106,7 @@ import javax.annotation.Nullable; 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; @@ -262,7 +263,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-09T01"), 181, 52 ) ); - + HashMap metadata = new HashMap(); TestHelper.assertExpectedResults( makeRenamedTimeResults( new DateTime("2011-01-01"), 50, 5000, @@ -283,7 +284,8 @@ public class CachingClusteredClientTest builder.intervals("2011-01-01/2011-01-10") .aggregators(RENAMED_AGGS) .postAggregators(RENAMED_POST_AGGS) - .build() + .build(), + metadata ) ); } @@ -314,7 +316,7 @@ public class CachingClusteredClientTest new DateTime("2011-11-07", TIMEZONE), 85, 102 ) ); - + HashMap metadata = new HashMap(); TestHelper.assertExpectedResults( makeRenamedTimeResults( new DateTime("2011-11-04", TIMEZONE), 50, 5000, @@ -326,7 +328,8 @@ public class CachingClusteredClientTest builder.intervals("2011-11-04/2011-11-08") .aggregators(RENAMED_AGGS) .postAggregators(RENAMED_POST_AGGS) - .build() + .build(), + metadata ) ); } @@ -442,7 +445,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-09T01"), "c2", 50, 4985, "b", 50, 4984, "c", 50, 4983 ) ); - + HashMap metadata = new HashMap(); TestHelper.assertExpectedResults( makeRenamedTopNResults( new DateTime("2011-01-01"), "a", 50, 5000, "b", 50, 4999, "c", 50, 4998, @@ -463,7 +466,8 @@ public class CachingClusteredClientTest .metric("imps") .aggregators(RENAMED_AGGS) .postAggregators(RENAMED_POST_AGGS) - .build() + .build(), + metadata ) ); } @@ -497,7 +501,7 @@ public class CachingClusteredClientTest new DateTime("2011-11-07", TIMEZONE), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986 ) ); - + HashMap metadata = new HashMap(); TestHelper.assertExpectedResults( makeRenamedTopNResults( @@ -511,7 +515,8 @@ public class CachingClusteredClientTest .metric("imps") .aggregators(RENAMED_AGGS) .postAggregators(RENAMED_POST_AGGS) - .build() + .build(), + metadata ) ); } @@ -561,7 +566,7 @@ public class CachingClusteredClientTest ) ); - + HashMap metadata = new HashMap(); TestHelper.assertExpectedResults( makeRenamedTopNResults( new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, @@ -580,7 +585,8 @@ public class CachingClusteredClientTest .metric("imps") .aggregators(RENAMED_AGGS) .postAggregators(RENAMED_POST_AGGS) - .build() + .build(), + metadata ) ); } @@ -629,7 +635,7 @@ public class CachingClusteredClientTest ) ); - + HashMap metadata = new HashMap(); TestHelper.assertExpectedResults( makeTopNResults( new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, @@ -648,7 +654,8 @@ public class CachingClusteredClientTest .metric("avg_imps_per_row_double") .aggregators(AGGS) .postAggregators(POST_AGGS) - .build() + .build(), + metadata ) ); } @@ -756,6 +763,7 @@ public class CachingClusteredClientTest .once(); final Capture capture = new Capture(); + final Capture metadata = new Capture(); queryCaptures.add(capture); QueryRunner queryable = expectations.getQueryRunner(); @@ -768,8 +776,7 @@ public class CachingClusteredClientTest intervals.add(expectation.getInterval()); results.add(expectation.getResults()); } - - EasyMock.expect(queryable.run(EasyMock.capture(capture))) + EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(metadata))) .andReturn(toQueryableTimeseriesResults(expectBySegment, segmentIds, intervals, results)) .once(); @@ -782,7 +789,7 @@ public class CachingClusteredClientTest intervals.add(expectation.getInterval()); results.add(expectation.getResults()); } - EasyMock.expect(queryable.run(EasyMock.capture(capture))) + EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(metadata))) .andReturn(toQueryableTopNResults(segmentIds, intervals, results)) .once(); } else if (query instanceof SearchQuery) { @@ -794,7 +801,7 @@ public class CachingClusteredClientTest intervals.add(expectation.getInterval()); results.add(expectation.getResults()); } - EasyMock.expect(queryable.run(EasyMock.capture(capture))) + EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(metadata))) .andReturn(toQueryableSearchResults(segmentIds, intervals, results)) .once(); } else if (query instanceof TimeBoundaryQuery) { @@ -806,7 +813,7 @@ public class CachingClusteredClientTest intervals.add(expectation.getInterval()); results.add(expectation.getResults()); } - EasyMock.expect(queryable.run(EasyMock.capture(capture))) + EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(metadata))) .andReturn(toQueryableTimeBoundaryResults(segmentIds, intervals, results)) .once(); } else { @@ -830,6 +837,7 @@ public class CachingClusteredClientTest @Override public void run() { + HashMap metadata = new HashMap(); for (int i = 0; i < numTimesToQuery; ++i) { TestHelper.assertExpectedResults( new MergeIterable<>( @@ -863,7 +871,8 @@ public class CachingClusteredClientTest actualQueryInterval ) ) - ) + ), + metadata ) ); } diff --git a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java index d72da61e751..e669d847f9d 100644 --- a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java @@ -57,6 +57,7 @@ import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; @@ -123,7 +124,7 @@ public class CachingQueryRunnerTest new QueryRunner() { @Override - public Sequence run(Query query) + public Sequence run(Query query, Map metadata) { return resultSeq; } @@ -140,8 +141,8 @@ public class CachingQueryRunnerTest cacheStrategy.computeCacheKey(query) ); - - Sequence res = runner.run(query); + HashMap metadata = new HashMap(); + Sequence res = runner.run(query, metadata); // base sequence is not closed yet Assert.assertFalse("sequence must not be closed", closable.isClosed()); Assert.assertNull("cache must be empty", cache.get(cacheKey)); @@ -213,7 +214,7 @@ public class CachingQueryRunnerTest new QueryRunner() { @Override - public Sequence run(Query query) + public Sequence run(Query query, Map metadata) { return Sequences.empty(); } @@ -221,8 +222,8 @@ public class CachingQueryRunnerTest new CacheConfig() ); - - List results = Sequences.toList(runner.run(query), new ArrayList()); + HashMap metadata = new HashMap(); + List results = Sequences.toList(runner.run(query, metadata), new ArrayList()); Assert.assertEquals(expectedResults, results); } diff --git a/server/src/test/java/io/druid/client/DirectDruidClientTest.java b/server/src/test/java/io/druid/client/DirectDruidClientTest.java index 4ad8ca5cd51..d6fbb1b7c9d 100644 --- a/server/src/test/java/io/druid/client/DirectDruidClientTest.java +++ b/server/src/test/java/io/druid/client/DirectDruidClientTest.java @@ -36,10 +36,8 @@ import io.druid.client.selector.QueryableDruidServer; import io.druid.client.selector.ServerSelector; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.Druids; -import io.druid.query.Query; import io.druid.query.QueryInterruptedException; import io.druid.query.QueryRunnerTestHelper; -import io.druid.query.QueryWatcher; import io.druid.query.ReflectionQueryToolChestWarehouse; import io.druid.query.Result; import io.druid.query.timeboundary.TimeBoundaryQuery; @@ -57,6 +55,7 @@ import org.junit.Test; import java.io.ByteArrayInputStream; import java.io.InputStream; import java.net.URL; +import java.util.HashMap; import java.util.List; public class DirectDruidClientTest @@ -118,20 +117,20 @@ public class DirectDruidClientTest serverSelector.addServer(queryableDruidServer2); TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build(); - - Sequence s1 = client1.run(query); + HashMap metadata = new HashMap(); + Sequence s1 = client1.run(query, metadata); Assert.assertEquals(1, client1.getNumOpenConnections()); // simulate read timeout - Sequence s2 = client1.run(query); + Sequence s2 = client1.run(query, metadata); Assert.assertEquals(2, client1.getNumOpenConnections()); futureException.setException(new ReadTimeoutException()); Assert.assertEquals(1, client1.getNumOpenConnections()); // subsequent connections should work - Sequence s3 = client1.run(query); - Sequence s4 = client1.run(query); - Sequence s5 = client1.run(query); + Sequence s3 = client1.run(query, metadata); + Sequence s4 = client1.run(query, metadata); + Sequence s5 = client1.run(query, metadata); Assert.assertTrue(client1.getNumOpenConnections() == 4); @@ -142,8 +141,8 @@ public class DirectDruidClientTest Assert.assertEquals(new DateTime("2014-01-01T01:02:03Z"), results.get(0).getTimestamp()); Assert.assertEquals(3, client1.getNumOpenConnections()); - client2.run(query); - client2.run(query); + client2.run(query, metadata); + client2.run(query, metadata); Assert.assertTrue(client2.getNumOpenConnections() == 2); @@ -201,9 +200,9 @@ public class DirectDruidClientTest serverSelector.addServer(queryableDruidServer1); TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build(); - + HashMap metadata = new HashMap(); cancellationFuture.set(new StatusResponseHolder(HttpResponseStatus.OK, new StringBuilder("cancelled"))); - Sequence results = client1.run(query); + Sequence results = client1.run(query, metadata); Assert.assertEquals(0, client1.getNumOpenConnections()); diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index 431fdce8318..371bdcc3671 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -71,8 +71,10 @@ import org.junit.Test; import java.io.File; import java.io.IOException; import java.util.Arrays; +import java.util.HashMap; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -421,14 +423,14 @@ public class ServerManagerTest query, intervals ); - return serverManagerExec.submit( new Runnable() { @Override public void run() { - Sequence> seq = runner.run(query); + Map metadata = new HashMap(); + Sequence> seq = runner.run(query, metadata); Sequences.toList(seq, Lists.>newArrayList()); Iterator adaptersIter = factory.getAdapters().iterator(); @@ -677,9 +679,9 @@ public class ServerManagerTest } @Override - public Sequence run(Query query) + public Sequence run(Query query, Map metadata) { - return new BlockingSequence(runner.run(query), waitLatch, waitYieldLatch, notifyLatch); + return new BlockingSequence(runner.run(query, metadata), waitLatch, waitYieldLatch, notifyLatch); } } diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index 2381c13d282..d89b4fc0ba8 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -44,6 +44,7 @@ import io.druid.guice.annotations.Self; import io.druid.query.MapQueryToolChestWarehouse; import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChestWarehouse; +import io.druid.query.RetryQueryRunnerConfig; import io.druid.server.ClientInfoResource; import io.druid.server.ClientQuerySegmentWalker; import io.druid.server.QueryResource; @@ -88,6 +89,7 @@ public class CliBroker extends ServerRunnable JsonConfigProvider.bind(binder, "druid.broker.select.tier", TierSelectorStrategy.class); JsonConfigProvider.bind(binder, "druid.broker.select.tier.custom", CustomTierSelectorStrategyConfig.class); JsonConfigProvider.bind(binder, "druid.broker.balancer", ServerSelectorStrategy.class); + JsonConfigProvider.bind(binder, "druid.broker.retryPolicy", RetryQueryRunnerConfig.class); binder.bind(QuerySegmentWalker.class).to(ClientQuerySegmentWalker.class).in(LazySingleton.class); From d8430b854ded1b736276431261833112801fb0a8 Mon Sep 17 00:00:00 2001 From: jisookim0513 Date: Wed, 18 Jun 2014 17:01:08 -0700 Subject: [PATCH 011/107] revert changes in pom file --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 46e6a3feb98..6717674b368 100644 --- a/pom.xml +++ b/pom.xml @@ -244,7 +244,7 @@ com.fasterxml.jackson.datatype jackson-datatype-joda - 2.4.0 + 2.2.3 com.fasterxml.jackson.dataformat From f4b1dc032b0be5012eade18b817662db73eeb388 Mon Sep 17 00:00:00 2001 From: jisookim0513 Date: Wed, 18 Jun 2014 17:15:04 -0700 Subject: [PATCH 012/107] change the way to use RetryQueryRunnerConfig --- .../src/main/java/io/druid/query/RetryQueryRunner.java | 6 ++++-- .../main/java/io/druid/query/RetryQueryRunnerConfig.java | 4 ++-- .../java/io/druid/server/ClientQuerySegmentWalker.java | 8 ++++++-- 3 files changed, 12 insertions(+), 6 deletions(-) diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunner.java b/processing/src/main/java/io/druid/query/RetryQueryRunner.java index 2bd8f34a46a..054ac19313d 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunner.java @@ -32,11 +32,13 @@ public class RetryQueryRunner implements QueryRunner { private final QueryRunner baseRunner; private final QueryToolChest> toolChest; + private final RetryQueryRunnerConfig config; - public RetryQueryRunner(QueryRunner baseRunner, QueryToolChest> toolChest) + public RetryQueryRunner(QueryRunner baseRunner, QueryToolChest> toolChest, RetryQueryRunnerConfig config) { this.baseRunner = baseRunner; this.toolChest = toolChest; + this.config = config; } @Override @@ -44,7 +46,7 @@ public class RetryQueryRunner implements QueryRunner { Sequence returningSeq = baseRunner.run(query, metadata); - for (int i = RetryQueryRunnerConfig.numTries(); i > 0; i--) { + for (int i = config.numTries(); i > 0; i--) { for (int j = metadata.get("missingSegments").size(); j > 0; j--) { QuerySegmentSpec segmentSpec = new SpecificSegmentSpec((SegmentDescriptor)metadata.get("missingSegments").remove(0)); returningSeq = toolChest.mergeSequences( diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java index f917c18c1b5..5b5ed2639b5 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java @@ -24,7 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; public class RetryQueryRunnerConfig { @JsonProperty - private static int numTries = 1; + private int numTries = 1; - public static int numTries() { return numTries; } + public int numTries() { return numTries; } } diff --git a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java index 4a5e6ef40ac..6c6a2d801f0 100644 --- a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java @@ -32,6 +32,7 @@ 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; @@ -45,17 +46,20 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker private final ServiceEmitter emitter; private final CachingClusteredClient baseClient; private final QueryToolChestWarehouse warehouse; + private final RetryQueryRunnerConfig retryConfig; @Inject public ClientQuerySegmentWalker( ServiceEmitter emitter, CachingClusteredClient baseClient, - QueryToolChestWarehouse warehouse + QueryToolChestWarehouse warehouse, + RetryQueryRunnerConfig retryConfig ) { this.emitter = emitter; this.baseClient = baseClient; this.warehouse = warehouse; + this.retryConfig = retryConfig; } @Override @@ -87,7 +91,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker return toolChest.makeMetricBuilder(query); } }, - toolChest.preMergeQueryDecoration(new RetryQueryRunner(baseClient, toolChest) + toolChest.preMergeQueryDecoration(new RetryQueryRunner(baseClient, toolChest, retryConfig) ) ).withWaitMeasuredFromNow(), toolChest From 8515a117874415354f9b55aa40daa3635fae07bf Mon Sep 17 00:00:00 2001 From: jisookim0513 Date: Wed, 18 Jun 2014 18:05:09 -0700 Subject: [PATCH 013/107] change the signature of run() --- .../main/java/io/druid/query/BaseQuery.java | 8 +- .../io/druid/query/BySegmentQueryRunner.java | 6 +- .../query/BySegmentSkippingQueryRunner.java | 9 +- .../query/ChainedExecutionQueryRunner.java | 4 +- .../io/druid/query/ConcatQueryRunner.java | 4 +- .../query/FinalizeResultsQueryRunner.java | 4 +- .../query/GroupByParallelQueryRunner.java | 4 +- .../query/IntervalChunkingQueryRunner.java | 6 +- .../query/MetricsEmittingQueryRunner.java | 6 +- .../java/io/druid/query/NoopQueryRunner.java | 2 +- .../src/main/java/io/druid/query/Query.java | 4 +- .../main/java/io/druid/query/QueryRunner.java | 3 +- .../ReferenceCountingSegmentQueryRunner.java | 4 +- .../druid/query/ResultMergeQueryRunner.java | 4 +- .../java/io/druid/query/RetryQueryRunner.java | 10 +- .../io/druid/query/SubqueryQueryRunner.java | 6 +- .../java/io/druid/query/UnionQueryRunner.java | 6 +- .../groupby/GroupByQueryQueryToolChest.java | 12 +- .../groupby/GroupByQueryRunnerFactory.java | 6 +- .../SegmentMetadataQueryRunnerFactory.java | 7 +- .../search/SearchQueryQueryToolChest.java | 6 +- .../druid/query/search/SearchQueryRunner.java | 2 +- .../select/SelectQueryRunnerFactory.java | 2 +- .../spec/SpecificSegmentQueryRunner.java | 6 +- .../TimeBoundaryQueryQueryToolChest.java | 4 +- .../TimeBoundaryQueryRunnerFactory.java | 2 +- .../TimeseriesQueryRunnerFactory.java | 2 +- .../query/topn/TopNQueryQueryToolChest.java | 6 +- .../query/topn/TopNQueryRunnerFactory.java | 2 +- .../ChainedExecutionQueryRunnerTest.java | 11 +- .../query/groupby/GroupByQueryRunnerTest.java | 72 +++++----- .../query/metadata/SegmentAnalyzerTest.java | 4 +- .../metadata/SegmentMetadataQueryTest.java | 4 +- .../query/search/SearchQueryRunnerTest.java | 4 +- .../query/select/SelectQueryRunnerTest.java | 16 +-- .../TimeBoundaryQueryRunnerTest.java | 5 +- .../TimeSeriesUnionQueryRunnerTest.java | 4 +- .../TimeseriesQueryRunnerBonusTest.java | 4 +- .../timeseries/TimeseriesQueryRunnerTest.java | 124 ++++++++-------- .../druid/query/topn/TopNQueryRunnerTest.java | 132 +++++++++--------- .../druid/query/topn/TopNUnionQueryTest.java | 4 +- .../filter/SpatialFilterBonusTest.java | 8 +- .../segment/filter/SpatialFilterTest.java | 8 +- .../druid/client/CachingClusteredClient.java | 6 +- .../io/druid/client/CachingQueryRunner.java | 6 +- .../io/druid/client/DirectDruidClient.java | 4 +- .../java/io/druid/server/QueryResource.java | 10 +- .../bridge/BridgeQuerySegmentWalker.java | 2 +- .../client/CachingClusteredClientTest.java | 38 ++--- .../druid/client/CachingQueryRunnerTest.java | 12 +- .../druid/client/DirectDruidClientTest.java | 20 +-- .../coordination/ServerManagerTest.java | 8 +- 52 files changed, 324 insertions(+), 329 deletions(-) diff --git a/processing/src/main/java/io/druid/query/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java index 3dac2fa0cfd..ad1c96a23d6 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, Map metadata) + public Sequence run(QuerySegmentWalker walker, Map context) { - return run(querySegmentSpec.lookup(this, walker), metadata); + return run(querySegmentSpec.lookup(this, walker), context); } - public Sequence run(QueryRunner runner, Map metadata) + public Sequence run(QueryRunner runner, Map context) { - return runner.run(this, metadata); + 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 1d41a58b58b..a537c7d48f5 100644 --- a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java @@ -52,10 +52,10 @@ public class BySegmentQueryRunner implements QueryRunner @Override @SuppressWarnings("unchecked") - public Sequence run(final Query query, Map metadata) + public Sequence run(final Query query, Map context) { if (query.getContextBySegment(false)) { - final Sequence baseSequence = base.run(query, metadata); + final Sequence baseSequence = base.run(query, context); return new Sequence() { @Override @@ -97,6 +97,6 @@ public class BySegmentQueryRunner implements QueryRunner } }; } - return base.run(query, metadata); + 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 09cf38ee40b..5f9651f5222 100644 --- a/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java @@ -21,7 +21,6 @@ package io.druid.query; import com.metamx.common.guava.Sequence; -import java.util.List; import java.util.Map; /** @@ -38,14 +37,14 @@ public abstract class BySegmentSkippingQueryRunner implements QueryRunner } @Override - public Sequence run(Query query, Map metadata) + public Sequence run(Query query, Map context) { if (query.getContextBySegment(false)) { - return baseRunner.run(query, metadata); + return baseRunner.run(query, context); } - return doRun(baseRunner, query, metadata); + return doRun(baseRunner, query, context); } - protected abstract Sequence doRun(QueryRunner baseRunner, Query query, Map metadata); + 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 45a9f5518d8..81340e16236 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -94,7 +94,7 @@ public class ChainedExecutionQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query, final Map metadata) + public Sequence run(final Query query, final Map context) { final int priority = query.getContextPriority(0); @@ -125,7 +125,7 @@ public class ChainedExecutionQueryRunner implements QueryRunner throw new ISE("Input is null?! How is this possible?!"); } - Sequence result = input.run(query, metadata); + 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 482216fe818..74c4a6481f5 100644 --- a/processing/src/main/java/io/druid/query/ConcatQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ConcatQueryRunner.java @@ -39,7 +39,7 @@ public class ConcatQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query, final Map metadata) + public Sequence run(final Query query, final Map context) { return Sequences.concat( Sequences.map( @@ -49,7 +49,7 @@ public class ConcatQueryRunner implements QueryRunner @Override public Sequence apply(final QueryRunner input) { - return input.run(query, metadata); + 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 565141aad7d..8fb5efeb4d2 100644 --- a/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java +++ b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java @@ -48,7 +48,7 @@ public class FinalizeResultsQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query, Map metadata) + public Sequence run(final Query query, Map context) { final boolean isBySegment = query.getContextBySegment(false); final boolean shouldFinalize = query.getContextFinalize(true); @@ -96,7 +96,7 @@ public class FinalizeResultsQueryRunner implements QueryRunner return Sequences.map( - baseRunner.run(queryToRun, metadata), + 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 ef1bd009523..48db79f1ec3 100644 --- a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java @@ -87,7 +87,7 @@ public class GroupByParallelQueryRunner implements QueryRunner } @Override - public Sequence run(final Query queryParam, final Map metadata) + public Sequence run(final Query queryParam, final Map context) { final GroupByQuery query = (GroupByQuery) queryParam; @@ -116,7 +116,7 @@ public class GroupByParallelQueryRunner implements QueryRunner public Boolean call() throws Exception { try { - input.run(queryParam, metadata).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); + input.run(queryParam, context).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); return true; } catch (QueryInterruptedException e) { diff --git a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java index 42f5093468f..f36a7f4edfa 100644 --- a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java @@ -49,10 +49,10 @@ public class IntervalChunkingQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query, final Map metadata) + public Sequence run(final Query query, final Map context) { if (period.getMillis() == 0) { - return baseRunner.run(query, metadata); + return baseRunner.run(query, context); } return Sequences.concat( @@ -76,7 +76,7 @@ public class IntervalChunkingQueryRunner implements QueryRunner { return baseRunner.run( query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(singleInterval))), - metadata + context ); } } diff --git a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java index 110aadf4ce5..355916f4b74 100644 --- a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java @@ -68,7 +68,7 @@ public class MetricsEmittingQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query, final Map metadata) + public Sequence run(final Query query, final Map context) { final ServiceMetricEvent.Builder builder = builderFn.apply(query); String queryId = query.getId(); @@ -86,7 +86,7 @@ public class MetricsEmittingQueryRunner implements QueryRunner long startTime = System.currentTimeMillis(); try { - retVal = queryRunner.run(query, metadata).accumulate(outType, accumulator); + retVal = queryRunner.run(query, context).accumulate(outType, accumulator); } catch (RuntimeException e) { builder.setUser10("failed"); @@ -116,7 +116,7 @@ public class MetricsEmittingQueryRunner implements QueryRunner long startTime = System.currentTimeMillis(); try { - retVal = queryRunner.run(query, metadata).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 0f659d01793..d2f3863ab62 100644 --- a/processing/src/main/java/io/druid/query/NoopQueryRunner.java +++ b/processing/src/main/java/io/druid/query/NoopQueryRunner.java @@ -30,7 +30,7 @@ import java.util.Map; public class NoopQueryRunner implements QueryRunner { @Override - public Sequence run(Query query, Map metadata) + 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 74484b7ab96..3a6c38dc028 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, Map metadata); + public Sequence run(QuerySegmentWalker walker, Map context); - public Sequence run(QueryRunner runner, Map metadata); + 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 81b5f4b39ad..d7a3f8af36f 100644 --- a/processing/src/main/java/io/druid/query/QueryRunner.java +++ b/processing/src/main/java/io/druid/query/QueryRunner.java @@ -21,12 +21,11 @@ package io.druid.query; import com.metamx.common.guava.Sequence; -import java.util.List; import java.util.Map; /** */ public interface QueryRunner { - public Sequence run(Query query, Map metadata); + 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 457b645544b..f104d8db026 100644 --- a/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java @@ -45,11 +45,11 @@ public class ReferenceCountingSegmentQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query, Map metadata) + public Sequence run(final Query query, Map context) { final Closeable closeable = adapter.increment(); try { - final Sequence baseSequence = factory.createRunner(adapter).run(query, metadata); + 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 c33cd032161..f5378cfc4ed 100644 --- a/processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java @@ -39,9 +39,9 @@ public abstract class ResultMergeQueryRunner extends BySegmentSkippingQueryRu } @Override - public Sequence doRun(QueryRunner baseRunner, Query query, Map metadata) + public Sequence doRun(QueryRunner baseRunner, Query query, Map context) { - return CombiningSequence.create(baseRunner.run(query, metadata), 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 index 054ac19313d..34781cfbb07 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunner.java @@ -42,20 +42,20 @@ public class RetryQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query, Map metadata) + public Sequence run(final Query query, Map context) { - Sequence returningSeq = baseRunner.run(query, metadata); + Sequence returningSeq = baseRunner.run(query, context); for (int i = config.numTries(); i > 0; i--) { - for (int j = metadata.get("missingSegments").size(); j > 0; j--) { - QuerySegmentSpec segmentSpec = new SpecificSegmentSpec((SegmentDescriptor)metadata.get("missingSegments").remove(0)); + for (int j = ((List)context.get("missingSegments")).size(); j > 0; j--) { + QuerySegmentSpec segmentSpec = new SpecificSegmentSpec((SegmentDescriptor)((List) context.get("missingSegments")).remove(0)); returningSeq = toolChest.mergeSequences( Sequences.simple( Arrays.asList( returningSeq, baseRunner.run( query.withQuerySegmentSpec(segmentSpec), - metadata + context ) ) ) diff --git a/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java b/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java index dc7994cd01e..d16a660e25a 100644 --- a/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java @@ -39,13 +39,13 @@ public class SubqueryQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query, Map metadata) + public Sequence run(final Query query, Map context) { DataSource dataSource = query.getDataSource(); if (dataSource instanceof QueryDataSource) { - return run((Query) ((QueryDataSource) dataSource).getQuery(), metadata); + return run((Query) ((QueryDataSource) dataSource).getQuery(), context); } else { - return baseRunner.run(query, metadata); + return baseRunner.run(query, context); } } } diff --git a/processing/src/main/java/io/druid/query/UnionQueryRunner.java b/processing/src/main/java/io/druid/query/UnionQueryRunner.java index d081d20cafe..dcaab858b9f 100644 --- a/processing/src/main/java/io/druid/query/UnionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/UnionQueryRunner.java @@ -44,7 +44,7 @@ public class UnionQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query, final Map metadata) + public Sequence run(final Query query, final Map context) { DataSource dataSource = query.getDataSource(); if (dataSource instanceof UnionDataSource) { @@ -59,7 +59,7 @@ public class UnionQueryRunner implements QueryRunner { return baseRunner.run( query.withDataSource(singleSource), - metadata + context ); } } @@ -67,7 +67,7 @@ public class UnionQueryRunner implements QueryRunner ) ); } else { - return baseRunner.run(query, metadata); + return baseRunner.run(query, context); } } 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 6ce42b14b53..d6ce385ed21 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -80,18 +80,18 @@ public class GroupByQueryQueryToolChest extends QueryToolChest() { @Override - public Sequence run(Query input, Map metadata) + public Sequence run(Query input, Map context) { if (Boolean.valueOf((String) input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) { - return mergeGroupByResults(((GroupByQuery) input).withOverriddenContext(NO_MERGE_CONTEXT), runner, metadata); + return mergeGroupByResults(((GroupByQuery) input).withOverriddenContext(NO_MERGE_CONTEXT), runner, context); } else { - return runner.run(input, metadata); + return runner.run(input, context); } } }; } - private Sequence mergeGroupByResults(final GroupByQuery query, QueryRunner runner, Map metadata) + private Sequence mergeGroupByResults(final GroupByQuery query, QueryRunner runner, Map context) { Sequence result; @@ -105,12 +105,12 @@ public class GroupByQueryQueryToolChest extends QueryToolChest subqueryResult = mergeGroupByResults(subquery, runner, metadata); + Sequence subqueryResult = mergeGroupByResults(subquery, runner, context); IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(makeIncrementalIndex(subquery, subqueryResult)); result = engine.process(query, adapter); } else { - result = runner.run(query, metadata); + result = runner.run(query, context); } return postAggregate(query, makeIncrementalIndex(query, result)); 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 f9bd43c20dd..95fdea5d158 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java @@ -102,7 +102,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory() { @Override - public Sequence run(final Query query, final Map metadata) + public Sequence run(final Query query, final Map context) { ListenableFuture> future = queryExecutor.submit( @@ -112,7 +112,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory call() throws Exception { return new ExecutorExecutingSequence( - input.run(query, metadata), + input.run(query, context), queryExecutor ); } @@ -168,7 +168,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory run(Query input, Map metadata) + 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/SegmentMetadataQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java index 395db8908fb..5c527d20e78 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java @@ -45,7 +45,6 @@ import io.druid.segment.QueryableIndex; import io.druid.segment.Segment; import java.util.Arrays; -import java.util.List; import java.util.Map; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; @@ -75,7 +74,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory() { @Override - public Sequence run(Query inQ, Map metadata) + public Sequence run(Query inQ, Map context) { SegmentMetadataQuery query = (SegmentMetadataQuery) inQ; @@ -136,7 +135,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory run( final Query query, - final Map metadata + final Map context ) { final int priority = query.getContextPriority(0); @@ -146,7 +145,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory call() throws Exception { - return input.run(query, metadata); + 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 229101adf95..1c02924cc84 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -286,7 +286,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest> run( Query> input, - Map metadata + Map context ) { if (!(input instanceof SearchQuery)) { @@ -295,13 +295,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 d95d63887b5..12ef77064b8 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java @@ -71,7 +71,7 @@ public class SearchQueryRunner implements QueryRunner> @Override public Sequence> run( final Query> input, - Map metadata + Map context ) { if (!(input instanceof SearchQuery)) { 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 702d6295c57..5210a56ae6a 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java @@ -92,7 +92,7 @@ public class SelectQueryRunnerFactory @Override public Sequence> run( Query> input, - Map metadata + Map context ) { if (!(input instanceof SelectQuery)) { 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 cd8134cb2ee..80eaa28fa33 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java @@ -51,7 +51,7 @@ public class SpecificSegmentQueryRunner implements QueryRunner } @Override - public Sequence run(final Query input, final Map metadata) + public Sequence run(final Query input, final Map context) { final Query query = input.withQuerySegmentSpec(specificSpec); @@ -66,9 +66,9 @@ public class SpecificSegmentQueryRunner implements QueryRunner { Sequence returningSeq; try { - returningSeq = base.run(query, metadata); + returningSeq = base.run(query, context); } catch (NullStorageAdapterException e) { - metadata.get("missingSegments").add(((SpecificSegmentSpec) specificSpec).getDescriptor()); + ((List)context.get("missingSegments")).add(((SpecificSegmentSpec) specificSpec).getDescriptor()); returningSeq = Sequences.empty(); } return returningSeq; 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 b9a5216bedf..5700aa68185 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -96,13 +96,13 @@ public class TimeBoundaryQueryQueryToolChest { @Override protected Sequence> doRun( - QueryRunner> baseRunner, Query> input, Map metadata + QueryRunner> baseRunner, Query> input, Map context ) { TimeBoundaryQuery query = (TimeBoundaryQuery) input; return Sequences.simple( query.mergeResults( - Sequences.toList(baseRunner.run(query, metadata), 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 3858c0d8cc4..57f15221791 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -86,7 +86,7 @@ public class TimeBoundaryQueryRunnerFactory @Override public Sequence> run( Query> input, - Map metadata + Map context ) { if (!(input instanceof TimeBoundaryQuery)) { 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 1706378ae38..3da84f7ff24 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java @@ -93,7 +93,7 @@ public class TimeseriesQueryRunnerFactory @Override public Sequence> run( Query> input, - Map metadata + Map context ) { if (!(input instanceof TimeseriesQuery)) { 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 43fc6b9f303..d574e2ffc83 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -412,7 +412,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest> run( Query> input, - Map metadata + Map context ) { if (!(input instanceof TopNQuery)) { @@ -421,13 +421,13 @@ public class TopNQueryQueryToolChest extends QueryToolChest minTopNThreshold) { - return runner.run(query, metadata); + return runner.run(query, context); } final boolean isBySegment = query.getContextBySegment(false); return Sequences.map( - runner.run(query.withThreshold(minTopNThreshold), metadata), + 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 7b3c5cb07a9..54d5286254b 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java @@ -67,7 +67,7 @@ public class TopNQueryRunnerFactory implements QueryRunnerFactory> run( Query> input, - Map metadata + Map context ) { if (!(input instanceof TopNQuery)) { diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index 4c1b7351c0d..9166b7f5b09 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -36,7 +36,6 @@ import org.junit.Assert; import org.junit.Test; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; @@ -102,14 +101,14 @@ public class ChainedExecutionQueryRunnerTest runner3 ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); final Sequence seq = chainedRunner.run( Druids.newTimeseriesQueryBuilder() .dataSource("test") .intervals("2014/2015") .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) .build(), - metadata + context ); Future resultFuture = Executors.newFixedThreadPool(1).submit( @@ -206,7 +205,7 @@ public class ChainedExecutionQueryRunnerTest runner3 ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); final Sequence seq = chainedRunner.run( Druids.newTimeseriesQueryBuilder() .dataSource("test") @@ -214,7 +213,7 @@ public class ChainedExecutionQueryRunnerTest .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) .context(ImmutableMap.of("timeout", (100), "queryId", "test")) .build(), - metadata + context ); Future resultFuture = Executors.newFixedThreadPool(1).submit( @@ -268,7 +267,7 @@ public class ChainedExecutionQueryRunnerTest } @Override - public Sequence run(Query query, Map metadata) + public Sequence run(Query query, Map context) { hasStarted = true; latch.countDown(); 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 ac073b6f723..d754cbebc62 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -344,7 +344,7 @@ public class GroupByQueryRunnerTest new QueryRunner() { @Override - public Sequence run(Query query, Map metadata) + public Sequence run(Query query, Map context) { // simulate two daily segments final Query query1 = query.withQuerySegmentSpec( @@ -353,7 +353,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, metadata), runner.run(query2, metadata)); + return Sequences.concat(runner.run(query1, context), runner.run(query2, context)); } } ); @@ -370,9 +370,9 @@ public class GroupByQueryRunnerTest createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedObjects(expectedResults, runner.run(fullQuery, metadata), "direct"); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, metadata), "merged"); + HashMap context = new HashMap(); + TestHelper.assertExpectedObjects(expectedResults, runner.run(fullQuery, context), "direct"); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged"); List allGranExpectedResults = Arrays.asList( createExpectedRow("2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L), @@ -386,8 +386,8 @@ public class GroupByQueryRunnerTest createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) ); - TestHelper.assertExpectedObjects(allGranExpectedResults, runner.run(allGranQuery, metadata), "direct"); - TestHelper.assertExpectedObjects(allGranExpectedResults, mergedRunner.run(allGranQuery, metadata), "merged"); + TestHelper.assertExpectedObjects(allGranExpectedResults, runner.run(allGranQuery, context), "direct"); + TestHelper.assertExpectedObjects(allGranExpectedResults, mergedRunner.run(allGranQuery, context), "merged"); } @Test @@ -429,9 +429,9 @@ public class GroupByQueryRunnerTest ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery, metadata), String.format("limit: %d", limit) + Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery, context), String.format("limit: %d", limit) ); } @@ -537,7 +537,7 @@ public class GroupByQueryRunnerTest new QueryRunner() { @Override - public Sequence run(Query query, Map metadata) + public Sequence run(Query query, Map context) { // simulate two daily segments final Query query1 = query.withQuerySegmentSpec( @@ -546,12 +546,12 @@ 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, metadata), runner.run(query2, metadata)); + return Sequences.concat(runner.run(query1, context), runner.run(query2, context)); } } ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, metadata), "merged"); + HashMap context = new HashMap(); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged"); } @Test @@ -587,11 +587,11 @@ public class GroupByQueryRunnerTest ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "no-limit"); + HashMap context = new HashMap(); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), metadata), "limited" + Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), context), "limited" ); } @@ -628,10 +628,10 @@ public class GroupByQueryRunnerTest ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "no-limit"); + HashMap context = new HashMap(); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), metadata), "limited" + Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), context), "limited" ); } @@ -668,10 +668,10 @@ public class GroupByQueryRunnerTest ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "no-limit"); + HashMap context = new HashMap(); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); TestHelper.assertExpectedObjects( - Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), metadata), "limited" + Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), context), "limited" ); } @@ -711,7 +711,7 @@ public class GroupByQueryRunnerTest new QueryRunner() { @Override - public Sequence run(Query query, Map metadata) + public Sequence run(Query query, Map context) { // simulate two daily segments final Query query1 = query.withQuerySegmentSpec( @@ -720,13 +720,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, metadata), runner.run(query2, metadata)); + return Sequences.concat(runner.run(query1, context), runner.run(query2, context)); } } ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, metadata), "merged"); + HashMap context = new HashMap(); + TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged"); } @Test @@ -765,9 +765,9 @@ public class GroupByQueryRunnerTest ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); } @Test @@ -799,8 +799,8 @@ public class GroupByQueryRunnerTest createExpectedRow("2011-04-01", "quality", "technology", "rows", 2L), createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedObjects(expectedResults, runner.run(query, metadata), "normal"); + HashMap context = new HashMap(); + TestHelper.assertExpectedObjects(expectedResults, runner.run(query, context), "normal"); final GroupByQueryEngine engine = new GroupByQueryEngine( configSupplier, new StupidPool( @@ -816,7 +816,7 @@ public class GroupByQueryRunnerTest ); QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); } @Test @@ -849,8 +849,8 @@ public class GroupByQueryRunnerTest createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedObjects(expectedResults, runner.run(query, metadata), "normal"); + HashMap context = new HashMap(); + TestHelper.assertExpectedObjects(expectedResults, runner.run(query, context), "normal"); final GroupByQueryEngine engine = new GroupByQueryEngine( configSupplier, new StupidPool( @@ -866,7 +866,7 @@ public class GroupByQueryRunnerTest ); QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "no-limit"); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); } // A subquery identical to the query should yield identical results @@ -1046,8 +1046,8 @@ public class GroupByQueryRunnerTest toolChest ); - HashMap metadata = new HashMap(); - Sequence queryResult = theRunner.run(query, metadata); + HashMap context = new HashMap(); + Sequence queryResult = theRunner.run(query, context); return Sequences.toList(queryResult, Lists.newArrayList()); } 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 bb2dcbdd85d..e6b50c2178f 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java @@ -102,7 +102,7 @@ public class SegmentAnalyzerTest final SegmentMetadataQuery query = new SegmentMetadataQuery( new LegacyDataSource("test"), QuerySegmentSpecs.create("2011/2012"), null, null, null ); - HashMap metadata = new HashMap(); - return Sequences.toList(query.run(runner, metadata), 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 3e627bdca49..091d1e2b1b8 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java @@ -72,9 +72,9 @@ public class SegmentMetadataQueryTest .toInclude(new ListColumnIncluderator(Arrays.asList("placement"))) .merge(true) .build(); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable results = Sequences.toList( - runner.run(query, metadata), + 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 1e64c284b84..fb14a4e8a0a 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java @@ -369,9 +369,9 @@ public class SearchQueryRunnerTest private void checkSearchQuery(SearchQuery searchQuery, Map> expectedResults) { - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(searchQuery, metadata), + 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 88a983be6d6..ba8bc3c6320 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java @@ -88,9 +88,9 @@ public class SelectQueryRunnerTest new PagingSpec(null, 3), null ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); @@ -157,9 +157,9 @@ public class SelectQueryRunnerTest new PagingSpec(null, 3), null ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); @@ -217,9 +217,9 @@ public class SelectQueryRunnerTest new PagingSpec(Maps.newLinkedHashMap(ImmutableMap.of(QueryRunnerTestHelper.segmentId, 3)), 3), null ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); @@ -277,9 +277,9 @@ public class SelectQueryRunnerTest new PagingSpec(Maps.newLinkedHashMap(ImmutableMap.of(QueryRunnerTestHelper.segmentId, 3)), 3), null ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), 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 28b378e08c9..3e7b919c5f5 100644 --- a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java @@ -34,7 +34,6 @@ import org.junit.runners.Parameterized; import java.io.IOException; import java.util.Collection; import java.util.HashMap; -import java.util.List; /** */ @@ -65,9 +64,9 @@ public class TimeBoundaryQueryRunnerTest TimeBoundaryQuery timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder() .dataSource("testing") .build(); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(timeBoundaryQuery, metadata), + 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 bedc433cc69..8ad6c40dbec 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java @@ -98,9 +98,9 @@ public class TimeSeriesUnionQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), 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 92821c4260c..056fdef948d 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java @@ -111,9 +111,9 @@ public class TimeseriesQueryRunnerBonusTest ) ) .build(); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); return Sequences.toList( - runner.run(query, metadata), + 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 17ee38449dd..1d1fcb21b00 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -101,10 +101,10 @@ public class TimeseriesQueryRunnerTest DateTime expectedEarliest = new DateTime("2011-01-12"); DateTime expectedLast = new DateTime("2011-04-15"); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); @@ -168,9 +168,9 @@ public class TimeseriesQueryRunnerTest DateTime expectedEarliest = new DateTime("2011-01-12"); DateTime expectedLast = new DateTime("2011-04-15"); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); Result result = results.iterator().next(); @@ -215,9 +215,9 @@ public class TimeseriesQueryRunnerTest DateTime expectedEarliest = new DateTime("2011-01-12"); DateTime expectedLast = new DateTime("2011-04-15"); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); @@ -281,9 +281,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); @@ -328,9 +328,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); @@ -370,9 +370,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results1 = Sequences.toList( - runner.run(query1, metadata), + runner.run(query1, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults1, results1); @@ -409,7 +409,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results2 = Sequences.toList( - runner.run(query2, metadata), + runner.run(query2, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults2, results2); @@ -460,9 +460,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results1 = Sequences.toList( - runner.run(query1, metadata), + runner.run(query1, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults1, results1); @@ -502,9 +502,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results1 = Sequences.toList( - runner.run(query1, metadata), + runner.run(query1, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults1, results1); @@ -542,7 +542,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results2 = Sequences.toList( - runner.run(query2, metadata), + runner.run(query2, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults2, results2); @@ -575,9 +575,9 @@ public class TimeseriesQueryRunnerTest .build(); List> expectedResults = Arrays.asList(); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -625,9 +625,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -675,9 +675,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -725,9 +725,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -775,9 +775,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -825,9 +825,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -883,9 +883,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -941,9 +941,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -999,9 +999,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1055,9 +1055,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1117,9 +1117,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1161,9 +1161,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1205,9 +1205,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1263,9 +1263,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1303,9 +1303,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> actualResults = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1344,9 +1344,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> actualResults = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1363,7 +1363,7 @@ public class TimeseriesQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> expectedResults = Sequences.toList( runner.run( Druids.newTimeseriesQueryBuilder() @@ -1373,12 +1373,12 @@ public class TimeseriesQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(), - metadata + context ), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1395,7 +1395,7 @@ public class TimeseriesQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> expectedResults = Sequences.toList( runner.run( Druids.newTimeseriesQueryBuilder() @@ -1406,12 +1406,12 @@ public class TimeseriesQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(), - metadata + context ), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1457,7 +1457,7 @@ public class TimeseriesQueryRunnerTest ) ) .build(); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> expectedResults = Sequences.toList( runner.run( Druids.newTimeseriesQueryBuilder() @@ -1468,12 +1468,12 @@ public class TimeseriesQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(), - metadata + context ), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query, metadata), + runner.run(query, context), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1521,7 +1521,7 @@ public class TimeseriesQueryRunnerTest ) ) .build(); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); Iterable> expectedResults = Sequences.toList( runner.run( Druids.newTimeseriesQueryBuilder() @@ -1532,12 +1532,12 @@ public class TimeseriesQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(), - metadata + context ), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query, metadata), + 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 44a166aa503..fd3522028ef 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -167,8 +167,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -231,8 +231,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -296,8 +296,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -346,8 +346,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -395,8 +395,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -444,8 +444,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -486,8 +486,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -521,8 +521,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -570,8 +570,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -623,8 +623,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -665,8 +665,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -683,7 +683,7 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults( Lists.>newArrayList( new Result( @@ -691,7 +691,7 @@ public class TopNQueryRunnerTest new TopNResultValue(Lists.>newArrayList()) ) ), - runner.run(query, metadata) + runner.run(query, context) ); } @@ -722,7 +722,7 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults( Lists.>newArrayList( new Result( @@ -730,7 +730,7 @@ public class TopNQueryRunnerTest new TopNResultValue(Lists.>newArrayList()) ) ), - runner.run(query, metadata) + runner.run(query, context) ); } @@ -748,7 +748,7 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults( Sequences.toList( runner.run( @@ -763,9 +763,9 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(), - metadata + context ), Lists.>newArrayList() - ), runner.run(query, metadata) + ), runner.run(query, context) ); } @@ -783,7 +783,7 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults( Sequences.toList( runner.run( @@ -798,10 +798,10 @@ public class TopNQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(), - metadata + context ), Lists.>newArrayList() ) - , runner.run(query, metadata) + , runner.run(query, context) ); } @@ -843,8 +843,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -892,8 +892,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -948,8 +948,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -996,8 +996,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1037,8 +1037,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1078,8 +1078,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1119,8 +1119,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1160,8 +1160,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1212,8 +1212,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1264,8 +1264,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1316,8 +1316,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1361,8 +1361,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @@ -1407,8 +1407,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1452,8 +1452,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1501,8 +1501,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1586,8 +1586,8 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } @Test @@ -1669,7 +1669,7 @@ public class TopNQueryRunnerTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + 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 4df1aa50818..35f80552127 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java @@ -174,8 +174,8 @@ public class TopNUnionQueryTest ) ) ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } 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 7b76e2d0f7d..7c28ac39612 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java @@ -449,8 +449,8 @@ public class SpatialFilterBonusTest factory.createRunner(segment), factory.getToolchest() ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } catch (Exception e) { throw Throwables.propagate(e); @@ -536,8 +536,8 @@ public class SpatialFilterBonusTest factory.createRunner(segment), factory.getToolchest() ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + 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 631d06c94d2..ed46d81b0c9 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java @@ -479,8 +479,8 @@ public class SpatialFilterTest factory.createRunner(segment), factory.getToolchest() ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } catch (Exception e) { throw Throwables.propagate(e); @@ -566,8 +566,8 @@ public class SpatialFilterTest factory.createRunner(segment), factory.getToolchest() ); - HashMap metadata = new HashMap(); - TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata)); + HashMap context = new HashMap(); + TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } catch (Exception e) { throw Throwables.propagate(e); diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 339849347ad..4983c7273dc 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -115,7 +115,7 @@ public class CachingClusteredClient implements QueryRunner } @Override - public Sequence run(final Query query, final Map metadata) + public Sequence run(final Query query, final Map context) { final QueryToolChest> toolChest = warehouse.getToolChest(query); final CacheStrategy> strategy = toolChest.getCacheStrategy(query); @@ -327,11 +327,11 @@ public class CachingClusteredClient implements QueryRunner List intervals = segmentSpec.getIntervals(); if (!server.isAssignable() || !populateCache || isBySegment) { - resultSeqToAdd = clientQueryable.run(query.withQuerySegmentSpec(segmentSpec), metadata); + resultSeqToAdd = clientQueryable.run(query.withQuerySegmentSpec(segmentSpec), context); } else { resultSeqToAdd = toolChest.mergeSequences( Sequences.map( - clientQueryable.run(rewrittenQuery.withQuerySegmentSpec(segmentSpec), metadata), + 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 2a5c932e73c..47f2ae8facf 100644 --- a/server/src/main/java/io/druid/client/CachingQueryRunner.java +++ b/server/src/main/java/io/druid/client/CachingQueryRunner.java @@ -73,7 +73,7 @@ public class CachingQueryRunner implements QueryRunner } @Override - public Sequence run(Query query, Map metadata) + public Sequence run(Query query, Map context) { final CacheStrategy strategy = toolChest.getCacheStrategy(query); @@ -141,7 +141,7 @@ public class CachingQueryRunner implements QueryRunner return Sequences.withEffect( Sequences.map( - base.run(query, metadata), + base.run(query, context), new Function() { @Override @@ -163,7 +163,7 @@ public class CachingQueryRunner implements QueryRunner MoreExecutors.sameThreadExecutor() ); } else { - return base.run(query, metadata); + 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 5bf0747a0cf..541ae1df62e 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -114,7 +114,7 @@ public class DirectDruidClient implements QueryRunner } @Override - public Sequence run(final Query query, final Map metadata) + public Sequence run(final Query query, final Map context) { QueryToolChest> toolChest = warehouse.getToolChest(query); boolean isBySegment = query.getContextBySegment(false); @@ -170,7 +170,7 @@ public class DirectDruidClient implements QueryRunner } catch (IOException e) { } - metadata.get("missingSegments").addAll(missingSegments); + ((List) context.get("missingSegments")).addAll(missingSegments); } return super.handleResponse(response); diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 2d13ae3cfa9..ebc33c9670f 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -142,9 +142,9 @@ public class QueryResource log.debug("Got query [%s]", query); } - HashMap metadata = new HashMap(); - metadata.put("missingSegments", new LinkedList()); - Sequence results = query.run(texasRanger, metadata); + HashMap context = new HashMap(); + context.put("missingSegments", new LinkedList()); + Sequence results = query.run(texasRanger, context); if (results == null) { results = Sequences.empty(); @@ -166,8 +166,8 @@ public class QueryResource ) { String missingSegments = ""; - if (!metadata.get("missingSegments").isEmpty()) { - missingSegments = jsonMapper.writeValueAsString(metadata.get("missingSegments")); + if (!((List)context.get("missingSegments")).isEmpty()) { + missingSegments = jsonMapper.writeValueAsString(context.get("missingSegments")); } long requestTime = System.currentTimeMillis() - start; 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 5590fd420b8..4f260002ccb 100644 --- a/server/src/main/java/io/druid/server/bridge/BridgeQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/server/bridge/BridgeQuerySegmentWalker.java @@ -87,7 +87,7 @@ public class BridgeQuerySegmentWalker implements QuerySegmentWalker return new QueryRunner() { @Override - public Sequence run(Query query, Map metadata) + public Sequence run(Query query, Map context) { try { Server instance = brokerSelector.pick(); diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 9c75067f678..45a3f544900 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -263,7 +263,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-09T01"), 181, 52 ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults( makeRenamedTimeResults( new DateTime("2011-01-01"), 50, 5000, @@ -285,7 +285,7 @@ public class CachingClusteredClientTest .aggregators(RENAMED_AGGS) .postAggregators(RENAMED_POST_AGGS) .build(), - metadata + context ) ); } @@ -316,7 +316,7 @@ public class CachingClusteredClientTest new DateTime("2011-11-07", TIMEZONE), 85, 102 ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults( makeRenamedTimeResults( new DateTime("2011-11-04", TIMEZONE), 50, 5000, @@ -329,7 +329,7 @@ public class CachingClusteredClientTest .aggregators(RENAMED_AGGS) .postAggregators(RENAMED_POST_AGGS) .build(), - metadata + context ) ); } @@ -445,7 +445,7 @@ public class CachingClusteredClientTest new DateTime("2011-01-09T01"), "c2", 50, 4985, "b", 50, 4984, "c", 50, 4983 ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults( makeRenamedTopNResults( new DateTime("2011-01-01"), "a", 50, 5000, "b", 50, 4999, "c", 50, 4998, @@ -467,7 +467,7 @@ public class CachingClusteredClientTest .aggregators(RENAMED_AGGS) .postAggregators(RENAMED_POST_AGGS) .build(), - metadata + context ) ); } @@ -501,7 +501,7 @@ public class CachingClusteredClientTest new DateTime("2011-11-07", TIMEZONE), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986 ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults( makeRenamedTopNResults( @@ -516,7 +516,7 @@ public class CachingClusteredClientTest .aggregators(RENAMED_AGGS) .postAggregators(RENAMED_POST_AGGS) .build(), - metadata + context ) ); } @@ -566,7 +566,7 @@ public class CachingClusteredClientTest ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults( makeRenamedTopNResults( new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, @@ -586,7 +586,7 @@ public class CachingClusteredClientTest .aggregators(RENAMED_AGGS) .postAggregators(RENAMED_POST_AGGS) .build(), - metadata + context ) ); } @@ -635,7 +635,7 @@ public class CachingClusteredClientTest ) ); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults( makeTopNResults( new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, @@ -655,7 +655,7 @@ public class CachingClusteredClientTest .aggregators(AGGS) .postAggregators(POST_AGGS) .build(), - metadata + context ) ); } @@ -763,7 +763,7 @@ public class CachingClusteredClientTest .once(); final Capture capture = new Capture(); - final Capture metadata = new Capture(); + final Capture context = new Capture(); queryCaptures.add(capture); QueryRunner queryable = expectations.getQueryRunner(); @@ -776,7 +776,7 @@ public class CachingClusteredClientTest intervals.add(expectation.getInterval()); results.add(expectation.getResults()); } - EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(metadata))) + EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(context))) .andReturn(toQueryableTimeseriesResults(expectBySegment, segmentIds, intervals, results)) .once(); @@ -789,7 +789,7 @@ public class CachingClusteredClientTest intervals.add(expectation.getInterval()); results.add(expectation.getResults()); } - EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(metadata))) + EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(context))) .andReturn(toQueryableTopNResults(segmentIds, intervals, results)) .once(); } else if (query instanceof SearchQuery) { @@ -801,7 +801,7 @@ public class CachingClusteredClientTest intervals.add(expectation.getInterval()); results.add(expectation.getResults()); } - EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(metadata))) + EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(context))) .andReturn(toQueryableSearchResults(segmentIds, intervals, results)) .once(); } else if (query instanceof TimeBoundaryQuery) { @@ -813,7 +813,7 @@ public class CachingClusteredClientTest intervals.add(expectation.getInterval()); results.add(expectation.getResults()); } - EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(metadata))) + EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(context))) .andReturn(toQueryableTimeBoundaryResults(segmentIds, intervals, results)) .once(); } else { @@ -837,7 +837,7 @@ public class CachingClusteredClientTest @Override public void run() { - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); for (int i = 0; i < numTimesToQuery; ++i) { TestHelper.assertExpectedResults( new MergeIterable<>( @@ -872,7 +872,7 @@ public class CachingClusteredClientTest ) ) ), - metadata + context ) ); } diff --git a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java index e669d847f9d..a358ec44798 100644 --- a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java @@ -124,7 +124,7 @@ public class CachingQueryRunnerTest new QueryRunner() { @Override - public Sequence run(Query query, Map metadata) + public Sequence run(Query query, Map context) { return resultSeq; } @@ -141,8 +141,8 @@ public class CachingQueryRunnerTest cacheStrategy.computeCacheKey(query) ); - HashMap metadata = new HashMap(); - Sequence res = runner.run(query, metadata); + HashMap context = new HashMap(); + Sequence res = runner.run(query, context); // base sequence is not closed yet Assert.assertFalse("sequence must not be closed", closable.isClosed()); Assert.assertNull("cache must be empty", cache.get(cacheKey)); @@ -214,7 +214,7 @@ public class CachingQueryRunnerTest new QueryRunner() { @Override - public Sequence run(Query query, Map metadata) + public Sequence run(Query query, Map context) { return Sequences.empty(); } @@ -222,8 +222,8 @@ public class CachingQueryRunnerTest new CacheConfig() ); - HashMap metadata = new HashMap(); - List results = Sequences.toList(runner.run(query, metadata), new ArrayList()); + HashMap context = new HashMap(); + List results = Sequences.toList(runner.run(query, context), new ArrayList()); Assert.assertEquals(expectedResults, results); } diff --git a/server/src/test/java/io/druid/client/DirectDruidClientTest.java b/server/src/test/java/io/druid/client/DirectDruidClientTest.java index d6fbb1b7c9d..8d2ea4daf0d 100644 --- a/server/src/test/java/io/druid/client/DirectDruidClientTest.java +++ b/server/src/test/java/io/druid/client/DirectDruidClientTest.java @@ -117,20 +117,20 @@ public class DirectDruidClientTest serverSelector.addServer(queryableDruidServer2); TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build(); - HashMap metadata = new HashMap(); - Sequence s1 = client1.run(query, metadata); + HashMap context = new HashMap(); + Sequence s1 = client1.run(query, context); Assert.assertEquals(1, client1.getNumOpenConnections()); // simulate read timeout - Sequence s2 = client1.run(query, metadata); + Sequence s2 = client1.run(query, context); Assert.assertEquals(2, client1.getNumOpenConnections()); futureException.setException(new ReadTimeoutException()); Assert.assertEquals(1, client1.getNumOpenConnections()); // subsequent connections should work - Sequence s3 = client1.run(query, metadata); - Sequence s4 = client1.run(query, metadata); - Sequence s5 = client1.run(query, metadata); + Sequence s3 = client1.run(query, context); + Sequence s4 = client1.run(query, context); + Sequence s5 = client1.run(query, context); Assert.assertTrue(client1.getNumOpenConnections() == 4); @@ -141,8 +141,8 @@ public class DirectDruidClientTest Assert.assertEquals(new DateTime("2014-01-01T01:02:03Z"), results.get(0).getTimestamp()); Assert.assertEquals(3, client1.getNumOpenConnections()); - client2.run(query, metadata); - client2.run(query, metadata); + client2.run(query, context); + client2.run(query, context); Assert.assertTrue(client2.getNumOpenConnections() == 2); @@ -200,9 +200,9 @@ public class DirectDruidClientTest serverSelector.addServer(queryableDruidServer1); TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build(); - HashMap metadata = new HashMap(); + HashMap context = new HashMap(); cancellationFuture.set(new StatusResponseHolder(HttpResponseStatus.OK, new StringBuilder("cancelled"))); - Sequence results = client1.run(query, metadata); + Sequence results = client1.run(query, context); Assert.assertEquals(0, client1.getNumOpenConnections()); diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index 371bdcc3671..ed06679858b 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -429,8 +429,8 @@ public class ServerManagerTest @Override public void run() { - Map metadata = new HashMap(); - Sequence> seq = runner.run(query, metadata); + Map context = new HashMap(); + Sequence> seq = runner.run(query, context); Sequences.toList(seq, Lists.>newArrayList()); Iterator adaptersIter = factory.getAdapters().iterator(); @@ -679,9 +679,9 @@ public class ServerManagerTest } @Override - public Sequence run(Query query, Map metadata) + public Sequence run(Query query, Map context) { - return new BlockingSequence(runner.run(query, metadata), waitLatch, waitYieldLatch, notifyLatch); + return new BlockingSequence(runner.run(query, context), waitLatch, waitYieldLatch, notifyLatch); } } From 35e080bbc14a0853f13dc3e29425c8719894d329 Mon Sep 17 00:00:00 2001 From: jisookim0513 Date: Thu, 19 Jun 2014 14:14:54 -0700 Subject: [PATCH 014/107] fix concurrency issue with the map; introduce new exception; add incomplete retry query runner test --- .../java/io/druid/query/RetryQueryRunner.java | 22 +++- .../druid/query/RetryQueryRunnerConfig.java | 2 + .../spec/SpecificSegmentQueryRunner.java | 4 +- .../timeseries/TimeseriesQueryEngine.java | 4 +- .../io/druid/query/topn/TopNQueryEngine.java | 4 +- ...tion.java => SegmentMissingException.java} | 6 +- .../io/druid/query/RetryQueryRunnerTest.java | 116 ++++++++++++++++++ .../java/io/druid/server/QueryResource.java | 9 +- 8 files changed, 149 insertions(+), 18 deletions(-) rename processing/src/main/java/io/druid/segment/{NullStorageAdapterException.java => SegmentMissingException.java} (85%) create mode 100644 processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunner.java b/processing/src/main/java/io/druid/query/RetryQueryRunner.java index 34781cfbb07..0c60d630433 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunner.java @@ -21,8 +21,8 @@ package io.druid.query; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import io.druid.query.spec.QuerySegmentSpec; -import io.druid.query.spec.SpecificSegmentSpec; +import io.druid.query.spec.MultipleSpecificSegmentSpec; +import io.druid.segment.SegmentMissingException; import java.util.Arrays; import java.util.List; @@ -30,6 +30,7 @@ import java.util.Map; public class RetryQueryRunner implements QueryRunner { + public static String missingSegments = "missingSegments"; private final QueryRunner baseRunner; private final QueryToolChest> toolChest; private final RetryQueryRunnerConfig config; @@ -46,23 +47,32 @@ public class RetryQueryRunner implements QueryRunner { Sequence returningSeq = baseRunner.run(query, context); - for (int i = config.numTries(); i > 0; i--) { - for (int j = ((List)context.get("missingSegments")).size(); j > 0; j--) { - QuerySegmentSpec segmentSpec = new SpecificSegmentSpec((SegmentDescriptor)((List) context.get("missingSegments")).remove(0)); + + for (int i = config.numTries(); i > 0 && !((List)context.get(missingSegments)).isEmpty(); i--) { + List segList= (List)context.get(missingSegments); + ((List)context.get(missingSegments)).clear(); returningSeq = toolChest.mergeSequences( Sequences.simple( Arrays.asList( returningSeq, baseRunner.run( - query.withQuerySegmentSpec(segmentSpec), + query.withQuerySegmentSpec(new MultipleSpecificSegmentSpec(segList)), context ) ) ) ); + } + + if (!config.returnPartialResults() && !((List)context.get(missingSegments)).isEmpty()) { + String failedSegments = ""; + for (SegmentDescriptor segment : (List) context.get("missingSegments")) { + failedSegments = failedSegments + segment.toString() + " "; } + throw new SegmentMissingException("The following segments are missing: " + failedSegments); } return returningSeq; } } + diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java index 5b5ed2639b5..5759b2794bb 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java @@ -25,6 +25,8 @@ public class RetryQueryRunnerConfig { @JsonProperty private int numTries = 1; + private boolean returnPartialResults = false; public int numTries() { return numTries; } + public boolean returnPartialResults() { return returnPartialResults; } } 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 80eaa28fa33..9a457b96e04 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java @@ -27,7 +27,7 @@ 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.segment.NullStorageAdapterException; +import io.druid.segment.SegmentMissingException; import java.io.IOException; import java.util.List; @@ -67,7 +67,7 @@ public class SpecificSegmentQueryRunner implements QueryRunner Sequence returningSeq; try { returningSeq = base.run(query, context); - } catch (NullStorageAdapterException e) { + } catch (SegmentMissingException e) { ((List)context.get("missingSegments")).add(((SpecificSegmentSpec) specificSpec).getDescriptor()); returningSeq = Sequences.empty(); } 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 ee239c2c572..bcc3e13512f 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java @@ -27,7 +27,7 @@ import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.segment.Cursor; -import io.druid.segment.NullStorageAdapterException; +import io.druid.segment.SegmentMissingException; import io.druid.segment.StorageAdapter; import io.druid.segment.filter.Filters; @@ -40,7 +40,7 @@ public class TimeseriesQueryEngine public Sequence> process(final TimeseriesQuery query, final StorageAdapter adapter) { if (adapter == null) { - throw new NullStorageAdapterException( + 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/TopNQueryEngine.java b/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java index 1faf1fb7699..65f2be86580 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java @@ -32,7 +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.NullStorageAdapterException; +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 NullStorageAdapterException( + 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/segment/NullStorageAdapterException.java b/processing/src/main/java/io/druid/segment/SegmentMissingException.java similarity index 85% rename from processing/src/main/java/io/druid/segment/NullStorageAdapterException.java rename to processing/src/main/java/io/druid/segment/SegmentMissingException.java index 8d2b967afff..aade5e560ca 100644 --- a/processing/src/main/java/io/druid/segment/NullStorageAdapterException.java +++ b/processing/src/main/java/io/druid/segment/SegmentMissingException.java @@ -19,9 +19,11 @@ package io.druid.segment; -public class NullStorageAdapterException extends IllegalStateException +import com.metamx.common.ISE; + +public class SegmentMissingException extends ISE { - public NullStorageAdapterException(String formatText, Object... arguments) { + public SegmentMissingException(String formatText, Object... arguments) { super(String.format(formatText, arguments)); } } 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..45da8bcad0e --- /dev/null +++ b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java @@ -0,0 +1,116 @@ +package io.druid.query; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import com.google.common.collect.MapMaker; +import com.google.common.collect.Ordering; +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.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.MetricManipulationFn; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesResultValue; +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 +{ + + 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("missingSegments", Lists.newArrayList()); + RetryQueryRunner runner = new RetryQueryRunner( + new QueryRunner() + { + @Override + public Sequence run(Query query, Map context) + { + ((List)context.get(RetryQueryRunner.missingSegments)).add(new SegmentDescriptor(new Interval(178888, 1999999), "test", 1)); + return Sequences.empty(); + } + }, + new QueryToolChest() + { + @Override + public QueryRunner mergeResults(QueryRunner runner) + { + return null; + } + + @Override + public Sequence mergeSequences(Sequence seqOfSequences) + { + return new OrderedMergeSequence>(getOrdering(), seqOfSequences); + } + + @Override + public ServiceMetricEvent.Builder makeMetricBuilder(Query query) + { + return null; + } + + @Override + public Function makePreComputeManipulatorFn( + Query query, MetricManipulationFn fn + ) + { + return null; + } + + @Override + public TypeReference getResultTypeReference() + { + return null; + } + + public Ordering> getOrdering() + { + return Ordering.natural(); + } + }, + new RetryQueryRunnerConfig() + { + private int numTries = 1; + private boolean returnPartialResults = true; + + public int numTries() { return numTries; } + public boolean returnPartialResults() { return returnPartialResults; } + } + ); + + Iterable> actualResults = Sequences.toList( + runner.run(query, context), + Lists.>newArrayList() + ); + + Assert.assertTrue("Should return an empty sequence as a result", ((List) actualResults).size() == 0); + } +} \ No newline at end of file diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index ebc33c9670f..353e7014e06 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -22,9 +22,11 @@ package io.druid.server; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; +import com.google.api.client.util.Lists; import com.google.common.base.Charsets; import com.google.common.base.Joiner; 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; @@ -57,9 +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.HashMap; -import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.UUID; /** @@ -142,8 +143,8 @@ public class QueryResource log.debug("Got query [%s]", query); } - HashMap context = new HashMap(); - context.put("missingSegments", new LinkedList()); + Map context = new MapMaker().makeMap(); + context.put("missingSegments", Lists.newArrayList()); Sequence results = query.run(texasRanger, context); if (results == null) { From 8e85097999f5b35a480d99d288490f6fb6578f2e Mon Sep 17 00:00:00 2001 From: jisookim0513 Date: Thu, 19 Jun 2014 14:38:11 -0700 Subject: [PATCH 015/107] fix header so it passes the entire context --- .../io/druid/client/DirectDruidClient.java | 29 ++++++++++++------- .../java/io/druid/server/QueryResource.java | 6 ++-- 2 files changed, 21 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index 541ae1df62e..e69cea4e417 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -54,6 +54,7 @@ import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChestWarehouse; import io.druid.query.QueryWatcher; import io.druid.query.Result; +import io.druid.query.RetryQueryRunner; import io.druid.query.SegmentDescriptor; import io.druid.query.aggregation.MetricManipulatorFns; import org.jboss.netty.handler.codec.http.HttpChunk; @@ -64,8 +65,8 @@ import java.io.Closeable; import java.io.IOException; import java.io.InputStream; import java.net.URL; +import java.util.ArrayList; import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.concurrent.CancellationException; @@ -160,18 +161,24 @@ public class DirectDruidClient implements QueryRunner startTime = System.currentTimeMillis(); byteCount += response.getContent().readableBytes(); - if (!response.getHeader("Missing-Segments").equals("")) { - LinkedList missingSegments = new LinkedList(); - try { - missingSegments = objectMapper.readValue(response.getHeader("Missing-Segments"), LinkedList.class); - for (int i = missingSegments.size(); i > 0; i--) { - missingSegments.add(objectMapper.convertValue(missingSegments.remove(0), SegmentDescriptor.class)); - } + + List missingSegments = new ArrayList(); + try { + Map headerContext = objectMapper.readValue(response.getHeader("Context"), Map.class); + missingSegments = (List)headerContext.get(RetryQueryRunner.missingSegments); + for (int i = missingSegments.size(); i > 0; i--) { + missingSegments.add( + objectMapper.convertValue( + missingSegments.remove(0), + SegmentDescriptor.class + ) + ); } - catch (IOException e) { - } - ((List) context.get("missingSegments")).addAll(missingSegments); } + catch (IOException e) { + e.printStackTrace(); + } + ((List) context.get("missingSegments")).addAll(missingSegments); return super.handleResponse(response); } diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 353e7014e06..ce6e255cd0d 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -166,9 +166,9 @@ public class QueryResource ) ) { - String missingSegments = ""; + String headerContext = ""; if (!((List)context.get("missingSegments")).isEmpty()) { - missingSegments = jsonMapper.writeValueAsString(context.get("missingSegments")); + headerContext = jsonMapper.writeValueAsString(context); } long requestTime = System.currentTimeMillis() - start; @@ -213,7 +213,7 @@ public class QueryResource isSmile ? APPLICATION_JSON : APPLICATION_SMILE ) .header("X-Druid-Query-Id", queryId) - .header("Missing-Segments", missingSegments) + .header("Context", headerContext) .build(); } } From c4048d06dd84d4f1bb4075a55fe02cb9f98e56b9 Mon Sep 17 00:00:00 2001 From: jisookim0513 Date: Thu, 19 Jun 2014 14:42:04 -0700 Subject: [PATCH 016/107] delete an unused flag in QueryableIndexSegment --- .../src/main/java/io/druid/segment/QueryableIndexSegment.java | 1 - 1 file changed, 1 deletion(-) diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexSegment.java b/processing/src/main/java/io/druid/segment/QueryableIndexSegment.java index e5075fc80ab..c2a7ee18aa6 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexSegment.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexSegment.java @@ -29,7 +29,6 @@ public class QueryableIndexSegment implements Segment { private final QueryableIndex index; private final String identifier; - private boolean nullStorage = false; public QueryableIndexSegment(final String segmentIdentifier, QueryableIndex index) { From ff980091c5af58cd317ea94c3c0277fc6a3e98c3 Mon Sep 17 00:00:00 2001 From: jisookim0513 Date: Thu, 19 Jun 2014 14:43:58 -0700 Subject: [PATCH 017/107] delete an unused flag in IncrementalIndexSegment --- .../src/main/java/io/druid/segment/IncrementalIndexSegment.java | 1 - 1 file changed, 1 deletion(-) diff --git a/processing/src/main/java/io/druid/segment/IncrementalIndexSegment.java b/processing/src/main/java/io/druid/segment/IncrementalIndexSegment.java index ab5d7974dcb..f21f7f1fa09 100644 --- a/processing/src/main/java/io/druid/segment/IncrementalIndexSegment.java +++ b/processing/src/main/java/io/druid/segment/IncrementalIndexSegment.java @@ -31,7 +31,6 @@ public class IncrementalIndexSegment implements Segment { private final IncrementalIndex index; private final String segmentIdentifier; - private boolean nullStorage = false; public IncrementalIndexSegment( IncrementalIndex index, From bdb35e2d7e64b957450e06db73e02856e0d09644 Mon Sep 17 00:00:00 2001 From: jisookim0513 Date: Thu, 19 Jun 2014 17:50:50 -0700 Subject: [PATCH 018/107] fix retry logic and change the default value of retry to 0 --- .../java/io/druid/query/RetryQueryRunner.java | 68 +++++++++++-------- .../druid/query/RetryQueryRunnerConfig.java | 2 +- .../druid/client/CachingClusteredClient.java | 1 + 3 files changed, 41 insertions(+), 30 deletions(-) diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunner.java b/processing/src/main/java/io/druid/query/RetryQueryRunner.java index 0c60d630433..a6f267c371a 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunner.java @@ -20,11 +20,11 @@ package io.druid.query; import com.metamx.common.guava.Sequence; -import com.metamx.common.guava.Sequences; -import io.druid.query.spec.MultipleSpecificSegmentSpec; +import com.metamx.common.guava.Yielder; +import com.metamx.common.guava.YieldingAccumulator; +import com.metamx.common.guava.YieldingSequenceBase; import io.druid.segment.SegmentMissingException; -import java.util.Arrays; import java.util.List; import java.util.Map; @@ -35,7 +35,11 @@ public class RetryQueryRunner implements QueryRunner private final QueryToolChest> toolChest; private final RetryQueryRunnerConfig config; - public RetryQueryRunner(QueryRunner baseRunner, QueryToolChest> toolChest, RetryQueryRunnerConfig config) + public RetryQueryRunner( + QueryRunner baseRunner, + QueryToolChest> toolChest, + RetryQueryRunnerConfig config + ) { this.baseRunner = baseRunner; this.toolChest = toolChest; @@ -43,36 +47,42 @@ public class RetryQueryRunner implements QueryRunner } @Override - public Sequence run(final Query query, Map context) + public Sequence run(final Query query, final Map context) { - Sequence returningSeq = baseRunner.run(query, 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); - for (int i = config.numTries(); i > 0 && !((List)context.get(missingSegments)).isEmpty(); i--) { - List segList= (List)context.get(missingSegments); - ((List)context.get(missingSegments)).clear(); - returningSeq = toolChest.mergeSequences( - Sequences.simple( - Arrays.asList( - returningSeq, - baseRunner.run( - query.withQuerySegmentSpec(new MultipleSpecificSegmentSpec(segList)), - context - ) - ) - ) - ); - } + if (((List) context.get(missingSegments)).isEmpty()) { + return yielder; + } - if (!config.returnPartialResults() && !((List)context.get(missingSegments)).isEmpty()) { - String failedSegments = ""; - for (SegmentDescriptor segment : (List) context.get("missingSegments")) { - failedSegments = failedSegments + segment.toString() + " "; + for (int i = config.numTries(); i > 0 && !((List) context.get(missingSegments)).isEmpty(); i--) { + ((List) context.get(missingSegments)).clear(); + yielder = baseRunner.run(query, context).toYielder(initValue, accumulator); + if (((List) context.get(missingSegments)).isEmpty()) { + break; + } + } + + if (!config.returnPartialResults() && !((List) context.get(missingSegments)).isEmpty()) { + String failedSegments = ""; + for (SegmentDescriptor segment : (List) context.get("missingSegments")) { + failedSegments = failedSegments + segment.toString() + " "; + } + throw new SegmentMissingException("The following segments are missing: " + failedSegments); + } + + return yielder; } - throw new SegmentMissingException("The following segments are missing: " + failedSegments); - } - - return returningSeq; + }; } } diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java index 5759b2794bb..cbbb4a4d49d 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java @@ -24,7 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; public class RetryQueryRunnerConfig { @JsonProperty - private int numTries = 1; + private int numTries = 0; private boolean returnPartialResults = false; public int numTries() { return numTries; } diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 4983c7273dc..3d893469cd3 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -253,6 +253,7 @@ public class CachingClusteredClient implements QueryRunner addSequencesFromServer(listOfSequences); addSequencesFromCache(listOfSequences); + Collections.sort( listOfSequences, Ordering.natural().onResultOf(Pair.>lhsFn()) From 4e2b5b743f7f449abd89053b534af50d7346c9e4 Mon Sep 17 00:00:00 2001 From: jisookim0513 Date: Thu, 19 Jun 2014 18:18:58 -0700 Subject: [PATCH 019/107] add more unit tests --- .../io/druid/query/RetryQueryRunnerTest.java | 171 +++++++++++++++++- .../timeseries/TimeseriesQueryRunnerTest.java | 28 +-- .../druid/client/CachingClusteredClient.java | 1 - 3 files changed, 184 insertions(+), 16 deletions(-) diff --git a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java index 45da8bcad0e..8391d36544a 100644 --- a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java @@ -14,10 +14,12 @@ import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.timeseries.TimeseriesQuery; import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.segment.SegmentMissingException; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -97,6 +99,94 @@ public class RetryQueryRunnerTest } }, new RetryQueryRunnerConfig() + { + private int numTries = 0; + private boolean returnPartialResults = true; + + public int numTries() { return numTries; } + public boolean returnPartialResults() { return returnPartialResults; } + } + ); + + 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.missingSegments)).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("missingSegments", Lists.newArrayList()); + RetryQueryRunner runner = new RetryQueryRunner( + new QueryRunner() + { + @Override + public Sequence run(Query query, Map context) + { + if (context.get("count") == 0) { + ((List) context.get(RetryQueryRunner.missingSegments)).add( + new SegmentDescriptor( + new Interval( + 178888, + 1999999 + ), "test", 1 + ) + ); + context.put("count", 1); + return Sequences.empty(); + } else { + ArrayList lst = new ArrayList(); + lst.add("hello world"); + return Sequences.simple(lst); + } + } + }, + new QueryToolChest() + { + @Override + public QueryRunner mergeResults(QueryRunner runner) + { + return null; + } + + @Override + public Sequence mergeSequences(Sequence seqOfSequences) + { + return new OrderedMergeSequence>(getOrdering(), seqOfSequences); + } + + @Override + public ServiceMetricEvent.Builder makeMetricBuilder(Query query) + { + return null; + } + + @Override + public Function makePreComputeManipulatorFn( + Query query, MetricManipulationFn fn + ) + { + return null; + } + + @Override + public TypeReference getResultTypeReference() + { + return null; + } + + public Ordering> getOrdering() + { + return Ordering.natural(); + } + }, + new RetryQueryRunnerConfig() { private int numTries = 1; private boolean returnPartialResults = true; @@ -111,6 +201,85 @@ public class RetryQueryRunnerTest Lists.>newArrayList() ); - Assert.assertTrue("Should return an empty sequence as a result", ((List) actualResults).size() == 0); + 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.missingSegments)).size() == 0); } + + @Test(expected= SegmentMissingException.class) + public void testException() throws Exception + { + Map context = new MapMaker().makeMap(); + context.put("missingSegments", Lists.newArrayList()); + RetryQueryRunner runner = new RetryQueryRunner( + new QueryRunner() + { + @Override + public Sequence run(Query query, Map context) + { + ((List)context.get(RetryQueryRunner.missingSegments)).add(new SegmentDescriptor(new Interval(178888, 1999999), "test", 1)); + return Sequences.empty(); + } + }, + new QueryToolChest() + { + @Override + public QueryRunner mergeResults(QueryRunner runner) + { + return null; + } + + @Override + public Sequence mergeSequences(Sequence seqOfSequences) + { + return new OrderedMergeSequence>(getOrdering(), seqOfSequences); + } + + @Override + public ServiceMetricEvent.Builder makeMetricBuilder(Query query) + { + return null; + } + + @Override + public Function makePreComputeManipulatorFn( + Query query, MetricManipulationFn fn + ) + { + return null; + } + + @Override + public TypeReference getResultTypeReference() + { + return null; + } + + public Ordering> getOrdering() + { + return Ordering.natural(); + } + }, + new RetryQueryRunnerConfig() + { + private int numTries = 1; + private boolean returnPartialResults = false; + + public int numTries() { return numTries; } + public boolean returnPartialResults() { return returnPartialResults; } + } + ); + + 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.missingSegments)).size() == 1); + } + } \ No newline at end of file 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 1d1fcb21b00..acd60aa051e 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -252,20 +252,20 @@ public class TimeseriesQueryRunnerTest public void testTimeseries() { 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(); + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory( + "idx", + "index" + ), + QueryRunnerTestHelper.qualityUniques + ) + ) + .build(); List> expectedResults = Arrays.asList( new Result( diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 3d893469cd3..4983c7273dc 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -253,7 +253,6 @@ public class CachingClusteredClient implements QueryRunner addSequencesFromServer(listOfSequences); addSequencesFromCache(listOfSequences); - Collections.sort( listOfSequences, Ordering.natural().onResultOf(Pair.>lhsFn()) From 0244172ccbd7a29ad1584c252f0a35c230f6cae8 Mon Sep 17 00:00:00 2001 From: jisookim0513 Date: Thu, 19 Jun 2014 18:22:36 -0700 Subject: [PATCH 020/107] change missigSegments to a static variable --- .../java/io/druid/query/RetryQueryRunner.java | 4 +-- .../spec/SpecificSegmentQueryRunner.java | 2 +- .../io/druid/query/RetryQueryRunnerTest.java | 6 ++-- .../timeseries/TimeseriesQueryRunnerTest.java | 28 +++++++++---------- .../io/druid/client/DirectDruidClient.java | 2 +- .../java/io/druid/server/QueryResource.java | 5 ++-- 6 files changed, 24 insertions(+), 23 deletions(-) diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunner.java b/processing/src/main/java/io/druid/query/RetryQueryRunner.java index a6f267c371a..62d7b4e8da4 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunner.java @@ -30,7 +30,7 @@ import java.util.Map; public class RetryQueryRunner implements QueryRunner { - public static String missingSegments = "missingSegments"; + public static String missingSegments = RetryQueryRunner.missingSegments; private final QueryRunner baseRunner; private final QueryToolChest> toolChest; private final RetryQueryRunnerConfig config; @@ -74,7 +74,7 @@ public class RetryQueryRunner implements QueryRunner if (!config.returnPartialResults() && !((List) context.get(missingSegments)).isEmpty()) { String failedSegments = ""; - for (SegmentDescriptor segment : (List) context.get("missingSegments")) { + for (SegmentDescriptor segment : (List) context.get(RetryQueryRunner.missingSegments)) { failedSegments = failedSegments + segment.toString() + " "; } throw new SegmentMissingException("The following segments are missing: " + failedSegments); 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 9a457b96e04..add0b2c754e 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java @@ -68,7 +68,7 @@ public class SpecificSegmentQueryRunner implements QueryRunner try { returningSeq = base.run(query, context); } catch (SegmentMissingException e) { - ((List)context.get("missingSegments")).add(((SpecificSegmentSpec) specificSpec).getDescriptor()); + ((List)context.get(RetryQueryRunner.missingSegments)).add(((SpecificSegmentSpec) specificSpec).getDescriptor()); returningSeq = Sequences.empty(); } return returningSeq; diff --git a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java index 8391d36544a..595f3133ec6 100644 --- a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java @@ -48,7 +48,7 @@ public class RetryQueryRunnerTest public void testRunWithMissingSegments() throws Exception { Map context = new MapMaker().makeMap(); - context.put("missingSegments", Lists.newArrayList()); + context.put(RetryQueryRunner.missingSegments, Lists.newArrayList()); RetryQueryRunner runner = new RetryQueryRunner( new QueryRunner() { @@ -122,7 +122,7 @@ public class RetryQueryRunnerTest { Map context = new MapMaker().makeMap(); context.put("count", 0); - context.put("missingSegments", Lists.newArrayList()); + context.put(RetryQueryRunner.missingSegments, Lists.newArrayList()); RetryQueryRunner runner = new RetryQueryRunner( new QueryRunner() { @@ -214,7 +214,7 @@ public class RetryQueryRunnerTest public void testException() throws Exception { Map context = new MapMaker().makeMap(); - context.put("missingSegments", Lists.newArrayList()); + context.put(RetryQueryRunner.missingSegments, Lists.newArrayList()); RetryQueryRunner runner = new RetryQueryRunner( new QueryRunner() { 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 acd60aa051e..1d1fcb21b00 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -252,20 +252,20 @@ public class TimeseriesQueryRunnerTest public void testTimeseries() { 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(); + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory( + "idx", + "index" + ), + QueryRunnerTestHelper.qualityUniques + ) + ) + .build(); List> expectedResults = Arrays.asList( new Result( diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index e69cea4e417..204da241ff9 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -178,7 +178,7 @@ public class DirectDruidClient implements QueryRunner catch (IOException e) { e.printStackTrace(); } - ((List) context.get("missingSegments")).addAll(missingSegments); + ((List) context.get(RetryQueryRunner.missingSegments)).addAll(missingSegments); return super.handleResponse(response); } diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index ce6e255cd0d..dffb66de79d 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -42,6 +42,7 @@ import io.druid.query.DataSourceUtil; import io.druid.query.Query; import io.druid.query.QueryInterruptedException; import io.druid.query.QuerySegmentWalker; +import io.druid.query.RetryQueryRunner; import io.druid.server.log.RequestLogger; import org.joda.time.DateTime; @@ -144,7 +145,7 @@ public class QueryResource } Map context = new MapMaker().makeMap(); - context.put("missingSegments", Lists.newArrayList()); + context.put(RetryQueryRunner.missingSegments, Lists.newArrayList()); Sequence results = query.run(texasRanger, context); if (results == null) { @@ -167,7 +168,7 @@ public class QueryResource ) { String headerContext = ""; - if (!((List)context.get("missingSegments")).isEmpty()) { + if (!((List)context.get(RetryQueryRunner.missingSegments)).isEmpty()) { headerContext = jsonMapper.writeValueAsString(context); } From 1cb3fc250ef4a7340e09bd9d8231ded4d42986b1 Mon Sep 17 00:00:00 2001 From: jisookim0513 Date: Thu, 19 Jun 2014 18:29:00 -0700 Subject: [PATCH 021/107] change missigSegments to a static variable --- processing/src/main/java/io/druid/query/RetryQueryRunner.java | 2 +- .../java/io/druid/query/spec/SpecificSegmentQueryRunner.java | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunner.java b/processing/src/main/java/io/druid/query/RetryQueryRunner.java index 62d7b4e8da4..cc4aa8af402 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunner.java @@ -30,7 +30,7 @@ import java.util.Map; public class RetryQueryRunner implements QueryRunner { - public static String missingSegments = RetryQueryRunner.missingSegments; + public static String missingSegments = "missingSegments"; private final QueryRunner baseRunner; private final QueryToolChest> toolChest; private final RetryQueryRunnerConfig config; 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 add0b2c754e..77beb6cb745 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java @@ -27,6 +27,7 @@ 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.segment.SegmentMissingException; import java.io.IOException; From 341829133382f449efda8a0a23a14614e9c01a36 Mon Sep 17 00:00:00 2001 From: jisookim0513 Date: Fri, 20 Jun 2014 13:15:46 -0700 Subject: [PATCH 022/107] add a test for retrying multiple times --- .../druid/query/RetryQueryRunnerConfig.java | 1 + .../io/druid/query/RetryQueryRunnerTest.java | 96 +++++++++++++++++++ 2 files changed, 97 insertions(+) diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java index cbbb4a4d49d..2b8bb730b68 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java @@ -25,6 +25,7 @@ public class RetryQueryRunnerConfig { @JsonProperty private int numTries = 0; + @JsonProperty private boolean returnPartialResults = false; public int numTries() { return numTries; } diff --git a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java index 595f3133ec6..bf10a38863a 100644 --- a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java @@ -117,6 +117,8 @@ public class RetryQueryRunnerTest Assert.assertTrue("Should return an empty sequence as a result", ((List) actualResults).size() == 0); } + + @Test public void testRetry() throws Exception { @@ -210,6 +212,100 @@ public class RetryQueryRunnerTest Assert.assertTrue("Should have nothing in missingSegment list", ((List) context.get(RetryQueryRunner.missingSegments)).size() == 0); } + @Test + public void testRetryMultiple() throws Exception + { + Map context = new MapMaker().makeMap(); + context.put("count", 0); + context.put(RetryQueryRunner.missingSegments, 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.missingSegments)).add( + new SegmentDescriptor( + new Interval( + 178888, + 1999999 + ), "test", 1 + ) + ); + context.put("count", (int)context.get("count") + 1); + return Sequences.empty(); + } else { + ArrayList lst = new ArrayList(); + lst.add("hello world"); + return Sequences.simple(lst); + } + } + }, + new QueryToolChest() + { + @Override + public QueryRunner mergeResults(QueryRunner runner) + { + return null; + } + + @Override + public Sequence mergeSequences(Sequence seqOfSequences) + { + return new OrderedMergeSequence>(getOrdering(), seqOfSequences); + } + + @Override + public ServiceMetricEvent.Builder makeMetricBuilder(Query query) + { + return null; + } + + @Override + public Function makePreComputeManipulatorFn( + Query query, MetricManipulationFn fn + ) + { + return null; + } + + @Override + public TypeReference getResultTypeReference() + { + return null; + } + + public Ordering> getOrdering() + { + return Ordering.natural(); + } + }, + new RetryQueryRunnerConfig() + { + private int numTries = 4; + private boolean returnPartialResults = true; + + public int numTries() { return numTries; } + public boolean returnPartialResults() { return returnPartialResults; } + } + ); + + Iterable> actualResults = Sequences.toList( + runner.run(query, context), + Lists.>newArrayList() + ); + + 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.missingSegments)).size() == 0); + } + + @Test(expected= SegmentMissingException.class) public void testException() throws Exception { From 5e7dfaf298a634b996080204194406d72c61d6b8 Mon Sep 17 00:00:00 2001 From: jisookim0513 Date: Fri, 20 Jun 2014 13:45:04 -0700 Subject: [PATCH 023/107] change retry tests --- .../java/io/druid/query/RetryQueryRunnerConfig.java | 2 +- .../test/java/io/druid/query/RetryQueryRunnerTest.java | 10 ---------- 2 files changed, 1 insertion(+), 11 deletions(-) diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java index 2b8bb730b68..1846c15339f 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java @@ -24,7 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; public class RetryQueryRunnerConfig { @JsonProperty - private int numTries = 0; + private int numTries = 4; @JsonProperty private boolean returnPartialResults = false; diff --git a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java index bf10a38863a..16c8a51504a 100644 --- a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java @@ -203,11 +203,6 @@ public class RetryQueryRunnerTest Lists.>newArrayList() ); - 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.missingSegments)).size() == 0); } @@ -296,11 +291,6 @@ public class RetryQueryRunnerTest Lists.>newArrayList() ); - 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.missingSegments)).size() == 0); } From cc8a4d08471beabd970601f8685878a6c8400e34 Mon Sep 17 00:00:00 2001 From: jisookim0513 Date: Fri, 20 Jun 2014 13:49:33 -0700 Subject: [PATCH 024/107] revert changes to RetryQueryRunnerConfig --- .../src/main/java/io/druid/query/RetryQueryRunnerConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java index 1846c15339f..2b8bb730b68 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunnerConfig.java @@ -24,7 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; public class RetryQueryRunnerConfig { @JsonProperty - private int numTries = 4; + private int numTries = 0; @JsonProperty private boolean returnPartialResults = false; From c40a315c819ac472605837c6d26d8158bc01561b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 8 May 2014 18:06:26 -0700 Subject: [PATCH 025/107] initial support for LZ4 compression --- pom.xml | 2 +- processing/pom.xml | 4 + .../segment/FloatMetricColumnSerializer.java | 4 +- .../java/io/druid/segment/IndexMerger.java | 4 +- .../CompressedFloatBufferObjectStrategy.java | 20 +- .../data/CompressedFloatsIndexedSupplier.java | 61 ++++- .../CompressedFloatsSupplierSerializer.java | 17 +- .../CompressedLongBufferObjectStrategy.java | 19 +- .../data/CompressedLongsIndexedSupplier.java | 47 +++- .../CompressedLongsSupplierSerializer.java | 16 +- .../data/CompressedObjectStrategy.java | 254 +++++++++++++++--- .../FixedSizeCompressedObjectStrategy.java | 50 ++++ .../data/InMemoryCompressedFloats.java | 10 +- .../segment/data/InMemoryCompressedLongs.java | 10 +- .../CompressedFloatsIndexedSupplierTest.java | 5 +- ...ompressedFloatsSupplierSerializerTest.java | 12 +- .../CompressedLongsIndexedSupplierTest.java | 5 +- ...CompressedLongsSupplierSerializerTest.java | 8 +- 18 files changed, 447 insertions(+), 101 deletions(-) create mode 100644 processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java diff --git a/pom.xml b/pom.xml index 2b7f88276e8..642a0cf6887 100644 --- a/pom.xml +++ b/pom.xml @@ -379,7 +379,7 @@ net.jpountz.lz4 lz4 - 1.1.2 + 1.2.0 com.google.protobuf diff --git a/processing/pom.xml b/processing/pom.xml index 755d2f553f7..31ac9652129 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -82,6 +82,10 @@ com.davekoelle alphanum + + net.jpountz.lz4 + lz4 + diff --git a/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java index 4caa6b95fe2..520708fe015 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.CompressionStrategy.LZ4 // TODO define this somewhere else ); writer.open(); diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index 99f854da9af..87cfa25a2ed 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -54,6 +54,7 @@ import io.druid.query.aggregation.ToLowerCaseAggregatorFactory; import io.druid.segment.column.ColumnConfig; 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; @@ -594,7 +595,8 @@ 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.CompressionStrategy.LZ4 // TODO define this somewhere else ); timeWriter.open(); 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..e6294462afe 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,16 @@ 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 final int sizePer; + + private CompressedFloatBufferObjectStrategy(final ByteOrder order, final CompressionStrategy compression, final int sizePer) { super( order, @@ -64,7 +66,15 @@ public class CompressedFloatBufferObjectStrategy extends CompressedObjectStrateg { return into.asFloatBuffer().put(from); } - } + }, + compression ); + this.sizePer = sizePer; + } + + @Override + public int getSize() + { + return 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 1beccc5426f..35a3b03f0b2 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 ); } @@ -191,23 +197,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( @@ -254,8 +290,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 da967898dba..89fa28d2c84 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java @@ -36,27 +36,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; @@ -64,11 +66,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(); @@ -110,6 +114,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..b0b511278f1 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,16 @@ 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 final int sizePer; + + private CompressedLongBufferObjectStrategy(final ByteOrder order, final CompressionStrategy compression, final int sizePer) { super( order, @@ -64,8 +66,15 @@ public class CompressedLongBufferObjectStrategy extends CompressedObjectStrategy { return into.asLongBuffer().put(from); } - } + }, + compression ); + this.sizePer = sizePer; } + @Override + public int getSize() + { + return 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 8ad267168a5..56998d09886 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,15 +174,17 @@ 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); } - public CompressedLongsIndexedSupplier convertByteOrder(ByteOrder order) + public CompressedLongsIndexedSupplier convertByteOrder(ByteOrder order, CompressedObjectStrategy.CompressionStrategy compression) { return new CompressedLongsIndexedSupplier( totalSize, sizePer, - GenericIndexed.fromIterable(baseLongBuffers, CompressedLongBufferObjectStrategy.getBufferForOrder(order)) + GenericIndexed.fromIterable(baseLongBuffers, CompressedLongBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)), + compression ); } @@ -196,23 +202,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( @@ -259,8 +279,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 b0e63a8e391..97a7545009a 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java @@ -37,20 +37,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; @@ -58,11 +61,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(); @@ -104,6 +109,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..e0865e8ec80 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java @@ -21,31 +21,216 @@ package io.druid.segment.data; import com.google.common.base.Throwables; import com.metamx.common.guava.CloseQuietly; +import com.google.common.collect.Maps; 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 net.jpountz.lz4.LZ4UnknownSizeDecompressor; 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 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 +243,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..c79e0edc615 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java @@ -0,0 +1,50 @@ +/* + * 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 +{ + protected FixedSizeCompressedObjectStrategy( + ByteOrder order, + BufferConverter converter, + CompressionStrategy compression + ) + { + super(order, converter, compression); + } + + public abstract int getSize(); + + @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/InMemoryCompressedFloats.java b/processing/src/main/java/io/druid/segment/data/InMemoryCompressedFloats.java index d056158fa7d..e8ae40efbab 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.CompressionStrategy.LZ4; 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..9fd314569d5 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.CompressionStrategy.LZ4; 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/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java index d9cf4c5b6ab..e481ee8cef6 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java @@ -68,7 +68,8 @@ public class CompressedFloatsIndexedSupplierTest supplier = CompressedFloatsIndexedSupplier.fromFloatBuffer( FloatBuffer.wrap(vals), 5, - ByteOrder.nativeOrder() + ByteOrder.nativeOrder(), + CompressedObjectStrategy.CompressionStrategy.LZ4 ); indexed = supplier.get(); @@ -82,7 +83,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(), CompressedObjectStrategy.CompressionStrategy.LZ4 ); 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..3b888ae132f 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java @@ -39,13 +39,19 @@ public class CompressedFloatsSupplierSerializerTest 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, + CompressedObjectStrategy.CompressionStrategy.LZ4, + sizePer + ) + ), + CompressedObjectStrategy.CompressionStrategy.LZ4 ); 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..dd33d532498 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java @@ -66,7 +66,8 @@ public class CompressedLongsIndexedSupplierTest supplier = CompressedLongsIndexedSupplier.fromLongBuffer( LongBuffer.wrap(vals), 5, - ByteOrder.nativeOrder() + ByteOrder.nativeOrder(), + CompressedObjectStrategy.CompressionStrategy.LZ4 ); indexed = supplier.get(); @@ -78,7 +79,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(), CompressedObjectStrategy.CompressionStrategy.LZ4 ); 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..ee13c55087e 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java @@ -39,13 +39,15 @@ public class CompressedLongsSupplierSerializerTest 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, CompressedObjectStrategy.CompressionStrategy.LZ4, sizePer) + ), + CompressedObjectStrategy.CompressionStrategy.LZ4 ); serializer.open(); From f5f05e3a9be910c50c62159b1a046cef309126ce Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Fri, 11 Jul 2014 16:15:10 +0530 Subject: [PATCH 026/107] Sync changes from branch new-ingestion PR #599 Sync and Resolve Conflicts --- .../io/druid/indexer/IndexGeneratorJob.java | 24 +- .../common/index/YeOldePlumberSchool.java | 7 +- .../indexing/common/task/AppendTask.java | 4 +- .../indexing/common/task/DeleteTask.java | 122 -- .../druid/indexing/common/task/IndexTask.java | 6 +- .../druid/indexing/common/task/MergeTask.java | 4 +- .../indexing/common/task/MergeTaskBase.java | 1 - .../io/druid/indexing/common/task/Task.java | 1 - .../common/task/MergeTaskBaseTest.java | 2 +- .../indexing/common/task/TaskSerdeTest.java | 47 - .../segment/AbstractProgressIndicator.java | 61 + .../main/java/io/druid/segment/IndexIO.java | 6 +- .../java/io/druid/segment/IndexMaker.java | 1564 +++++++++++++++++ .../java/io/druid/segment/IndexMerger.java | 1182 ------------- .../io/druid/segment/IndexableAdapter.java | 9 +- .../segment/LoggingProgressIndicator.java | 100 ++ .../io/druid/segment/MMappedIndexAdapter.java | 197 --- .../io/druid/segment/ProgressIndicator.java | 37 + .../QueryableIndexIndexableAdapter.java | 31 +- .../main/java/io/druid/segment/Rowboat.java | 11 +- .../segment/RowboatFilteringIndexAdapter.java | 15 +- .../segment/column/ColumnCapabilities.java | 2 + .../column/ColumnCapabilitiesImpl.java | 25 + .../segment/data/GenericIndexedWriter.java | 2 - .../druid/segment/data/VSizeIndexedInts.java | 7 +- .../segment/data/VSizeIndexedWriter.java | 145 -- .../segment/incremental/IncrementalIndex.java | 95 +- .../incremental/IncrementalIndexAdapter.java | 19 +- .../incremental/IncrementalIndexSchema.java | 65 +- ...va => SpatialDimensionRowTransformer.java} | 7 +- .../DictionaryEncodedColumnPartSerde.java | 55 +- .../java/io/druid/segment/EmptyIndexTest.java | 10 +- ...dexMergerTest.java => IndexMakerTest.java} | 23 +- .../test/java/io/druid/segment/TestIndex.java | 8 +- .../filter/SpatialFilterBonusTest.java | 88 +- .../segment/filter/SpatialFilterTest.java | 102 +- .../inventory/CuratorInventoryManager.java | 3 +- .../realtime/plumber/RealtimePlumber.java | 6 +- .../druid/segment/realtime/plumber/Sink.java | 4 +- 39 files changed, 2134 insertions(+), 1963 deletions(-) delete mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/DeleteTask.java create mode 100644 processing/src/main/java/io/druid/segment/AbstractProgressIndicator.java create mode 100644 processing/src/main/java/io/druid/segment/IndexMaker.java delete mode 100644 processing/src/main/java/io/druid/segment/IndexMerger.java create mode 100644 processing/src/main/java/io/druid/segment/LoggingProgressIndicator.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 delete mode 100644 processing/src/main/java/io/druid/segment/data/VSizeIndexedWriter.java rename processing/src/main/java/io/druid/segment/incremental/{SpatialDimensionRowFormatter.java => SpatialDimensionRowTransformer.java} (96%) rename processing/src/test/java/io/druid/segment/{IndexMergerTest.java => IndexMakerTest.java} (87%) 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 cb15480da03..929a6565625 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -36,8 +36,9 @@ 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.AbstractProgressIndicator; 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.incremental.IncrementalIndex; @@ -99,7 +100,6 @@ public class IndexGeneratorJob implements Jobby public static List getPublishedSegments(HadoopDruidIndexerConfig config) { - final Configuration conf = new Configuration(); final ObjectMapper jsonMapper = HadoopDruidIndexerConfig.jsonMapper; @@ -297,7 +297,7 @@ public class IndexGeneratorJob implements Jobby for (final Text value : values) { context.progress(); - final InputRow inputRow = index.getSpatialDimensionRowFormatter().formatRow(parser.parse(value.toString())); + final InputRow inputRow = index.formatRow(parser.parse(value.toString())); allDimensionNames.addAll(inputRow.getDimensions()); int numRows = index.add(inputRow); @@ -316,8 +316,8 @@ public class IndexGeneratorJob implements Jobby toMerge.add(file); context.progress(); - IndexMerger.persist( - index, interval, file, new IndexMerger.ProgressIndicator() + IndexMaker.persist( + index, interval, file, new AbstractProgressIndicator() { @Override public void progress() @@ -345,8 +345,8 @@ public class IndexGeneratorJob implements Jobby } mergedBase = new File(baseFlushFile, "merged"); - IndexMerger.persist( - index, interval, mergedBase, new IndexMerger.ProgressIndicator() + IndexMaker.persist( + index, interval, mergedBase, new AbstractProgressIndicator() { @Override public void progress() @@ -358,8 +358,8 @@ public class IndexGeneratorJob implements Jobby } else { if (!index.isEmpty()) { final File finalFile = new File(baseFlushFile, "final"); - IndexMerger.persist( - index, interval, finalFile, new IndexMerger.ProgressIndicator() + IndexMaker.persist( + index, interval, finalFile, new AbstractProgressIndicator() { @Override public void progress() @@ -374,8 +374,8 @@ public class IndexGeneratorJob implements Jobby for (File file : toMerge) { indexes.add(IndexIO.loadIndex(file)); } - mergedBase = IndexMerger.mergeQueryableIndex( - indexes, aggs, new File(baseFlushFile, "merged"), new IndexMerger.ProgressIndicator() + mergedBase = IndexMaker.mergeQueryableIndex( + indexes, aggs, new File(baseFlushFile, "merged"), new AbstractProgressIndicator() { @Override public void progress() @@ -625,7 +625,7 @@ public class IndexGeneratorJob implements Jobby return new IncrementalIndex( new IncrementalIndexSchema.Builder() .withMinTimestamp(theBucket.time.getMillis()) - .withSpatialDimensions(config.getSchema().getDataSchema().getParser()) + .withDimensionsSpec(config.getSchema().getDataSchema().getParser()) .withQueryGranularity(config.getSchema().getDataSchema().getGranularitySpec().getQueryGranularity()) .withMetrics(aggs) .build(), 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 40807073874..02bfeac5576 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 d5995f9756f..00000000000 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/DeleteTask.java +++ /dev/null @@ -1,122 +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.ImmutableSet; -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.indexing.common.actions.SegmentInsertAction; -import io.druid.offheap.OffheapBufferPool; -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], - new OffheapBufferPool(0) - ); - try { - 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()); - } - finally { - empty.close(); - } - } -} 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 182a1569903..0e778a2f5b9 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 @@ -383,7 +383,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), + metrics + ); // rowFlushBoundary for this job final int myRowFlushBoundary = rowFlushBoundary > 0 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/MergeTaskBase.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java index 40b07f72d71..29749501227 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java @@ -41,7 +41,6 @@ import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; -import io.druid.indexing.common.actions.SegmentInsertAction; import io.druid.indexing.common.actions.SegmentListUsedAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.segment.IndexIO; 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 f9395165f27..0d6990224ba 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/test/java/io/druid/indexing/common/task/MergeTaskBaseTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/MergeTaskBaseTest.java index 7e8dd86f5a0..328818b8e19 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/MergeTaskBaseTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/MergeTaskBaseTest.java @@ -46,7 +46,7 @@ public class MergeTaskBaseTest final MergeTaskBase testMergeTaskBase = new MergeTaskBase(null, "foo", segments) { @Override - protected File merge(Map segments, File outDir) throws Exception + protected File merge( Map segments, File outDir) throws Exception { return null; } 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 00407381afc..7f855fc43c0 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 @@ -242,53 +242,6 @@ public class TaskSerdeTest ); } - @Test - public void testDeleteTaskSerde() throws Exception - { - final DeleteTask task = new DeleteTask( - null, - "foo", - new Interval("2010-01-01/P1D") - ); - - final ObjectMapper jsonMapper = new DefaultObjectMapper(); - 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 ObjectMapper jsonMapper = new DefaultObjectMapper(); - 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 { diff --git a/processing/src/main/java/io/druid/segment/AbstractProgressIndicator.java b/processing/src/main/java/io/druid/segment/AbstractProgressIndicator.java new file mode 100644 index 00000000000..567f989767f --- /dev/null +++ b/processing/src/main/java/io/druid/segment/AbstractProgressIndicator.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 abstract class AbstractProgressIndicator 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/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index 403789190ea..7653990fe4e 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -202,7 +202,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 ); @@ -689,7 +689,7 @@ public class IndexIO return new SimpleQueryableIndex( index.getDataInterval(), - new ArrayIndexed(cols, String.class), + new ArrayIndexed<>(cols, String.class), index.getAvailableDimensions(), new ColumnBuilder() .setType(ValueType.LONG) @@ -723,8 +723,6 @@ public class IndexIO Map columns = Maps.newHashMap(); - ObjectMapper mapper = new DefaultObjectMapper(); - for (String columnName : cols) { columns.put(columnName, deserializeColumn(mapper, smooshedFiles.mapFile(columnName))); } 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..dbc0be1a8da --- /dev/null +++ b/processing/src/main/java/io/druid/segment/IndexMaker.java @@ -0,0 +1,1564 @@ +/* + * 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.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.common.utils.JodaUtils; +import io.druid.common.utils.SerializerUtils; +import io.druid.jackson.DefaultObjectMapper; +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.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 java.io.ByteArrayOutputStream; +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.Collections; +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(","); + // This should really be provided by DI, should be changed once we switch around to using a DI framework + private static final ObjectMapper mapper = new DefaultObjectMapper(); + + + 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( + dimValueLookups, + Ordering.natural().nullsFirst() + ); + + 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 + ); + + 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++; + } + + GenericIndexed dictionary = null; + final Iterable dimensionValues = dimensionValuesLookup.get(dimension); + boolean bumpDictionary = false; + + if (hasMultipleValues) { + List> vals = ((MultiValColumnDictionaryEntryStore) adder).get(); + multiValCol = VSizeIndexed.fromIterable( + Iterables.transform( + vals, + new Function, VSizeIndexedInts>() + { + @Override + public VSizeIndexedInts apply(List input) + { + return VSizeIndexedInts.fromList( + input, + Collections.max(input) + ); + } + } + ) + ); + dictionary = GenericIndexed.fromIterable( + dimensionValues, + GenericIndexed.stringStrategy + ); + } 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 { + dictionary = GenericIndexed.fromIterable( + dimensionValues, + GenericIndexed.stringStrategy + ); + 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 (!hasMultipleValues) { + 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 + ); + } + } 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) { + 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++)); + } + } + if (hasSpatialIndexes) { + spatialIndex = ImmutableRTree.newImmutableFromMutable(tree); + } + + 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]; // TODO + arr[rowNum++] = (obj == null) ? 0 : ((Number) obj).floatValue(); + } + + CompressedFloatsIndexedSupplier compressedFloats = CompressedFloatsIndexedSupplier.fromFloatBuffer( + FloatBuffer.wrap(arr), + IndexIO.BYTE_ORDER + ); + + writeColumn( + v9Smoosher, + new FloatGenericColumnPartSerde(compressedFloats, 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]; // TODO + } + } + ), + 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 class DimValueConverter + { + private final Indexed dimSet; + private final IntBuffer conversionBuf; + + private int currIndex; + private String lastVal = null; + + DimValueConverter( + Indexed dimSet + ) + { + this.dimSet = dimSet; + conversionBuf = ByteBuffer.allocateDirect(dimSet.size() * Ints.BYTES).asIntBuffer(); + + 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(), + lhs.getMetrics(), + 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 interface ColumnDictionaryEntryStore + { + public void add(int[] vals); + } + + 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) + { + 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 deleted file mode 100644 index 672cec724b5..00000000000 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ /dev/null @@ -1,1182 +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.base.Function; -import com.google.common.base.Objects; -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.Closeables; -import com.google.common.io.Files; -import com.google.common.io.OutputSupplier; -import com.google.common.primitives.Ints; -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.Smoosh; -import com.metamx.common.logger.Logger; -import io.druid.collections.CombiningIterable; -import io.druid.common.guava.FileOutputSupplier; -import io.druid.common.guava.GuavaUtils; -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.data.ByteBufferWriter; -import io.druid.segment.data.CompressedLongsSupplierSerializer; -import io.druid.segment.data.ConciseCompressedIndexedInts; -import io.druid.segment.data.GenericIndexed; -import io.druid.segment.data.GenericIndexedWriter; -import io.druid.segment.data.IOPeon; -import io.druid.segment.data.Indexed; -import io.druid.segment.data.IndexedInts; -import io.druid.segment.data.IndexedIterable; -import io.druid.segment.data.IndexedRTree; -import io.druid.segment.data.TmpFileIOPeon; -import io.druid.segment.data.VSizeIndexedWriter; -import io.druid.segment.incremental.IncrementalIndex; -import io.druid.segment.incremental.IncrementalIndexAdapter; -import io.druid.segment.serde.ComplexMetricColumnSerializer; -import io.druid.segment.serde.ComplexMetricSerde; -import io.druid.segment.serde.ComplexMetrics; -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.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.IntBuffer; -import java.nio.MappedByteBuffer; -import java.nio.channels.FileChannel; -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 IndexMerger -{ - private static final Logger log = new Logger(IndexMerger.class); - - private static final SerializerUtils serializerUtils = new SerializerUtils(); - private static final int INVALID_ROW = -1; - private static final Splitter SPLITTER = Splitter.on(","); - - 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 NoopProgressIndicator()); - } - - 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 NoopProgressIndicator()); - } - - 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 indexes, final AggregatorFactory[] metricAggs, File outDir - ) throws IOException - { - return merge(indexes, metricAggs, outDir, new NoopProgressIndicator()); - } - - public static File merge( - List indexes, 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( - indexes, - new Function>() - { - @Override - public Iterable apply(@Nullable IndexableAdapter input) - { - return Iterables.transform( - input.getAvailableDimensions(), - new Function() - { - @Override - public String apply(@Nullable String input) - { - return input.toLowerCase(); - } - } - ); - } - } - ) - ); - final List mergedMetrics = Lists.transform( - mergeIndexed( - Lists.>newArrayList( - FunctionalIterable - .create(indexes) - .transform( - new Function>() - { - @Override - public Iterable apply(@Nullable IndexableAdapter input) - { - return Iterables.transform( - input.getAvailableMetrics(), - new Function() - { - @Override - public String apply(@Nullable String input) - { - return input.toLowerCase(); - } - } - ); - } - } - ) - .concat(Arrays.>asList(new AggFactoryStringIndexed(lowerCaseMetricAggs))) - ) - ), - new Function() - { - @Override - public String apply(@Nullable 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( - @Nullable ArrayList> boats - ) - { - return CombiningIterable.create( - new MergeIterable( - Ordering.natural().nullsFirst(), - boats - ), - Ordering.natural().nullsFirst(), - new RowboatMergeFunction(sortedMetricAggs) - ); - } - }; - - return makeIndexFiles(indexes, outDir, progress, mergedDimensions, mergedMetrics, rowMergerFn); - } - - public static File append( - List indexes, File outDir - ) throws IOException - { - return append(indexes, outDir, new NoopProgressIndicator()); - } - - public static File append( - List indexes, File outDir, ProgressIndicator progress - ) throws IOException - { - FileUtils.deleteDirectory(outDir); - if (!outDir.mkdirs()) { - throw new ISE("Couldn't make outdir[%s].", outDir); - } - -/* - if (indexes.size() < 2) { - throw new ISE("Too few indexes provided for append [%d].", indexes.size()); - } -*/ - - final List mergedDimensions = mergeIndexed( - Lists.transform( - indexes, - new Function>() - { - @Override - public Iterable apply(@Nullable IndexableAdapter input) - { - return Iterables.transform( - input.getAvailableDimensions(), - new Function() - { - @Override - public String apply(@Nullable String input) - { - return input.toLowerCase(); - } - } - ); - } - } - ) - ); - final List mergedMetrics = mergeIndexed( - Lists.transform( - indexes, - new Function>() - { - @Override - public Iterable apply(@Nullable IndexableAdapter input) - { - return Iterables.transform( - input.getAvailableMetrics(), - new Function() - { - @Override - public String apply(@Nullable String input) - { - return input.toLowerCase(); - } - } - ); - } - } - ) - ); - - Function>, Iterable> rowMergerFn = new Function>, Iterable>() - { - @Override - public Iterable apply( - @Nullable final ArrayList> boats - ) - { - return new MergeIterable( - Ordering.natural().nullsFirst(), - boats - ); - } - }; - - return makeIndexFiles(indexes, outDir, progress, mergedDimensions, mergedMetrics, rowMergerFn); - } - - private static File makeIndexFiles( - final List indexes, - final File outDir, - final ProgressIndicator progress, - final List mergedDimensions, - final List mergedMetrics, - final Function>, Iterable> rowMergerFn - ) throws IOException - { - Map metricTypes = Maps.newTreeMap(Ordering.natural().nullsFirst()); - for (IndexableAdapter adapter : indexes) { - for (String metric : adapter.getAvailableMetrics()) { - metricTypes.put(metric, adapter.getMetricType(metric)); - } - } - final Interval dataInterval; - File v8OutDir = new File(outDir, "v8-tmp"); - v8OutDir.mkdirs(); - - /************* Main index.drd file **************/ - progress.progress(); - long startTime = System.currentTimeMillis(); - File indexFile = new File(v8OutDir, "index.drd"); - - FileOutputStream fileOutputStream = null; - FileChannel channel = null; - try { - fileOutputStream = new FileOutputStream(indexFile); - channel = fileOutputStream.getChannel(); - channel.write(ByteBuffer.wrap(new byte[]{IndexIO.V8_VERSION})); - - GenericIndexed.fromIterable(mergedDimensions, GenericIndexed.stringStrategy).writeToChannel(channel); - GenericIndexed.fromIterable(mergedMetrics, GenericIndexed.stringStrategy).writeToChannel(channel); - - DateTime minTime = new DateTime(Long.MAX_VALUE); - DateTime maxTime = new DateTime(0l); - - for (IndexableAdapter index : indexes) { - minTime = JodaUtils.minDateTime(minTime, index.getDataInterval().getStart()); - maxTime = JodaUtils.maxDateTime(maxTime, index.getDataInterval().getEnd()); - } - - dataInterval = new Interval(minTime, maxTime); - serializerUtils.writeString(channel, String.format("%s/%s", minTime, maxTime)); - } - finally { - Closeables.closeQuietly(channel); - channel = null; - Closeables.closeQuietly(fileOutputStream); - fileOutputStream = null; - } - IndexIO.checkFileSize(indexFile); - log.info("outDir[%s] completed index.drd in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime); - - /************* Setup Dim Conversions **************/ - progress.progress(); - startTime = System.currentTimeMillis(); - - IOPeon ioPeon = new TmpFileIOPeon(); - ArrayList dimOuts = Lists.newArrayListWithCapacity(mergedDimensions.size()); - Map dimensionCardinalities = Maps.newHashMap(); - ArrayList> dimConversions = Lists.newArrayListWithCapacity(indexes.size()); - - for (IndexableAdapter index : indexes) { - dimConversions.add(Maps.newHashMap()); - } - - for (String dimension : mergedDimensions) { - final GenericIndexedWriter writer = new GenericIndexedWriter( - ioPeon, dimension, GenericIndexed.stringStrategy - ); - writer.open(); - - List> dimValueLookups = Lists.newArrayListWithCapacity(indexes.size()); - DimValueConverter[] converters = new DimValueConverter[indexes.size()]; - for (int i = 0; i < indexes.size(); i++) { - Indexed dimValues = indexes.get(i).getDimValueLookup(dimension); - if (dimValues != null) { - dimValueLookups.add(dimValues); - converters[i] = new DimValueConverter(dimValues); - } - } - - Iterable dimensionValues = CombiningIterable.createSplatted( - Iterables.transform( - dimValueLookups, - new Function, Iterable>() - { - @Override - public Iterable apply(@Nullable Indexed indexed) - { - return Iterables.transform( - indexed, - new Function() - { - @Override - public String apply(@Nullable String input) - { - return (input == null) ? "" : input; - } - } - ); - } - } - ) - , - Ordering.natural().nullsFirst() - ); - - int count = 0; - for (String value : dimensionValues) { - value = value == null ? "" : value; - writer.write(value); - - for (int i = 0; i < indexes.size(); i++) { - DimValueConverter converter = converters[i]; - if (converter != null) { - converter.convert(value, count); - } - } - - ++count; - } - dimensionCardinalities.put(dimension, count); - - FileOutputSupplier dimOut = new FileOutputSupplier(IndexIO.makeDimFile(v8OutDir, dimension), true); - dimOuts.add(dimOut); - - writer.close(); - serializerUtils.writeString(dimOut, dimension); - ByteStreams.copy(writer.combineStreams(), dimOut); - for (int i = 0; i < indexes.size(); ++i) { - DimValueConverter converter = converters[i]; - if (converter != null) { - dimConversions.get(i).put(dimension, converters[i].getConversionBuffer()); - } - } - - ioPeon.cleanup(); - } - log.info("outDir[%s] completed dim conversions in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime); - - /************* Walk through data sets and merge them *************/ - progress.progress(); - startTime = System.currentTimeMillis(); - - ArrayList> boats = Lists.newArrayListWithCapacity(indexes.size()); - - for (int i = 0; i < indexes.size(); ++i) { - final IndexableAdapter adapter = indexes.get(i); - - final int[] dimLookup = new int[mergedDimensions.size()]; - int count = 0; - for (String dim : adapter.getAvailableDimensions()) { - dimLookup[count] = mergedDimensions.indexOf(dim.toLowerCase()); - count++; - } - - final int[] metricLookup = new int[mergedMetrics.size()]; - count = 0; - for (String metric : adapter.getAvailableMetrics()) { - metricLookup[count] = mergedMetrics.indexOf(metric); - count++; - } - - boats.add( - new MMappedIndexRowIterable( - Iterables.transform( - indexes.get(i).getRows(), - new Function() - { - @Override - public Rowboat apply(@Nullable 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(), - input.getDescriptions() - ); - } - } - ) - , mergedDimensions, dimConversions.get(i), i - ) - ); - } - - Iterable theRows = rowMergerFn.apply(boats); - - CompressedLongsSupplierSerializer timeWriter = CompressedLongsSupplierSerializer.create( - ioPeon, "little_end_time", IndexIO.BYTE_ORDER - ); - - timeWriter.open(); - - ArrayList forwardDimWriters = Lists.newArrayListWithCapacity(mergedDimensions.size()); - for (String dimension : mergedDimensions) { - VSizeIndexedWriter writer = new VSizeIndexedWriter(ioPeon, dimension, dimensionCardinalities.get(dimension)); - writer.open(); - forwardDimWriters.add(writer); - } - - 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); - - if (serde == null) { - throw new ISE("Unknown type[%s]", typeName); - } - - metWriters.add(new ComplexMetricColumnSerializer(metric, v8OutDir, ioPeon, serde)); - } - } - for (MetricColumnSerializer metWriter : metWriters) { - metWriter.open(); - } - - int rowCount = 0; - long time = System.currentTimeMillis(); - List rowNumConversions = Lists.newArrayListWithCapacity(indexes.size()); - for (IndexableAdapter index : indexes) { - int[] arr = new int[index.getNumRows()]; - Arrays.fill(arr, INVALID_ROW); - rowNumConversions.add(IntBuffer.wrap(arr)); - } - - final Map descriptions = Maps.newHashMap(); - for (Rowboat theRow : theRows) { - progress.progress(); - timeWriter.add(theRow.getTimestamp()); - - final Object[] metrics = theRow.getMetrics(); - for (int i = 0; i < metrics.length; ++i) { - metWriters.get(i).serialize(metrics[i]); - } - - int[][] dims = theRow.getDims(); - for (int i = 0; i < dims.length; ++i) { - List listToWrite = (i >= dims.length || dims[i] == null) - ? null - : Ints.asList(dims[i]); - forwardDimWriters.get(i).write(listToWrite); - } - - 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) { - log.info( - "outDir[%s] walked 500,000/%,d rows in %,d millis.", v8OutDir, rowCount, System.currentTimeMillis() - time - ); - time = System.currentTimeMillis(); - } - - descriptions.putAll(theRow.getDescriptions()); - } - - for (IntBuffer rowNumConversion : rowNumConversions) { - rowNumConversion.rewind(); - } - - final File timeFile = IndexIO.makeTimeFile(v8OutDir, IndexIO.BYTE_ORDER); - timeFile.delete(); - OutputSupplier out = Files.newOutputStreamSupplier(timeFile, true); - timeWriter.closeAndConsolidate(out); - IndexIO.checkFileSize(timeFile); - - for (int i = 0; i < mergedDimensions.size(); ++i) { - forwardDimWriters.get(i).close(); - ByteStreams.copy(forwardDimWriters.get(i).combineStreams(), dimOuts.get(i)); - } - - for (MetricColumnSerializer metWriter : metWriters) { - metWriter.close(); - } - - ioPeon.cleanup(); - log.info( - "outDir[%s] completed walk through of %,d rows in %,d millis.", - v8OutDir, - rowCount, - System.currentTimeMillis() - startTime - ); - - /************ Create Inverted Indexes *************/ - startTime = System.currentTimeMillis(); - - final File invertedFile = new File(v8OutDir, "inverted.drd"); - Files.touch(invertedFile); - out = Files.newOutputStreamSupplier(invertedFile, true); - - final File geoFile = new File(v8OutDir, "spatial.drd"); - Files.touch(geoFile); - OutputSupplier spatialOut = Files.newOutputStreamSupplier(geoFile, true); - - for (int i = 0; i < mergedDimensions.size(); ++i) { - long dimStartTime = System.currentTimeMillis(); - String dimension = mergedDimensions.get(i); - - File dimOutFile = dimOuts.get(i).getFile(); - final MappedByteBuffer dimValsMapped = Files.map(dimOutFile); - - if (!dimension.equals(serializerUtils.readString(dimValsMapped))) { - throw new ISE("dimensions[%s] didn't equate!? This is a major WTF moment.", dimension); - } - Indexed dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.stringStrategy); - log.info("Starting dimension[%s] with cardinality[%,d]", dimension, dimVals.size()); - - GenericIndexedWriter writer = new GenericIndexedWriter( - ioPeon, dimension, ConciseCompressedIndexedInts.objectStrategy - ); - writer.open(); - - boolean isSpatialDim = "spatial".equals(descriptions.get(dimension)); - ByteBufferWriter spatialWriter = null; - RTree tree = null; - IOPeon spatialIoPeon = new TmpFileIOPeon(); - if (isSpatialDim) { - spatialWriter = new ByteBufferWriter( - spatialIoPeon, dimension, IndexedRTree.objectStrategy - ); - spatialWriter.open(); - tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50)); - } - - for (String dimVal : IndexedIterable.create(dimVals)) { - progress.progress(); - List> convertedInverteds = Lists.newArrayListWithCapacity(indexes.size()); - for (int j = 0; j < indexes.size(); ++j) { - convertedInverteds.add( - new ConvertingIndexedInts( - indexes.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); - } - } - - writer.write(ImmutableConciseSet.newImmutableFromMutable(bitset)); - - if (isSpatialDim && dimVal != null) { - 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, bitset); - } - } - writer.close(); - - serializerUtils.writeString(out, dimension); - ByteStreams.copy(writer.combineStreams(), out); - ioPeon.cleanup(); - - log.info("Completed dimension[%s] in %,d millis.", dimension, System.currentTimeMillis() - dimStartTime); - - if (isSpatialDim) { - spatialWriter.write(ImmutableRTree.newImmutableFromMutable(tree)); - spatialWriter.close(); - - serializerUtils.writeString(spatialOut, dimension); - ByteStreams.copy(spatialWriter.combineStreams(), spatialOut); - spatialIoPeon.cleanup(); - } - - } - - log.info("outDir[%s] completed inverted.drd in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime); - - final ArrayList expectedFiles = Lists.newArrayList( - Iterables.concat( - Arrays.asList( - "index.drd", "inverted.drd", "spatial.drd", String.format("time_%s.drd", IndexIO.BYTE_ORDER) - ), - Iterables.transform(mergedDimensions, GuavaUtils.formatFunction("dim_%s.drd")), - Iterables.transform( - mergedMetrics, GuavaUtils.formatFunction(String.format("met_%%s_%s.drd", IndexIO.BYTE_ORDER)) - ) - ) - ); - - Map files = Maps.newLinkedHashMap(); - for (String fileName : expectedFiles) { - files.put(fileName, new File(v8OutDir, fileName)); - } - - File smooshDir = new File(v8OutDir, "smoosher"); - smooshDir.mkdir(); - - for (Map.Entry entry : Smoosh.smoosh(v8OutDir, smooshDir, files).entrySet()) { - entry.getValue().delete(); - } - - for (File file : smooshDir.listFiles()) { - Files.move(file, new File(v8OutDir, file.getName())); - } - - if (!smooshDir.delete()) { - log.info("Unable to delete temporary dir[%s], contains[%s]", smooshDir, Arrays.asList(smooshDir.listFiles())); - throw new IOException(String.format("Unable to delete temporary dir[%s]", smooshDir)); - } - - createIndexDrdFile( - IndexIO.V8_VERSION, - v8OutDir, - GenericIndexed.fromIterable(mergedDimensions, GenericIndexed.stringStrategy), - GenericIndexed.fromIterable(mergedMetrics, GenericIndexed.stringStrategy), - dataInterval - ); - - IndexIO.DefaultIndexIOHandler.convertV8toV9(v8OutDir, outDir); - FileUtils.deleteDirectory(v8OutDir); - - return outDir; - } - - 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); - } - - public static void createIndexDrdFile( - byte versionId, - File inDir, - GenericIndexed availableDimensions, - GenericIndexed availableMetrics, - Interval dataInterval - ) throws IOException - { - File indexFile = new File(inDir, "index.drd"); - - FileChannel channel = null; - try { - channel = new FileOutputStream(indexFile).getChannel(); - channel.write(ByteBuffer.wrap(new byte[]{versionId})); - - availableDimensions.writeToChannel(channel); - availableMetrics.writeToChannel(channel); - serializerUtils.writeString( - channel, String.format("%s/%s", dataInterval.getStart(), dataInterval.getEnd()) - ); - } - finally { - Closeables.closeQuietly(channel); - channel = null; - } - IndexIO.checkFileSize(indexFile); - } - - 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; - conversionBuf = ByteBuffer.allocateDirect(dimSet.size() * Ints.BYTES).asIntBuffer(); - - 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(@Nullable 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() - { - int rowCount = 0; - - @Override - public Rowboat apply(@Nullable 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()) { - log.error("Converter mismatch! wtfbbq!"); - } - newDims[i][j] = converter.get(dims[i][j]); - } - } - - final Rowboat retVal = new Rowboat( - input.getTimestamp(), - newDims, - input.getMetrics(), - input.getRowNum(), - input.getDescriptions() - ); - - 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(), - lhs.getDescriptions() - ); - - 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; - } - } - - 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..4d977ba6b92 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java @@ -0,0 +1,100 @@ +/* + * 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 AbstractProgressIndicator +{ + 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 = new Stopwatch(); + } + + @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 = new Stopwatch(); + sections.put(section, sectionWatch); + sectionWatch.start(); + } + + @Override + public void progressSection(String section, String message) + { + Stopwatch sectionWatch = sections.remove(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/MMappedIndexAdapter.java b/processing/src/main/java/io/druid/segment/MMappedIndexAdapter.java deleted file mode 100644 index 0db39fb9261..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.google.common.io.Closeables; -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) { - Closeables.closeQuietly(timestamps); - for (IndexedFloats floatMetric : floatMetrics) { - Closeables.closeQuietly(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/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 efe09b51f55..96b327c05d5 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java @@ -27,6 +27,7 @@ import com.metamx.common.ISE; 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; @@ -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 ); @@ -174,18 +175,18 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter 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)); @@ -254,14 +255,8 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter } } - 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; @@ -311,4 +306,10 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter 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/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/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/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/VSizeIndexedInts.java b/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java index a2c106ff309..015fad23d69 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java @@ -57,7 +57,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 +138,8 @@ public class VSizeIndexedInts implements IndexedInts, Comparable ints) throws IOException - { - byte[] bytesToWrite = ints == null ? EMPTY_ARRAY : VSizeIndexedInts.fromList(ints, maxId).getBytesNoPadding(); - - valuesOut.write(bytesToWrite); - - headerOut.write(Ints.toByteArray((int) valuesOut.getCount())); - - ++numWritten; - } - - private String makeFilename(String suffix) - { - return String.format("%s.%s", filenameBase, suffix); - } - - @Override - public void close() throws IOException - { - final byte numBytesForMax = VSizeIndexedInts.getNumBytesForMax(maxId); - - valuesOut.write(new byte[4 - numBytesForMax]); - - Closeables.close(headerOut, false); - Closeables.close(valuesOut, false); - - final long numBytesWritten = headerOut.getCount() + valuesOut.getCount(); - - Preconditions.checkState( - headerOut.getCount() == (numWritten * 4), - "numWritten[%s] number of rows should have [%s] bytes written to headerOut, had[%s]", - numWritten, - numWritten * 4, - headerOut.getCount() - ); - Preconditions.checkState( - numBytesWritten < Integer.MAX_VALUE, "Wrote[%s] bytes, which is too many.", numBytesWritten - ); - - OutputStream metaOut = ioPeon.makeOutputStream(makeFilename("meta")); - - try { - metaOut.write(new byte[]{version, numBytesForMax}); - metaOut.write(Ints.toByteArray((int) numBytesWritten + 4)); - metaOut.write(Ints.toByteArray(numWritten)); - } - finally { - metaOut.close(); - } - } - - public InputSupplier combineStreams() - { - return ByteStreams.join( - Iterables.transform( - Arrays.asList("meta", "header", "values"), - new Function>() { - - @Override - public InputSupplier apply(final String input) - { - return new InputSupplier() - { - @Override - public InputStream getInput() throws IOException - { - return ioPeon.makeInputStream(makeFilename(input)); - } - }; - } - } - ) - ); - } -} 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 b821353a7fe..7835e0665d9 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -20,7 +20,6 @@ package io.druid.segment.incremental; import com.google.common.base.Function; -import com.google.common.base.Joiner; import com.google.common.base.Throwables; import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; @@ -29,11 +28,11 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.metamx.common.IAE; import com.metamx.common.ISE; -import com.metamx.common.logger.Logger; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidPool; import io.druid.data.input.InputRow; @@ -49,6 +48,9 @@ import io.druid.segment.DimensionSelector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.TimestampColumnSelector; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ColumnCapabilitiesImpl; +import io.druid.segment.column.ValueType; import io.druid.segment.data.IndexedInts; import io.druid.segment.serde.ComplexMetricExtractor; import io.druid.segment.serde.ComplexMetricSerde; @@ -76,10 +78,11 @@ import java.util.concurrent.atomic.AtomicInteger; */ 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 Set> rowTransformers; + private final AggregatorFactory[] metrics; private final Map metricIndexes; private final Map metricTypes; @@ -89,9 +92,10 @@ public class IncrementalIndex implements Iterable, Closeable private final int totalAggSize; private final LinkedHashMap dimensionOrder; private final CopyOnWriteArrayList dimensions; - private final List spatialDimensions; - private final SpatialDimensionRowFormatter spatialDimensionRowFormatter; private final DimensionHolder dimValues; + + private final Map columnCapabilities; + private final ConcurrentSkipListMap facts; private final ResourceHolder bufferHolder; private volatile AtomicInteger numEntries = new AtomicInteger(); @@ -116,6 +120,7 @@ public class IncrementalIndex implements Iterable, Closeable this.minTimestamp = incrementalIndexSchema.getMinTimestamp(); this.gran = incrementalIndexSchema.getGran(); this.metrics = incrementalIndexSchema.getMetrics(); + this.rowTransformers = Sets.newHashSet(); final ImmutableList.Builder metricNamesBuilder = ImmutableList.builder(); final ImmutableMap.Builder metricIndexesBuilder = ImmutableMap.builder(); @@ -280,17 +285,44 @@ public class IncrementalIndex implements Iterable, Closeable this.totalAggSize = currAggSize; this.dimensionOrder = Maps.newLinkedHashMap(); - this.dimensions = new CopyOnWriteArrayList(); + this.dimensions = new CopyOnWriteArrayList<>(); int index = 0; - for (String dim : incrementalIndexSchema.getDimensions()) { + for (String dim : incrementalIndexSchema.getDimensionsSpec().getDimensions()) { dimensionOrder.put(dim, index++); dimensions.add(dim); } - this.spatialDimensions = incrementalIndexSchema.getSpatialDimensions(); - this.spatialDimensionRowFormatter = new SpatialDimensionRowFormatter(spatialDimensions); + // 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 { + 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 = new ConcurrentSkipListMap<>(); } public IncrementalIndex( @@ -336,6 +368,18 @@ public class IncrementalIndex implements Iterable, Closeable ); } + 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. @@ -350,7 +394,7 @@ public class IncrementalIndex implements Iterable, Closeable */ 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)); } @@ -364,6 +408,18 @@ public class IncrementalIndex implements Iterable, Closeable 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()); @@ -466,16 +522,6 @@ public class IncrementalIndex implements Iterable, Closeable return dimensions; } - public List getSpatialDimensions() - { - return spatialDimensions; - } - - public SpatialDimensionRowFormatter getSpatialDimensionRowFormatter() - { - return spatialDimensionRowFormatter; - } - public String getMetricType(String metric) { return metricTypes.get(metric); @@ -541,6 +587,11 @@ public class IncrementalIndex implements Iterable, Closeable return aggs[metricIndex]; } + ColumnCapabilities getCapabilities(String column) + { + return columnCapabilities.get(column); + } + ConcurrentSkipListMap getFacts() { return facts; 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 bf44a8b25ac..0d7d10d2212 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java @@ -26,6 +26,8 @@ import com.metamx.common.logger.Logger; import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.segment.IndexableAdapter; import io.druid.segment.Rowboat; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ValueType; import io.druid.segment.data.EmptyIndexedInts; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; @@ -111,13 +113,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); } @@ -208,16 +210,11 @@ public class IncrementalIndexAdapter implements IndexableAdapter .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++ ); } } @@ -287,4 +284,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/SpatialDimensionRowFormatter.java b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java similarity index 96% 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 ca98d740dc3..d919fed289a 100644 --- a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java +++ b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java @@ -40,7 +40,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(","); @@ -49,7 +49,7 @@ public class SpatialDimensionRowFormatter private final Set spatialDimNames; private final Set spatialPartialDimNames; - public SpatialDimensionRowFormatter(List spatialDimensions) + public SpatialDimensionRowTransformer(List spatialDimensions) { this.spatialDimensions = spatialDimensions; this.spatialDimNames = Sets.newHashSet( @@ -82,7 +82,8 @@ public class SpatialDimensionRowFormatter ); } - public InputRow formatRow(final InputRow row) + @Override + public InputRow apply(final InputRow row) { final Map> spatialLookup = Maps.newHashMap(); 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 391bc62ffdb..aac7ee4fc1e 100644 --- a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -42,12 +42,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; private final VSizeIndexed multiValuedColumn; @@ -64,6 +59,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde ImmutableRTree spatialIndex ) { + this.isSingleValued = multiValCol == null; this.dictionary = dictionary; this.singleValuedColumn = singleValCol; this.multiValuedColumn = multiValCol; @@ -86,20 +82,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 @@ -111,14 +112,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/segment/EmptyIndexTest.java b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java index 04db1387a37..667004b792a 100644 --- a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java +++ b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java @@ -48,12 +48,16 @@ public class EmptyIndexTest 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()); } diff --git a/processing/src/test/java/io/druid/segment/IndexMergerTest.java b/processing/src/test/java/io/druid/segment/IndexMakerTest.java similarity index 87% rename from processing/src/test/java/io/druid/segment/IndexMergerTest.java rename to processing/src/test/java/io/druid/segment/IndexMakerTest.java index ace0ae7c33e..3b8b70bdee3 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMakerTest.java @@ -38,7 +38,7 @@ import java.util.Arrays; /** */ -public class IndexMergerTest +public class IndexMakerTest { @Test public void testPersistCaseInsensitive() throws Exception @@ -49,7 +49,7 @@ 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(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); @@ -88,20 +88,24 @@ 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(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(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()); @@ -141,10 +145,10 @@ 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()); @@ -155,7 +159,8 @@ public class IndexMergerTest Assert.assertEquals(1, merged.getTimeColumn().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/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index 14eb6c9db3c..7151e0acd8f 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -131,11 +131,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 @@ -229,7 +229,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/filter/SpatialFilterBonusTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java index 30650988b1c..c46a7aa2fa8 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; @@ -45,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; @@ -71,13 +72,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 @@ -106,11 +111,15 @@ 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(), @@ -221,7 +230,7 @@ public class SpatialFilterBonusTest tmpFile.mkdirs(); tmpFile.deleteOnExit(); - IndexMerger.persist(theIndex, tmpFile); + IndexMaker.persist(theIndex, tmpFile); return IndexIO.loadIndex(tmpFile); } @@ -232,13 +241,18 @@ 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(), TestQueryRunners.pool ); @@ -246,29 +260,40 @@ 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(), TestQueryRunners.pool + ); 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(), TestQueryRunners.pool + ); @@ -384,12 +409,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 @@ -403,13 +428,6 @@ public class SpatialFilterBonusTest } } - private final Segment segment; - - public SpatialFilterBonusTest(Segment segment) - { - this.segment = segment; - } - @Test public void testSpatialQuery() { 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 eff852a4300..6be894ca985 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.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; @@ -45,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; @@ -71,13 +72,17 @@ import java.util.Random; public class SpatialFilterTest { 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", "lat", "long"); + private final Segment segment; + + public SpatialFilterTest(Segment segment) + { + this.segment = segment; + } @Parameterized.Parameters public static Collection constructorFeeder() throws IOException @@ -106,11 +111,15 @@ 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") + .withDimensionsSpec( + new DimensionsSpec( + null, + null, + Arrays.asList( + new SpatialDimensionSchema( + "dim.geo", + Arrays.asList("lat", "long") + ) ) ) ).build(), @@ -236,7 +245,7 @@ public class SpatialFilterTest tmpFile.mkdirs(); tmpFile.deleteOnExit(); - IndexMerger.persist(theIndex, tmpFile); + IndexMaker.persist(theIndex, tmpFile); return IndexIO.loadIndex(tmpFile); } @@ -247,11 +256,15 @@ 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") + .withDimensionsSpec( + new DimensionsSpec( + null, + null, + Arrays.asList( + new SpatialDimensionSchema( + "dim.geo", + Arrays.asList("lat", "long") + ) ) ) ).build(), @@ -261,13 +274,18 @@ 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") + .withDimensionsSpec( + new DimensionsSpec( + null, + null, + Arrays.asList( + new SpatialDimensionSchema( + "dim.geo", + Arrays.asList("lat", "long") + ) ) ) + ).build(), TestQueryRunners.pool ); @@ -275,13 +293,18 @@ 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") + .withDimensionsSpec( + new DimensionsSpec( + null, + null, + Arrays.asList( + new SpatialDimensionSchema( + "dim.geo", + Arrays.asList("lat", "long") + ) ) ) + ).build(), TestQueryRunners.pool ); @@ -414,12 +437,12 @@ public class SpatialFilterTest 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 @@ -433,13 +456,6 @@ public class SpatialFilterTest } } - private final Segment segment; - - public SpatialFilterTest(Segment segment) - { - this.segment = segment; - } - @Test public void testSpatialQuery() { @@ -454,7 +470,7 @@ public class SpatialFilterTest ) ) .aggregators( - Arrays.asList( + Arrays.asList( new CountAggregatorFactory("rows"), new LongSumAggregatorFactory("val", "val") ) @@ -462,7 +478,7 @@ public class SpatialFilterTest .build(); List> expectedResults = Arrays.asList( - new Result( + new Result<>( new DateTime("2013-01-01T00:00:00.000Z"), new TimeseriesResultValue( ImmutableMap.builder() @@ -505,7 +521,7 @@ public class SpatialFilterTest ) ) .aggregators( - Arrays.asList( + Arrays.asList( new CountAggregatorFactory("rows"), new LongSumAggregatorFactory("val", "val") ) @@ -513,7 +529,7 @@ public class SpatialFilterTest .build(); List> expectedResults = Arrays.asList( - new Result( + new Result<>( new DateTime("2013-01-01T00:00:00.000Z"), new TimeseriesResultValue( ImmutableMap.builder() @@ -522,7 +538,7 @@ public class SpatialFilterTest .build() ) ), - new Result( + new Result<>( new DateTime("2013-01-02T00:00:00.000Z"), new TimeseriesResultValue( ImmutableMap.builder() @@ -531,7 +547,7 @@ public class SpatialFilterTest .build() ) ), - new Result( + new Result<>( new DateTime("2013-01-03T00:00:00.000Z"), new TimeseriesResultValue( ImmutableMap.builder() @@ -540,7 +556,7 @@ public class SpatialFilterTest .build() ) ), - new Result( + new Result<>( new DateTime("2013-01-04T00:00:00.000Z"), new TimeseriesResultValue( ImmutableMap.builder() @@ -549,7 +565,7 @@ public class SpatialFilterTest .build() ) ), - new Result( + new Result<>( new DateTime("2013-01-05T00:00:00.000Z"), new TimeseriesResultValue( ImmutableMap.builder() 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 dcff14024e3..81d51534d57 100644 --- a/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java +++ b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java @@ -100,8 +100,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/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 4a24d0a0e83..8d60ded0c54 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -33,7 +33,7 @@ import io.druid.query.SegmentDescriptor; import io.druid.query.spec.SpecificSegmentQueryRunner; import io.druid.query.spec.SpecificSegmentSpec; 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; @@ -349,7 +349,7 @@ public class RealtimePlumber implements Plumber indexes.add(queryableIndex); } - mergedFile = IndexMerger.mergeQueryableIndex( + mergedFile = IndexMaker.mergeQueryableIndex( indexes, schema.getAggregators(), mergedTarget @@ -714,7 +714,7 @@ public class RealtimePlumber implements Plumber try { int numRows = indexToPersist.getIndex().size(); - File persistedFile = IndexMerger.persist( + File persistedFile = IndexMaker.persist( indexToPersist.getIndex(), new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())) ); 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 57db2201d87..6a557b44822 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 @@ -27,7 +27,6 @@ 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.collections.StupidPool; import io.druid.data.input.InputRow; import io.druid.offheap.OffheapBufferPool; import io.druid.query.aggregation.AggregatorFactory; @@ -40,7 +39,6 @@ import io.druid.timeline.DataSegment; import org.joda.time.Interval; import javax.annotation.Nullable; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Iterator; import java.util.List; @@ -187,7 +185,7 @@ public class Sink implements Iterable new IncrementalIndexSchema.Builder() .withMinTimestamp(minTimestamp) .withQueryGranularity(schema.getGranularitySpec().getQueryGranularity()) - .withSpatialDimensions(schema.getParser()) + .withDimensionsSpec(schema.getParser()) .withMetrics(schema.getAggregators()) .build(), new OffheapBufferPool(bufferSize) From 32b9290723eb8577304a9b0ef5978ae48ab4a1a4 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Thu, 31 Jul 2014 16:20:58 +0530 Subject: [PATCH 027/107] fix compilation --- .../segment/LoggingProgressIndicator.java | 5 ++-- .../io/druid/segment/SchemalessIndex.java | 25 ++++++++++--------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java b/processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java index 4d977ba6b92..c40aedfbc69 100644 --- a/processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java +++ b/processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java @@ -41,7 +41,7 @@ public class LoggingProgressIndicator extends AbstractProgressIndicator public LoggingProgressIndicator(String progressName) { this.progressName = progressName; - this.global = new Stopwatch(); + this.global = Stopwatch.createUnstarted(); } @Override @@ -69,9 +69,8 @@ public class LoggingProgressIndicator extends AbstractProgressIndicator if (sectionWatch != null) { throw new ISE("[%s]: Cannot start progress tracker for [%s]. It is already started.", progressName, section); } - sectionWatch = new Stopwatch(); + sectionWatch = Stopwatch.createStarted(); sections.put(section, sectionWatch); - sectionWatch.start(); } @Override 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, From 6fa1dfe68f01b312e2413d48d383a19d3aef7ad7 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 31 Jul 2014 14:47:44 -0700 Subject: [PATCH 028/107] new docs on logging --- docs/content/Logging.md | 39 +++++++++++++++++++++++++++++++++++++++ docs/content/toc.textile | 4 ++-- 2 files changed, 41 insertions(+), 2 deletions(-) create mode 100644 docs/content/Logging.md diff --git a/docs/content/Logging.md b/docs/content/Logging.md new file mode 100644 index 00000000000..d949aa7858d --- /dev/null +++ b/docs/content/Logging.md @@ -0,0 +1,39 @@ +--- +layout: doc_page +--- +Logging +========================== + +Druid nodes will emit logs that are useful for debugging to the console. Druid nodes also emit periodic metrics about their state. For more about metrics, see [Configuration](Configuration.html). Metric logs are printed to the console by default, and can be disabled with `-Ddruid.emitter.logging.logLevel=debug`. + +Druid uses [log4j](http://logging.apache.org/log4j/2.x/) for logging, and console logs can be configured by adding a log4j.xml file. Add this xml file to your classpath if you want to override default Druid log configuration. + +An example log4j.xml file is shown below: + +``` + + + + + + + + + + + + + + + + + + + + + + + + + +``` \ No newline at end of file diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 5ca3d978fe4..d35114ae928 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -28,17 +28,17 @@ h2. Configuration * "Realtime":Realtime-Config.html h2. Data Ingestion +* "Ingestion FAQ":./Ingestion-FAQ.html * "Realtime":./Realtime-ingestion.html * "Batch":./Batch-ingestion.html * "Indexing Service":./Indexing-Service.html ** "Tasks":./Tasks.html * "Data Formats":./Data_formats.html -* "Ingestion FAQ":./Ingestion-FAQ.html h2. Operations +* "Performance FAQ":./Performance-FAQ.html * "Extending Druid":./Modules.html * "Booting a Production Cluster":./Booting-a-production-cluster.html -* "Performance FAQ":./Performance-FAQ.html h2. Querying * "Querying":./Querying.html From 93afc26d92f2b2d0e15510e6c9aaf865f398522c Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 31 Jul 2014 15:36:01 -0700 Subject: [PATCH 029/107] another log fix --- .../main/java/io/druid/server/coordinator/LoadQueuePeon.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java b/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java index e2577795a27..c7ee45ce492 100644 --- a/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java +++ b/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java @@ -208,7 +208,7 @@ public class LoadQueuePeon } } - log.info("Asking server peon[%s] to drop segment[%s]", basePath, segment); + log.info("Asking server peon[%s] to drop segment[%s]", basePath, segment.getIdentifier()); segmentsToDrop.add(holder); doNext(); } From a5bc6fc706bf3f372577b98d7f8deffa0bf61413 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 31 Jul 2014 17:08:08 -0700 Subject: [PATCH 030/107] fix to string error --- .../java/io/druid/query/timeboundary/TimeBoundaryQuery.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java index db9fa913bc9..5aae5d9fc67 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java @@ -199,7 +199,7 @@ public class TimeBoundaryQuery extends BaseQuery "dataSource='" + getDataSource() + '\'' + ", querySegmentSpec=" + getQuerySegmentSpec() + ", duration=" + getDuration() + - ", bound" + bound + + ", bound=" + bound + '}'; } From 8de945ee24d219b71694221256b46e78f05c1695 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 1 Aug 2014 13:24:12 -0700 Subject: [PATCH 031/107] rework the readme --- README | 7 ------- README.md | 20 ++++++++++++++++++++ 2 files changed, 20 insertions(+), 7 deletions(-) delete mode 100644 README create mode 100644 README.md diff --git a/README b/README deleted file mode 100644 index 2f3003db430..00000000000 --- a/README +++ /dev/null @@ -1,7 +0,0 @@ -What is Druid? http://www.druid.io - -Looking for docs? http://druid.io/docs/latest/ - -Want to get started? http://druid.io/docs/latest/Tutorial:-A-First-Look-at-Druid.html - -Questions about setting up Druid? https://groups.google.com/forum/#!forum/druid-development diff --git a/README.md b/README.md new file mode 100644 index 00000000000..625b6e638b8 --- /dev/null +++ b/README.md @@ -0,0 +1,20 @@ +## Druid + +Druid is a distributed, column-oriented, real-time analytics data store that is +commonly used to power exploratory dashboards in multi-tenant environments. +Druid excels as a data warehousing solution for fast aggregate queries on +petabyte sized data sets. Druid supports a variety of flexible filters, exact +calculations, approximate algorithms, and other tools commonly found in the +business intelligence space. Druid can load both streaming and batch data. + +### More Information +Much more information about Druid can be found on our [website](http://www.druid.io). + +### Documentation +We host documentation on our [website](http://druid.io/docs/latest/). If you want to contribute documentation changes, please submit a pull request to this repository. + +### Tutorials +We have a series of tutorials to get started with Druid, starting with this [one](http://druid.io/docs/latest/Tutorial:-A-First-Look-at-Druid.html). + +### Support +Contact us through our [forum](https://groups.google.com/forum/#!forum/druid-development) or on IRC in #druid-dev on irc.freenode.net. From 991e1828b05326eb2865929f3f0623108e79b0a5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 6 Aug 2014 15:50:04 -0700 Subject: [PATCH 032/107] make LZ4 the default compression strategy - LZ4 is now hardwired to be the default strategy - Rework tests to test all available compression strategies --- .../segment/FloatMetricColumnSerializer.java | 2 +- .../java/io/druid/segment/IndexMerger.java | 2 +- .../data/CompressedObjectStrategy.java | 2 + .../data/InMemoryCompressedFloats.java | 2 +- .../segment/data/InMemoryCompressedLongs.java | 2 +- .../CompressedFloatsIndexedSupplierTest.java | 16 ++++-- ...ompressedFloatsSupplierSerializerTest.java | 16 ++++-- .../CompressedLongsIndexedSupplierTest.java | 11 ++-- ...CompressedLongsSupplierSerializerTest.java | 16 ++++-- .../segment/data/CompressionStrategyTest.java | 52 +++++++++++++++++++ 10 files changed, 99 insertions(+), 22 deletions(-) create mode 100644 processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java diff --git a/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java index 520708fe015..2bc50c19d30 100644 --- a/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java @@ -53,7 +53,7 @@ public class FloatMetricColumnSerializer implements MetricColumnSerializer { writer = CompressedFloatsSupplierSerializer.create( ioPeon, String.format("%s_little", metricName), IndexIO.BYTE_ORDER, - CompressedObjectStrategy.CompressionStrategy.LZ4 // TODO define this somewhere else + CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY ); writer.open(); diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index 87cfa25a2ed..b817e3ea089 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -596,7 +596,7 @@ public class IndexMerger CompressedLongsSupplierSerializer timeWriter = CompressedLongsSupplierSerializer.create( ioPeon, "little_end_time", IndexIO.BYTE_ORDER, - CompressedObjectStrategy.CompressionStrategy.LZ4 // TODO define this somewhere else + CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY ); timeWriter.open(); 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 e0865e8ec80..36a3ad4876e 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java @@ -42,6 +42,8 @@ import java.util.Map; */ public class CompressedObjectStrategy implements ObjectStrategy> { + public static final CompressionStrategy DEFAULT_COMPRESSION_STRATEGY = CompressionStrategy.LZ4; + public static enum CompressionStrategy { LZF ((byte)0x0) { 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 e8ae40efbab..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,7 +37,7 @@ import java.util.List; */ public class InMemoryCompressedFloats implements IndexedFloats { - public static final CompressedObjectStrategy.CompressionStrategy COMPRESSION = CompressedObjectStrategy.CompressionStrategy.LZ4; + public static final CompressedObjectStrategy.CompressionStrategy COMPRESSION = CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY; private final CompressedFloatBufferObjectStrategy strategy; private final int sizePer; 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 9fd314569d5..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,7 +38,7 @@ import java.util.List; */ public class InMemoryCompressedLongs implements IndexedLongs { - public static final CompressedObjectStrategy.CompressionStrategy COMPRESSION = CompressedObjectStrategy.CompressionStrategy.LZ4; + public static final CompressedObjectStrategy.CompressionStrategy COMPRESSION = CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY; private final CompressedLongBufferObjectStrategy strategy; private final int sizePer; 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 e481ee8cef6..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; @@ -69,7 +75,7 @@ public class CompressedFloatsIndexedSupplierTest FloatBuffer.wrap(vals), 5, ByteOrder.nativeOrder(), - CompressedObjectStrategy.CompressionStrategy.LZ4 + compressionStrategy ); indexed = supplier.get(); @@ -83,7 +89,7 @@ public class CompressedFloatsIndexedSupplierTest ByteArrayOutputStream baos = new ByteArrayOutputStream(); final CompressedFloatsIndexedSupplier theSupplier = CompressedFloatsIndexedSupplier.fromFloatBuffer( - FloatBuffer.wrap(vals), 5, ByteOrder.nativeOrder(), CompressedObjectStrategy.CompressionStrategy.LZ4 + 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 3b888ae132f..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,10 +33,14 @@ 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 { @@ -47,11 +53,11 @@ public class CompressedFloatsSupplierSerializerTest "test", CompressedFloatBufferObjectStrategy.getBufferForOrder( order, - CompressedObjectStrategy.CompressionStrategy.LZ4, + compressionStrategy, sizePer ) ), - CompressedObjectStrategy.CompressionStrategy.LZ4 + 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 dd33d532498..fc29e284443 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java @@ -38,8 +38,13 @@ import java.util.concurrent.atomic.AtomicReference; /** */ -public class CompressedLongsIndexedSupplierTest +public class CompressedLongsIndexedSupplierTest extends CompressionStrategyTest { + public CompressedLongsIndexedSupplierTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy) + { + super(compressionStrategy); + } + private IndexedLongs indexed; private CompressedLongsIndexedSupplier supplier; private long[] vals; @@ -67,7 +72,7 @@ public class CompressedLongsIndexedSupplierTest LongBuffer.wrap(vals), 5, ByteOrder.nativeOrder(), - CompressedObjectStrategy.CompressionStrategy.LZ4 + compressionStrategy ); indexed = supplier.get(); @@ -79,7 +84,7 @@ public class CompressedLongsIndexedSupplierTest ByteArrayOutputStream baos = new ByteArrayOutputStream(); final CompressedLongsIndexedSupplier theSupplier = CompressedLongsIndexedSupplier.fromLongBuffer( - LongBuffer.wrap(vals), 5, ByteOrder.nativeOrder(), CompressedObjectStrategy.CompressionStrategy.LZ4 + 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 ee13c55087e..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,10 +33,14 @@ 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 { @@ -45,9 +51,9 @@ public class CompressedLongsSupplierSerializerTest new GenericIndexedWriter>( new IOPeonForTesting(), "test", - CompressedLongBufferObjectStrategy.getBufferForOrder(order, CompressedObjectStrategy.CompressionStrategy.LZ4, sizePer) + CompressedLongBufferObjectStrategy.getBufferForOrder(order, compressionStrategy, sizePer) ), - CompressedObjectStrategy.CompressionStrategy.LZ4 + 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; + } +} From 6fa611c2620d285934b5abf6feda832ccc1a3543 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 6 Aug 2014 16:13:39 -0700 Subject: [PATCH 033/107] refactor FixedSizeCompressedObjectStrategy --- .../data/CompressedFloatBufferObjectStrategy.java | 12 ++---------- .../data/CompressedLongBufferObjectStrategy.java | 12 ++---------- .../data/FixedSizeCompressedObjectStrategy.java | 10 ++++++++-- 3 files changed, 12 insertions(+), 22 deletions(-) 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 e6294462afe..b0680b2ec23 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedFloatBufferObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedFloatBufferObjectStrategy.java @@ -35,8 +35,6 @@ public class CompressedFloatBufferObjectStrategy extends FixedSizeCompressedObje return new CompressedFloatBufferObjectStrategy(order, compression, sizePer); } - private final int sizePer; - private CompressedFloatBufferObjectStrategy(final ByteOrder order, final CompressionStrategy compression, final int sizePer) { super( @@ -67,14 +65,8 @@ public class CompressedFloatBufferObjectStrategy extends FixedSizeCompressedObje return into.asFloatBuffer().put(from); } }, - compression + compression, + sizePer ); - this.sizePer = sizePer; - } - - @Override - public int getSize() - { - return sizePer; } } 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 b0b511278f1..13fd264eba3 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongBufferObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongBufferObjectStrategy.java @@ -35,8 +35,6 @@ public class CompressedLongBufferObjectStrategy extends FixedSizeCompressedObjec return new CompressedLongBufferObjectStrategy(order, compression, sizePer); } - private final int sizePer; - private CompressedLongBufferObjectStrategy(final ByteOrder order, final CompressionStrategy compression, final int sizePer) { super( @@ -67,14 +65,8 @@ public class CompressedLongBufferObjectStrategy extends FixedSizeCompressedObjec return into.asLongBuffer().put(from); } }, - compression + compression, + sizePer ); - this.sizePer = sizePer; - } - - @Override - public int getSize() - { - return sizePer; } } diff --git a/processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java index c79e0edc615..3efc1ba06ac 100644 --- a/processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java @@ -25,16 +25,22 @@ import java.nio.ByteOrder; public abstract class FixedSizeCompressedObjectStrategy extends CompressedObjectStrategy { + private final int sizePer; + protected FixedSizeCompressedObjectStrategy( ByteOrder order, BufferConverter converter, - CompressionStrategy compression + CompressionStrategy compression, + int sizePer ) { super(order, converter, compression); + this.sizePer = sizePer; } - public abstract int getSize(); + public int getSize() { + return sizePer; + } @Override protected ByteBuffer bufferFor(T val) From 263463dccc27d9ae2fb6ddc8475a98b2fc2d9752 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 8 Aug 2014 13:44:44 -0700 Subject: [PATCH 034/107] update pom~ --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 31bd84a4f63..9a47fd2d6e4 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.140-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.131-SNAPSHOT + druid-0.7.0-SNAPSHOT From c860de43cfeaa6500420e0739aa0c071c76f88a6 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 8 Aug 2014 13:46:25 -0700 Subject: [PATCH 035/107] fix dependencies --- 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 +- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 14 insertions(+), 14 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index e8ebb570a86..c8fec800e3e 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 0b57dcf484e..03e902df7c5 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 321e4a226e8..896b772738e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 1fe657ba77a..ac2c29b7405 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index 24e9937621a..a520de6725b 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 22fb248bf9b..f23382b6de9 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 09368b7c105..81260ba3295 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 7c09c218cc7..e0c5fb43ebe 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index cbd3a85f0e0..a5e6ef4b806 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index 312b9b408cd..8b04ea52763 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index fd953df211a..4a7866d1030 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 586529ab49f..bdd0d117b2c 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index d752f9a1811..0b9726a61b5 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index f0f3090e198..0c45dc6c753 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT From ddcea18631782b140e600d6a9afa55174dc5627a Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 8 Aug 2014 14:02:14 -0700 Subject: [PATCH 036/107] fix dependencies --- 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 +- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 14 insertions(+), 14 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index e8ebb570a86..c8fec800e3e 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 0b57dcf484e..03e902df7c5 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 321e4a226e8..896b772738e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 1fe657ba77a..ac2c29b7405 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index 24e9937621a..a520de6725b 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 22fb248bf9b..f23382b6de9 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 09368b7c105..81260ba3295 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 7c09c218cc7..e0c5fb43ebe 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index cbd3a85f0e0..a5e6ef4b806 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index 312b9b408cd..8b04ea52763 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index fd953df211a..4a7866d1030 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 586529ab49f..bdd0d117b2c 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index d752f9a1811..0b9726a61b5 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index f0f3090e198..0c45dc6c753 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.140-SNAPSHOT + 0.7.0-SNAPSHOT From d198959c606cdd0613760c407e914c269ccacb97 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 8 Aug 2014 14:05:10 -0700 Subject: [PATCH 037/107] fix test harness --- .../segment/data/CompressedLongsIndexedSupplierTest.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 fc29e284443..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,8 +38,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; -/** - */ +@RunWith(Parameterized.class) public class CompressedLongsIndexedSupplierTest extends CompressionStrategyTest { public CompressedLongsIndexedSupplierTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy) From d082718ed2aadf1872edfc8e2e6fbf84e1a2b874 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 11 Aug 2014 14:28:44 -0700 Subject: [PATCH 038/107] fix assembly file --- services/src/assembly/assembly.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/services/src/assembly/assembly.xml b/services/src/assembly/assembly.xml index 3dfb6a68f7e..6ab4cefa7e6 100644 --- a/services/src/assembly/assembly.xml +++ b/services/src/assembly/assembly.xml @@ -14,6 +14,13 @@ config + + ../examples/config/_global + + * + + config/_global + ../examples/config/broker From 9598a524a8bc59ddf772d73a2ccaee20437995ce Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Tue, 12 Aug 2014 14:58:55 +0530 Subject: [PATCH 039/107] review comment - move index closure to finally --- .../io/druid/indexer/IndexGeneratorJob.java | 172 +++++++++--------- 1 file changed, 88 insertions(+), 84 deletions(-) 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 29fe8e05728..6e0b4d7496b 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -283,84 +283,72 @@ 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.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(); + for (final Text value : values) { context.progress(); - IndexMaker.persist( - index, interval, file, new AbstractProgressIndicator() - { - @Override - public void progress() - { - context.progress(); - } - } - ); - index.close(); - 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"); - IndexMaker.persist( - index, interval, mergedBase, new AbstractProgressIndicator() - { - @Override - public void progress() - { context.progress(); + IndexMaker.persist( + index, interval, file, new AbstractProgressIndicator() + { + @Override + public void progress() + { + context.progress(); + } + } + ); + // 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"); + + 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"); IndexMaker.persist( - index, interval, finalFile, new AbstractProgressIndicator() + index, interval, mergedBase, new AbstractProgressIndicator() { @Override public void progress() @@ -369,29 +357,45 @@ public class IndexGeneratorJob implements Jobby } } ); - toMerge.add(finalFile); - } + } else { + if (!index.isEmpty()) { + final File finalFile = new File(baseFlushFile, "final"); + IndexMaker.persist( + index, interval, finalFile, new AbstractProgressIndicator() + { + @Override + public void progress() + { + context.progress(); + } + } + ); + toMerge.add(finalFile); + } + for (File file : toMerge) { + indexes.add(IndexIO.loadIndex(file)); + } + mergedBase = IndexMaker.mergeQueryableIndex( + indexes, aggs, new File(baseFlushFile, "merged"), new AbstractProgressIndicator() + { + @Override + public void progress() + { + context.progress(); + } + } + ); + } + serializeOutIndex(context, bucket, mergedBase, Lists.newArrayList(allDimensionNames)); for (File file : toMerge) { - indexes.add(IndexIO.loadIndex(file)); + FileUtils.deleteDirectory(file); } - mergedBase = IndexMaker.mergeQueryableIndex( - indexes, aggs, new File(baseFlushFile, "merged"), new AbstractProgressIndicator() - { - @Override - public void progress() - { - context.progress(); - } - } - ); } - index.close(); - serializeOutIndex(context, bucket, mergedBase, Lists.newArrayList(allDimensionNames)); + finally { + index.close(); + } - for (File file : toMerge) { - FileUtils.deleteDirectory(file); - } } private void serializeOutIndex(Context context, Bucket bucket, File mergedBase, List dimensionNames) From f36d030ef20e9cf810ae229fa673b61ee6f69271 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 13 Aug 2014 00:54:31 +0530 Subject: [PATCH 040/107] fix NPE while indexing --- .../java/io/druid/segment/incremental/IncrementalIndex.java | 5 ----- 1 file changed, 5 deletions(-) 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 7835e0665d9..26985edc024 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -286,11 +286,6 @@ public class IncrementalIndex implements Iterable, Closeable this.dimensionOrder = Maps.newLinkedHashMap(); this.dimensions = new CopyOnWriteArrayList<>(); - int index = 0; - for (String dim : incrementalIndexSchema.getDimensionsSpec().getDimensions()) { - dimensionOrder.put(dim, index++); - dimensions.add(dim); - } // This should really be more generic List spatialDimensions = incrementalIndexSchema.getDimensionsSpec().getSpatialDimensions(); if (!spatialDimensions.isEmpty()) { From b448deeca0ada385e5ae8af374dd0ce08e126ad0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 12 Aug 2014 14:02:08 -0700 Subject: [PATCH 041/107] fix compilation with Java 8 --- .../src/test/java/io/druid/query/RetryQueryRunnerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java index 16c8a51504a..9c6329a3df4 100644 --- a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java @@ -131,7 +131,7 @@ public class RetryQueryRunnerTest @Override public Sequence run(Query query, Map context) { - if (context.get("count") == 0) { + if ((int)context.get("count") == 0) { ((List) context.get(RetryQueryRunner.missingSegments)).add( new SegmentDescriptor( new Interval( @@ -368,4 +368,4 @@ public class RetryQueryRunnerTest Assert.assertTrue("Should have one entry in the list of missing segments", ((List) context.get(RetryQueryRunner.missingSegments)).size() == 1); } -} \ No newline at end of file +} From 1adec2312685df6b7d7d2ebf180aa988fc57e971 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 12 Aug 2014 16:40:02 -0700 Subject: [PATCH 042/107] some better null handling for MV --- .../java/io/druid/segment/IndexMaker.java | 45 ++++++++++++------- .../druid/segment/data/VSizeIndexedInts.java | 6 +++ 2 files changed, 34 insertions(+), 17 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/IndexMaker.java b/processing/src/main/java/io/druid/segment/IndexMaker.java index cbc6fcc89a8..f88a25bf573 100644 --- a/processing/src/main/java/io/druid/segment/IndexMaker.java +++ b/processing/src/main/java/io/druid/segment/IndexMaker.java @@ -23,6 +23,7 @@ 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.Predicates; import com.google.common.base.Splitter; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; @@ -79,6 +80,7 @@ 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.File; import java.io.IOException; @@ -843,20 +845,25 @@ public class IndexMaker if (hasMultipleValues) { List> vals = ((MultiValColumnDictionaryEntryStore) adder).get(); multiValCol = VSizeIndexed.fromIterable( - Iterables.transform( - vals, - new Function, VSizeIndexedInts>() - { - @Override - public VSizeIndexedInts apply(List input) - { - return VSizeIndexedInts.fromList( - input, - Collections.max(input) - ); - } - } - ) + FunctionalIterable + .create(vals) + //.filter(Predicates.>notNull()) + .transform( + new Function, VSizeIndexedInts>() + { + @Override + public VSizeIndexedInts apply(List input) + { + if (input == null) { + return VSizeIndexedInts.empty(); + } + return VSizeIndexedInts.fromList( + input, + Collections.max(input) + ); + } + } + ) ); dictionary = GenericIndexed.fromIterable( dimensionValues, @@ -1099,7 +1106,7 @@ public class IndexMaker float[] arr = new float[rowCount]; int rowNum = 0; for (Rowboat theRow : theRows) { - Object obj = theRow.getMetrics()[metricIndex]; // TODO + Object obj = theRow.getMetrics()[metricIndex]; arr[rowNum++] = (obj == null) ? 0 : ((Number) obj).floatValue(); } @@ -1133,7 +1140,7 @@ public class IndexMaker @Override public Object apply(Rowboat input) { - return input.getMetrics()[metricIndex]; // TODO + return input.getMetrics()[metricIndex]; } } ), @@ -1557,7 +1564,11 @@ public class IndexMaker public void add(int[] vals) { - data.add(Ints.asList(vals)); + if (vals == null || vals.length == 0) { + data.add(null); + } else { + data.add(Ints.asList(vals)); + } } public List> get() 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 015fad23d69..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); From 1b0a72751ba4d3c9ce79630bbf5e15ee62164133 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 13 Aug 2014 08:52:36 +0530 Subject: [PATCH 043/107] Add support for LongColumn --- .../ApproximateHistogramBufferAggregator.java | 7 ++ ...imateHistogramFoldingBufferAggregator.java | 6 ++ .../IngestSegmentFirehoseFactory.java | 6 +- pom.xml | 2 +- .../druid/query/aggregation/Aggregators.java | 7 ++ .../query/aggregation/BufferAggregator.java | 17 ++++ .../aggregation/CountBufferAggregator.java | 7 ++ .../DoubleSumBufferAggregator.java | 7 ++ .../HistogramBufferAggregator.java | 6 ++ .../JavaScriptBufferAggregator.java | 7 ++ .../query/aggregation/LongSumAggregator.java | 7 +- .../aggregation/LongSumAggregatorFactory.java | 6 +- .../aggregation/LongSumBufferAggregator.java | 14 +++- .../aggregation/MaxBufferAggregator.java | 7 ++ .../aggregation/MinBufferAggregator.java | 7 ++ .../CardinalityBufferAggregator.java | 7 ++ .../HyperUniquesBufferAggregator.java | 7 ++ .../druid/query/select/SelectQueryEngine.java | 6 +- .../druid/segment/ColumnSelectorFactory.java | 3 +- .../java/io/druid/segment/IndexMaker.java | 41 ++++++--- ...nSelector.java => LongColumnSelector.java} | 4 +- .../QueryableIndexIndexableAdapter.java | 12 ++- .../segment/QueryableIndexStorageAdapter.java | 84 +++++++++++++++++-- .../segment/incremental/IncrementalIndex.java | 24 +++++- .../IncrementalIndexStorageAdapter.java | 40 ++++++++- .../SpatialDimensionRowTransformer.java | 11 +++ .../aggregation/LongSumAggregatorTest.java | 6 +- .../aggregation/TestLongColumnSelector.java | 48 +++++++++++ .../java/io/druid/segment/AppendTest.java | 1 + .../firehose/WikipediaIrcDecoder.java | 6 ++ .../CombiningFirehoseFactoryTest.java | 6 ++ .../segment/realtime/RealtimeManagerTest.java | 6 ++ .../segment/realtime/plumber/SinkTest.java | 12 +++ .../shard/HashBasedNumberedShardSpecTest.java | 6 ++ 34 files changed, 394 insertions(+), 49 deletions(-) rename processing/src/main/java/io/druid/segment/{TimestampColumnSelector.java => LongColumnSelector.java} (92%) create mode 100644 processing/src/test/java/io/druid/query/aggregation/TestLongColumnSelector.java 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/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index 6987a24e97f..dfb6f4c2a2d 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,10 @@ 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.data.IndexedInts; import io.druid.segment.filter.Filters; import io.druid.segment.loading.SegmentLoadingException; @@ -250,7 +250,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory apply(@Nullable final Cursor cursor) { - final TimestampColumnSelector timestampColumnSelector = cursor.makeTimestampColumnSelector(); + final LongColumnSelector timestampColumnSelector = cursor.makeTimestampColumnSelector(); final Map dimSelectors = Maps.newHashMap(); for (String dim : dims) { @@ -282,7 +282,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/pom.xml b/pom.xml index ce074209e92..30552336acb 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.6 2.6.0 - 0.2.7 + 0.2.8-SNAPSHOT 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/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 963d9458beb..acf16e9e7af 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java @@ -57,14 +57,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 @@ -133,7 +133,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..d5dbf6a1598 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,13 @@ 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..d7ad98c85bc 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 @@ -82,6 +82,13 @@ public class CardinalityBufferAggregator implements BufferAggregator throw new UnsupportedOperationException(); } + + @Override + public long getLong(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("CardinalityBufferAggregator does not support getLong()"); + } + @Override public void close() { 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..22968bd68fc 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 @@ -82,6 +82,13 @@ public class HyperUniquesBufferAggregator implements BufferAggregator throw new UnsupportedOperationException(); } + + @Override + public long getLong(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("HyperUniquesBufferAggregator does not support getLong()"); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java b/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java index 432d311b453..9e51ca2bd46 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,10 @@ 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.data.IndexedInts; import io.druid.segment.filter.Filters; import org.joda.time.DateTime; @@ -83,7 +83,7 @@ public class SelectQueryEngine .getThreshold() ); - final TimestampColumnSelector timestampColumnSelector = cursor.makeTimestampColumnSelector(); + final LongColumnSelector timestampColumnSelector = cursor.makeTimestampColumnSelector(); final Map dimSelectors = Maps.newHashMap(); for (String dim : dims) { @@ -110,7 +110,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/segment/ColumnSelectorFactory.java b/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java index 318f93af9a3..c7c6a54289e 100644 --- a/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java +++ b/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java @@ -24,8 +24,9 @@ package io.druid.segment; */ public interface ColumnSelectorFactory { - public TimestampColumnSelector makeTimestampColumnSelector(); + public LongColumnSelector 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/IndexMaker.java b/processing/src/main/java/io/druid/segment/IndexMaker.java index cbc6fcc89a8..e5717231ec2 100644 --- a/processing/src/main/java/io/druid/segment/IndexMaker.java +++ b/processing/src/main/java/io/druid/segment/IndexMaker.java @@ -101,14 +101,12 @@ 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(","); // This should really be provided by DI, should be changed once we switch around to using a DI framework private static final ObjectMapper mapper = new DefaultObjectMapper(); - public static File persist(final IncrementalIndex index, File outDir) throws IOException { return persist(index, index.getInterval(), outDir); @@ -777,7 +775,6 @@ public class IndexMaker progress.stopSection(dimSection); } - private static void makeDimColumn( final FileSmoosher v9Smoosher, final List adapters, @@ -1093,7 +1090,7 @@ public class IndexMaker ValueType type = valueTypes.get(metric); switch (type) { - case FLOAT: + case FLOAT: { metBuilder.setValueType(ValueType.FLOAT); float[] arr = new float[rowCount]; @@ -1116,6 +1113,31 @@ public class IndexMaker 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); @@ -1248,11 +1270,15 @@ public class IndexMaker 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; @@ -1526,11 +1552,6 @@ public class IndexMaker } } - private static interface ColumnDictionaryEntryStore - { - public void add(int[] vals); - } - private static class SingleValColumnDictionaryEntryStore implements ColumnDictionaryEntryStore { private final List data = Lists.newArrayList(); 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/QueryableIndexIndexableAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java index 87ea58f6414..7e082eea805 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java @@ -31,6 +31,8 @@ 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; @@ -54,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) @@ -173,6 +173,7 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter { final GenericColumn timestamps = input.getTimeColumn().getGenericColumn(); final Object[] metrics; + final Map dimensions; final int numMetrics = getMetricNames().size(); @@ -193,6 +194,7 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter final ValueType type = column.getCapabilities().getType(); switch (type) { case FLOAT: + case LONG: metrics[i] = column.getGenericColumn(); break; case COMPLEX: @@ -248,8 +250,10 @@ 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); } @@ -300,6 +304,8 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter switch (type) { case FLOAT: return "float"; + case LONG: + return "long"; case COMPLEX: return column.getComplexColumn().getTypeName(); default: diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index 9dcd136e024..84bef1b33a4 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -256,12 +256,12 @@ public class QueryableIndexStorageAdapter implements StorageAdapter } @Override - public TimestampColumnSelector makeTimestampColumnSelector() + public LongColumnSelector makeTimestampColumnSelector() { - return new TimestampColumnSelector() + return new LongColumnSelector() { @Override - public long getTimestamp() + public long get() { return timestamps.getLongSingleValueRow(cursorOffset.getOffset()); } @@ -399,6 +399,42 @@ 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) { + 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) { @@ -711,12 +747,12 @@ public class QueryableIndexStorageAdapter implements StorageAdapter } @Override - public TimestampColumnSelector makeTimestampColumnSelector() + public LongColumnSelector makeTimestampColumnSelector() { - return new TimestampColumnSelector() + return new LongColumnSelector() { @Override - public long getTimestamp() + public long get() { return timestamps.getLongSingleValueRow(currRow); } @@ -854,6 +890,42 @@ 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) { + 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/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 7835e0665d9..b2f05b51235 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -46,8 +46,8 @@ import io.druid.query.aggregation.PostAggregator; 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.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ValueType; @@ -134,18 +134,32 @@ public class IncrementalIndex implements Iterable, Closeable new ColumnSelectorFactory() { @Override - public TimestampColumnSelector makeTimestampColumnSelector() + public LongColumnSelector makeTimestampColumnSelector() { - return new TimestampColumnSelector() + return new LongColumnSelector() { @Override - public long getTimestamp() + public long get() { return in.get().getTimestampFromEpoch(); } }; } + @Override + public LongColumnSelector makeLongColumnSelector(String columnName) + { + final String metricName = columnName.toLowerCase(); + return new LongColumnSelector() + { + @Override + public long get() + { + return in.get().getLongMetric(metricName); + } + }; + } + @Override public FloatColumnSelector makeFloatColumnSelector(String columnName) { @@ -302,6 +316,8 @@ public class IncrementalIndex implements Iterable, Closeable ValueType type; if (entry.getValue().equalsIgnoreCase("float")) { type = ValueType.FLOAT; + } else if (entry.getValue().equalsIgnoreCase("long")) { + type = ValueType.LONG; } else { type = ValueType.COMPLEX; } 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 d02a8c361b4..142d634b628 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -38,9 +38,9 @@ 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.data.Indexed; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.ListIndexed; @@ -262,12 +262,12 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter } @Override - public TimestampColumnSelector makeTimestampColumnSelector() + public LongColumnSelector makeTimestampColumnSelector() { - return new TimestampColumnSelector() + return new LongColumnSelector() { @Override - public long getTimestamp() + public long get() { return currEntry.getKey().getTimestamp(); } @@ -378,6 +378,38 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter }; } + @Override + public LongColumnSelector makeLongColumnSelector(String columnName) + { + final String metricName = columnName.toLowerCase(); + 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) + ); + } + }; + } + @Override public ObjectColumnSelector makeObjectColumnSelector(String column) { diff --git a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java index f100a29a87e..4f873e1aa4b 100644 --- a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java +++ b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java @@ -161,6 +161,17 @@ public class SpatialDimensionRowTransformer implements Function Date: Wed, 13 Aug 2014 10:55:20 +0530 Subject: [PATCH 044/107] make tests pass --- .../java/io/druid/segment/QueryableIndexStorageAdapter.java | 4 ++-- .../io/druid/segment/column/IndexedFloatsGenericColumn.java | 2 +- .../io/druid/segment/column/IndexedLongsGenericColumn.java | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index 84bef1b33a4..6e3386e36af 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -407,7 +407,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter if (cachedMetricVals == null) { Column holder = index.getColumn(metricName); - if (holder != null && holder.getCapabilities().getType() == ValueType.LONG) { + if (holder != null && (holder.getCapabilities().getType() == ValueType.LONG || holder.getCapabilities().getType() == ValueType.FLOAT)) { cachedMetricVals = holder.getGenericColumn(); genericColumnCache.put(metricName, cachedMetricVals); } @@ -898,7 +898,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter if (cachedMetricVals == null) { Column holder = index.getColumn(metricName); - if (holder != null && holder.getCapabilities().getType() == ValueType.LONG) { + if (holder != null && (holder.getCapabilities().getType() == ValueType.LONG || holder.getCapabilities().getType() == ValueType.FLOAT)) { cachedMetricVals = holder.getGenericColumn(); genericColumnCache.put(metricName, cachedMetricVals); } 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 From 6f60a3f60437ddfff2065110d8622ef218fd4b65 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 13 Aug 2014 14:37:57 +0530 Subject: [PATCH 045/107] make valueType configurable --- .../indexing/common/task/TaskSerdeTest.java | 3 +- .../aggregation/CountAggregatorFactory.java | 16 ++++-- .../aggregation/LongSumAggregatorFactory.java | 12 +++-- .../ChainedExecutionQueryRunnerTest.java | 4 +- .../java/io/druid/query/DataSourceTest.java | 2 +- .../test/java/io/druid/query/QueriesTest.java | 8 +-- .../io/druid/query/QueryRunnerTestHelper.java | 4 +- .../io/druid/query/RetryQueryRunnerTest.java | 3 +- .../aggregation/CountAggregatorTest.java | 2 +- .../aggregation/LongSumAggregatorTest.java | 2 +- .../query/groupby/GroupByQueryRunnerTest.java | 49 ++++++++++--------- .../druid/query/groupby/GroupByQueryTest.java | 4 +- .../TimeSeriesUnionQueryRunnerTest.java | 3 +- .../timeseries/TimeseriesBinaryFnTest.java | 4 +- .../TimeseriesQueryRunnerBonusTest.java | 2 +- .../timeseries/TimeseriesQueryRunnerTest.java | 24 ++++++--- .../query/topn/TopNBinaryFnBenchmark.java | 6 +-- .../io/druid/query/topn/TopNBinaryFnTest.java | 4 +- .../java/io/druid/segment/AppendTest.java | 6 +-- .../io/druid/segment/SchemalessIndex.java | 4 +- .../io/druid/segment/SchemalessTestFull.java | 2 +- .../druid/segment/SchemalessTestSimple.java | 2 +- .../segment/data/IncrementalIndexTest.java | 2 +- .../filter/SpatialFilterBonusTest.java | 13 +++-- .../segment/filter/SpatialFilterTest.java | 13 +++-- .../IncrementalIndexStorageAdapterTest.java | 16 +++--- .../antlr4/io/druid/sql/antlr4/DruidSQL.g4 | 2 +- .../client/CachingClusteredClientTest.java | 12 ++--- .../druid/client/CachingQueryRunnerTest.java | 6 +-- .../segment/realtime/FireDepartmentTest.java | 2 +- .../segment/realtime/RealtimeManagerTest.java | 2 +- .../plumber/RealtimePlumberSchoolTest.java | 3 +- .../segment/realtime/plumber/SinkTest.java | 2 +- .../router/TieredBrokerHostSelectorTest.java | 14 +++--- 34 files changed, 135 insertions(+), 118 deletions(-) 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 daa35c792fb..94c76d3c339 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 @@ -19,7 +19,6 @@ package io.druid.indexing.common.task; -import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; @@ -104,7 +103,7 @@ public class TaskSerdeTest DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build() ), ImmutableList.of( - new CountAggregatorFactory("cnt") + new CountAggregatorFactory("cnt", null) ) ); diff --git a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java index 9549015f687..83baa4fd41b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java @@ -35,16 +35,22 @@ import java.util.List; public class CountAggregatorFactory implements AggregatorFactory { private static final byte[] CACHE_KEY = new byte[]{0x0}; + private static final String DEFAULT_VALUE_TYPE = "long"; + private static final List supportedTypes = Arrays.asList("float", "long"); private final String name; + private final String valueType; + @JsonCreator public CountAggregatorFactory( - @JsonProperty("name") String name + @JsonProperty("name") String name, + @JsonProperty("valueType") String valueType ) { Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); - this.name = name; + this.valueType = valueType == null ? DEFAULT_VALUE_TYPE : valueType; + Preconditions.checkArgument(supportedTypes.contains(this.valueType)); } @Override @@ -74,13 +80,13 @@ public class CountAggregatorFactory implements AggregatorFactory @Override public AggregatorFactory getCombiningFactory() { - return new LongSumAggregatorFactory(name, name); + return new LongSumAggregatorFactory(name, name, valueType); } @Override public List getRequiredColumns() { - return Arrays.asList(new CountAggregatorFactory(name)); + return Arrays.asList(new CountAggregatorFactory(name, valueType)); } @Override @@ -117,7 +123,7 @@ public class CountAggregatorFactory implements AggregatorFactory @Override public String getTypeName() { - return "float"; + return "long"; } @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 acf16e9e7af..bbac473f5de 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java @@ -35,14 +35,18 @@ import java.util.List; public class LongSumAggregatorFactory implements AggregatorFactory { private static final byte CACHE_TYPE_ID = 0x1; + private static final String DEFAULT_VALUE_TYPE = "long"; + private static final List supportedTypes = Arrays.asList("float", "long"); private final String fieldName; private final String name; + private final String valueType; @JsonCreator public LongSumAggregatorFactory( @JsonProperty("name") String name, - @JsonProperty("fieldName") final String fieldName + @JsonProperty("fieldName") final String fieldName, + @JsonProperty("valueType") String valueType ) { Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); @@ -50,6 +54,8 @@ public class LongSumAggregatorFactory implements AggregatorFactory this.name = name; this.fieldName = fieldName; + this.valueType = valueType == null ? DEFAULT_VALUE_TYPE : valueType; + Preconditions.checkArgument(supportedTypes.contains(this.valueType)); } @Override @@ -82,13 +88,13 @@ public class LongSumAggregatorFactory implements AggregatorFactory @Override public AggregatorFactory getCombiningFactory() { - return new LongSumAggregatorFactory(name, name); + return new LongSumAggregatorFactory(name, name, valueType); } @Override public List getRequiredColumns() { - return Arrays.asList(new LongSumAggregatorFactory(fieldName, fieldName)); + return Arrays.asList(new LongSumAggregatorFactory(fieldName, fieldName, valueType)); } @Override diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index 489c36e799f..8a704e635a9 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -107,7 +107,7 @@ public class ChainedExecutionQueryRunnerTest Druids.newTimeseriesQueryBuilder() .dataSource("test") .intervals("2014/2015") - .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) + .aggregators(Lists.newArrayList(new CountAggregatorFactory("count", null))) .build(), context ); @@ -215,7 +215,7 @@ public class ChainedExecutionQueryRunnerTest Druids.newTimeseriesQueryBuilder() .dataSource("test") .intervals("2014/2015") - .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) + .aggregators(Lists.newArrayList(new CountAggregatorFactory("count", null))) .context(ImmutableMap.of("timeout", 100, "queryId", "test")) .build(), context diff --git a/processing/src/test/java/io/druid/query/DataSourceTest.java b/processing/src/test/java/io/druid/query/DataSourceTest.java index 368ece5634c..1c48ff6231f 100644 --- a/processing/src/test/java/io/druid/query/DataSourceTest.java +++ b/processing/src/test/java/io/druid/query/DataSourceTest.java @@ -73,7 +73,7 @@ public class DataSourceTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index") + new LongSumAggregatorFactory("idx", "index", null) ) ) .setGranularity(QueryRunnerTestHelper.dayGran) diff --git a/processing/src/test/java/io/druid/query/QueriesTest.java b/processing/src/test/java/io/druid/query/QueriesTest.java index 763aeb135fd..d212f660529 100644 --- a/processing/src/test/java/io/druid/query/QueriesTest.java +++ b/processing/src/test/java/io/druid/query/QueriesTest.java @@ -40,7 +40,7 @@ public class QueriesTest public void testVerifyAggregations() throws Exception { List aggFactories = Arrays.asList( - new CountAggregatorFactory("count"), + new CountAggregatorFactory("count", null), new DoubleSumAggregatorFactory("idx", "index"), new DoubleSumAggregatorFactory("rev", "revenue") ); @@ -72,7 +72,7 @@ public class QueriesTest public void testVerifyAggregationsMissingVal() throws Exception { List aggFactories = Arrays.asList( - new CountAggregatorFactory("count"), + new CountAggregatorFactory("count", null), new DoubleSumAggregatorFactory("idx", "index"), new DoubleSumAggregatorFactory("rev", "revenue") ); @@ -104,7 +104,7 @@ public class QueriesTest public void testVerifyAggregationsMultiLevel() throws Exception { List aggFactories = Arrays.asList( - new CountAggregatorFactory("count"), + new CountAggregatorFactory("count", null), new DoubleSumAggregatorFactory("idx", "index"), new DoubleSumAggregatorFactory("rev", "revenue") ); @@ -158,7 +158,7 @@ public class QueriesTest public void testVerifyAggregationsMultiLevelMissingVal() throws Exception { List aggFactories = Arrays.asList( - new CountAggregatorFactory("count"), + new CountAggregatorFactory("count", null), new DoubleSumAggregatorFactory("idx", "index"), new DoubleSumAggregatorFactory("rev", "revenue") ); diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index deeaff563e8..5592a299b4a 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -90,8 +90,8 @@ public class QueryRunnerTestHelper public static final String uniqueMetric = "uniques"; public static final String addRowsIndexConstantMetric = "addRowsIndexConstant"; public static String dependentPostAggMetric = "dependentPostAgg"; - public static final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); - public static final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index"); + public static final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows", null); + public static final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index", null); public static final DoubleSumAggregatorFactory indexDoubleSum = new DoubleSumAggregatorFactory("index", "index"); public static final JavaScriptAggregatorFactory jsIndexSumIfPlacementishA = new JavaScriptAggregatorFactory( "nindex", diff --git a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java index 16c8a51504a..76e31558674 100644 --- a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java @@ -36,7 +36,8 @@ public class RetryQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index" + "index", + null ), QueryRunnerTestHelper.qualityUniques ) diff --git a/processing/src/test/java/io/druid/query/aggregation/CountAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/CountAggregatorTest.java index 2ca8584c939..24fe633f67c 100644 --- a/processing/src/test/java/io/druid/query/aggregation/CountAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/CountAggregatorTest.java @@ -56,7 +56,7 @@ public class CountAggregatorTest Object first = agg.get(); agg.aggregate(); - Comparator comp = new CountAggregatorFactory("null").getComparator(); + Comparator comp = new CountAggregatorFactory("null", null).getComparator(); Assert.assertEquals(-1, comp.compare(first, agg.get())); Assert.assertEquals(0, comp.compare(first, first)); 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 046e836248b..0e57cce691f 100644 --- a/processing/src/test/java/io/druid/query/aggregation/LongSumAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/LongSumAggregatorTest.java @@ -66,7 +66,7 @@ public class LongSumAggregatorTest Object first = agg.get(); agg.aggregate(); - Comparator comp = new LongSumAggregatorFactory("null", "null").getComparator(); + Comparator comp = new LongSumAggregatorFactory("null", "null", null).getComparator(); Assert.assertEquals(-1, comp.compare(first, agg.get())); Assert.assertEquals(0, comp.compare(first, first)); 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 4d7f285d13f..0009173d5b8 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -191,7 +191,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index") + new LongSumAggregatorFactory("idx", "index", null) ) ) .setGranularity(QueryRunnerTestHelper.dayGran) @@ -302,7 +302,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index") + new LongSumAggregatorFactory("idx", "index", null) ) ) .setGranularity(QueryRunnerTestHelper.dayGran) @@ -353,7 +353,8 @@ public class GroupByQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index" + "index", + null ) ) ) @@ -403,7 +404,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index") + new LongSumAggregatorFactory("idx", "index", null) ) ) .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)); @@ -479,7 +480,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index") + new LongSumAggregatorFactory("idx", "index", null) ) ) .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)) @@ -517,7 +518,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index") + new LongSumAggregatorFactory("idx", "index", null) ) ) .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)) @@ -596,7 +597,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index") + new LongSumAggregatorFactory("idx", "index", null) ) ) .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)) @@ -636,7 +637,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index") + new LongSumAggregatorFactory("idx", "index", null) ) ) .addOrderByColumn("rows") @@ -677,7 +678,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index") + new LongSumAggregatorFactory("idx", "index", null) ) ) .addOrderByColumn("rows", "desc") @@ -763,7 +764,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index") + new LongSumAggregatorFactory("idx", "index", null) ) ) .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)) @@ -954,7 +955,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index") + new LongSumAggregatorFactory("idx", "index", null) ) ) .setGranularity(QueryRunnerTestHelper.dayGran) @@ -967,8 +968,8 @@ public class GroupByQueryRunnerTest .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias"))) .setAggregatorSpecs( Arrays.asList( - new LongSumAggregatorFactory("rows", "rows"), - new LongSumAggregatorFactory("idx", "idx") + new LongSumAggregatorFactory("rows", "rows", null), + new LongSumAggregatorFactory("idx", "idx", null) ) ) .setGranularity(QueryRunnerTestHelper.dayGran) @@ -1012,7 +1013,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index") + new LongSumAggregatorFactory("idx", "index", null) ) ) .setGranularity(QueryRunnerTestHelper.dayGran) @@ -1050,7 +1051,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index") + new LongSumAggregatorFactory("idx", "index", null) ) ) .setGranularity(QueryRunnerTestHelper.dayGran) @@ -1087,7 +1088,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index") + new LongSumAggregatorFactory("idx", "index", null) ) ) .setGranularity(QueryRunnerTestHelper.dayGran) @@ -1121,7 +1122,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx_subagg", "index") + new LongSumAggregatorFactory("idx_subagg", "index", null) ) ) .setPostAggregatorSpecs( @@ -1145,8 +1146,8 @@ public class GroupByQueryRunnerTest .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias"))) .setAggregatorSpecs( Arrays.asList( - new LongSumAggregatorFactory("rows", "rows"), - new LongSumAggregatorFactory("idx", "idx_subpostagg") + new LongSumAggregatorFactory("rows", "rows", null), + new LongSumAggregatorFactory("idx", "idx_subpostagg", null) ) ) .setPostAggregatorSpecs( @@ -1202,7 +1203,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx_subagg", "index") + new LongSumAggregatorFactory("idx_subagg", "index", null) ) ) .setPostAggregatorSpecs( @@ -1245,8 +1246,8 @@ public class GroupByQueryRunnerTest .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias"))) .setAggregatorSpecs( Arrays.asList( - new LongSumAggregatorFactory("rows", "rows"), - new LongSumAggregatorFactory("idx", "idx_subpostagg") + new LongSumAggregatorFactory("rows", "rows", null), + new LongSumAggregatorFactory("idx", "idx_subpostagg", null) ) ) .setPostAggregatorSpecs( @@ -1350,8 +1351,8 @@ public class GroupByQueryRunnerTest .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias"))) .setAggregatorSpecs( Arrays.asList( - new LongSumAggregatorFactory("rows", "rows"), - new LongSumAggregatorFactory("idx", "idx_subpostagg"), + new LongSumAggregatorFactory("rows", "rows", null), + new LongSumAggregatorFactory("idx", "idx_subpostagg", null), new DoubleSumAggregatorFactory("js_outer_agg", "js_agg") ) ) diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java index 8557ba04c24..036f4a5ed9a 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java @@ -24,12 +24,10 @@ package io.druid.query.groupby; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; import io.druid.jackson.DefaultObjectMapper; -import io.druid.query.Druids; import io.druid.query.Query; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; -import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; import org.junit.Assert; @@ -53,7 +51,7 @@ public class GroupByQueryTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index") + new LongSumAggregatorFactory("idx", "index", null) ) ) .setGranularity(QueryRunnerTestHelper.dayGran) 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 8ad6c40dbec..f091134f7d7 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java @@ -77,7 +77,8 @@ public class TimeSeriesUnionQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index" + "index", + null ), QueryRunnerTestHelper.qualityUniques ) diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java index ff3266f303c..90fb7c327d9 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java @@ -36,8 +36,8 @@ import java.util.List; */ public class TimeseriesBinaryFnTest { - final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); - final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index"); + final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows", null); + final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index", null); final List aggregatorFactories = Arrays.asList( rowsCount, indexLongSum 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 95d654b9153..ae1a6f8e3c5 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java @@ -109,7 +109,7 @@ public class TimeseriesQueryRunnerBonusTest .intervals(ImmutableList.of(new Interval("2012-01-01T00:00:00Z/P1D"))) .aggregators( ImmutableList.of( - new CountAggregatorFactory("rows") + new CountAggregatorFactory("rows", null) ) ) .build(); 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 1d1fcb21b00..88f2fbb51e1 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -260,7 +260,8 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index" + "index", + null ), QueryRunnerTestHelper.qualityUniques ) @@ -301,7 +302,8 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index" + "index", + null ) ) ) @@ -355,7 +357,8 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index" + "index", + null ), QueryRunnerTestHelper.qualityUniques ) @@ -392,7 +395,8 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index" + "index", + null ), QueryRunnerTestHelper.qualityUniques ) @@ -440,7 +444,8 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index" + "index", + null ) ) ) @@ -487,7 +492,8 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index" + "index", + null ), QueryRunnerTestHelper.qualityUniques ) @@ -525,7 +531,8 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index" + "index", + null ), QueryRunnerTestHelper.qualityUniques ) @@ -568,7 +575,8 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index" + "index", + null ) ) ) diff --git a/processing/src/test/java/io/druid/query/topn/TopNBinaryFnBenchmark.java b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnBenchmark.java index 65ab16ad19b..eb97d72e4c5 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNBinaryFnBenchmark.java +++ b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnBenchmark.java @@ -67,10 +67,10 @@ public class TopNBinaryFnBenchmark extends SimpleBenchmark final List aggregatorFactories = new ArrayList<>(); - aggregatorFactories.add(new CountAggregatorFactory("rows")); - aggregatorFactories.add(new LongSumAggregatorFactory("index", "index")); + aggregatorFactories.add(new CountAggregatorFactory("rows", null)); + aggregatorFactories.add(new LongSumAggregatorFactory("index", "index", null)); for (int i = 1; i < aggCount; i++) { - aggregatorFactories.add(new CountAggregatorFactory("rows" + i)); + aggregatorFactories.add(new CountAggregatorFactory("rows" + i, null)); } final List postAggregators = new ArrayList<>(); for (int i = 0; i < postAggCount; i++) { diff --git a/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java index c8958dc6580..0b8d0c93c08 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java @@ -45,8 +45,8 @@ import java.util.Map; */ public class TopNBinaryFnTest { - final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); - final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index"); + final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows", null); + final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index", null); final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null); final FieldAccessPostAggregator rowsPostAgg = new FieldAccessPostAggregator("rows", "rows"); final FieldAccessPostAggregator indexPostAgg = new FieldAccessPostAggregator("index", "index"); diff --git a/processing/src/test/java/io/druid/segment/AppendTest.java b/processing/src/test/java/io/druid/segment/AppendTest.java index 017f99c293e..e32d5a11410 100644 --- a/processing/src/test/java/io/druid/segment/AppendTest.java +++ b/processing/src/test/java/io/druid/segment/AppendTest.java @@ -69,12 +69,12 @@ public class AppendTest { private static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ new DoubleSumAggregatorFactory("index", "index"), - new CountAggregatorFactory("count"), + new CountAggregatorFactory("count", null), new HyperUniquesAggregatorFactory("quality_uniques", "quality") }; private static final AggregatorFactory[] METRIC_AGGS_NO_UNIQ = new AggregatorFactory[]{ new DoubleSumAggregatorFactory("index", "index"), - new CountAggregatorFactory("count") + new CountAggregatorFactory("count", null) }; final String dataSource = "testing"; @@ -86,7 +86,7 @@ public class AppendTest final String placementDimension = "placement"; final String placementishDimension = "placementish"; final String indexMetric = "index"; - final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); + final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows", null); final DoubleSumAggregatorFactory indexDoubleSum = new DoubleSumAggregatorFactory("index", "index"); final HyperUniquesAggregatorFactory uniques = new HyperUniquesAggregatorFactory("uniques", "quality_uniques"); final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null); diff --git a/processing/src/test/java/io/druid/segment/SchemalessIndex.java b/processing/src/test/java/io/druid/segment/SchemalessIndex.java index 823fd83774a..9c03f107658 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessIndex.java +++ b/processing/src/test/java/io/druid/segment/SchemalessIndex.java @@ -68,12 +68,12 @@ public class SchemalessIndex private static final List METRICS = Arrays.asList("index"); private static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ new DoubleSumAggregatorFactory("index", "index"), - new CountAggregatorFactory("count"), + new CountAggregatorFactory("count", null), new HyperUniquesAggregatorFactory("quality_uniques", "quality") }; private static final AggregatorFactory[] METRIC_AGGS_NO_UNIQ = new AggregatorFactory[]{ new DoubleSumAggregatorFactory("index", "index"), - new CountAggregatorFactory("count") + new CountAggregatorFactory("count", null) }; private static final List> events = Lists.newArrayList(); diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java index 58a391c7515..1f3dbd3af24 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java @@ -76,7 +76,7 @@ public class SchemalessTestFull final String placementDimension = "placement"; final String placementishDimension = "placementish"; final String indexMetric = "index"; - final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); + final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows", null); final DoubleSumAggregatorFactory indexDoubleSum = new DoubleSumAggregatorFactory("index", "index"); final HyperUniquesAggregatorFactory uniques = new HyperUniquesAggregatorFactory("uniques", "quality_uniques"); final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null); diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java b/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java index 153a1226619..14909e3a5fe 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java @@ -103,7 +103,7 @@ public class SchemalessTestSimple final String placementDimension = "placement"; final String placementishDimension = "placementish"; final String indexMetric = "index"; - final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); + final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows", null); final DoubleSumAggregatorFactory indexDoubleSum = new DoubleSumAggregatorFactory("index", "index"); final HyperUniquesAggregatorFactory uniques = new HyperUniquesAggregatorFactory("uniques", "quality_uniques"); final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null); 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 7a131901501..4bf0638f0fd 100644 --- a/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java @@ -109,7 +109,7 @@ public class IncrementalIndexTest final IncrementalIndex index = new IncrementalIndex( 0L, QueryGranularity.NONE, - new AggregatorFactory[]{new CountAggregatorFactory("count")}, + new AggregatorFactory[]{new CountAggregatorFactory("count", null)}, TestQueryRunners.pool ); final int threadCount = 10; 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 12c3ca633f1..6992c531105 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java @@ -51,7 +51,6 @@ import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; import io.druid.segment.TestHelper; -import io.druid.segment.column.ColumnConfig; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import org.joda.time.DateTime; @@ -75,8 +74,8 @@ 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") + new CountAggregatorFactory("rows", null), + new LongSumAggregatorFactory("val", "val", null) }; private static List DIMS = Lists.newArrayList("dim", "dim.geo"); private final Segment segment; @@ -445,8 +444,8 @@ public class SpatialFilterBonusTest ) .aggregators( Arrays.asList( - new CountAggregatorFactory("rows"), - new LongSumAggregatorFactory("val", "val") + new CountAggregatorFactory("rows", null), + new LongSumAggregatorFactory("val", "val", null) ) ) .build(); @@ -496,8 +495,8 @@ public class SpatialFilterBonusTest ) .aggregators( Arrays.asList( - new CountAggregatorFactory("rows"), - new LongSumAggregatorFactory("val", "val") + new CountAggregatorFactory("rows", null), + new LongSumAggregatorFactory("val", "val", null) ) ) .build(); 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 f9693bdabf8..d9b815a3235 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java @@ -51,7 +51,6 @@ import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; import io.druid.segment.TestHelper; -import io.druid.segment.column.ColumnConfig; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import org.joda.time.DateTime; @@ -75,8 +74,8 @@ public class SpatialFilterTest { 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") + new CountAggregatorFactory("rows", null), + new LongSumAggregatorFactory("val", "val", null) }; private static List DIMS = Lists.newArrayList("dim", "lat", "long"); private final Segment segment; @@ -473,8 +472,8 @@ public class SpatialFilterTest ) .aggregators( Arrays.asList( - new CountAggregatorFactory("rows"), - new LongSumAggregatorFactory("val", "val") + new CountAggregatorFactory("rows", null), + new LongSumAggregatorFactory("val", "val", null) ) ) .build(); @@ -524,8 +523,8 @@ public class SpatialFilterTest ) .aggregators( Arrays.asList( - new CountAggregatorFactory("rows"), - new LongSumAggregatorFactory("val", "val") + new CountAggregatorFactory("rows", null), + new LongSumAggregatorFactory("val", "val", null) ) ) .build(); 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 1eee18fe782..8fd1446a8d3 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -53,7 +53,6 @@ import org.junit.Test; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.List; /** */ @@ -63,7 +62,7 @@ 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", null)}, TestQueryRunners.pool ); @@ -110,7 +109,7 @@ public class IncrementalIndexStorageAdapterTest .setInterval(new Interval(0, new DateTime().getMillis())) .addDimension("billy") .addDimension("sally") - .addAggregator(new LongSumAggregatorFactory("cnt", "cnt")) + .addAggregator(new LongSumAggregatorFactory("cnt", "cnt", null)) .build(), new IncrementalIndexStorageAdapter(index) ); @@ -129,7 +128,7 @@ 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", null)}, TestQueryRunners.pool ); @@ -182,7 +181,7 @@ 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", null)}, TestQueryRunners.pool ); @@ -220,7 +219,8 @@ public class IncrementalIndexStorageAdapterTest Lists.newArrayList( new LongSumAggregatorFactory( "cnt", - "cnt" + "cnt", + null ) ) ) @@ -238,7 +238,7 @@ 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", null)}, TestQueryRunners.pool ); @@ -285,7 +285,7 @@ public class IncrementalIndexStorageAdapterTest .setInterval(new Interval(0, new DateTime().getMillis())) .addDimension("billy") .addDimension("sally") - .addAggregator(new LongSumAggregatorFactory("cnt", "cnt")) + .addAggregator(new LongSumAggregatorFactory("cnt", "cnt", null)) .setDimFilter(DimFilters.dimEquals("sally", (String) null)) .build(), new IncrementalIndexStorageAdapter(index) diff --git a/server/src/main/antlr4/io/druid/sql/antlr4/DruidSQL.g4 b/server/src/main/antlr4/io/druid/sql/antlr4/DruidSQL.g4 index 37716ab6673..ed810051070 100644 --- a/server/src/main/antlr4/io/druid/sql/antlr4/DruidSQL.g4 +++ b/server/src/main/antlr4/io/druid/sql/antlr4/DruidSQL.g4 @@ -72,7 +72,7 @@ import java.util.Map; case SUM: return new DoubleSumAggregatorFactory("sum("+name+")", name); case MIN: return new MinAggregatorFactory("min("+name+")", name); case MAX: return new MaxAggregatorFactory("max("+name+")", name); - case COUNT: return new CountAggregatorFactory(name); + case COUNT: return new CountAggregatorFactory(name, null); } throw new IllegalArgumentException("Unknown function [" + fn + "]"); } diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 0834a0d9f8b..edd1e307179 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -150,9 +150,9 @@ public class CachingClusteredClientTest */ private static final int RANDOMNESS = 10; private static final List AGGS = Arrays.asList( - new CountAggregatorFactory("rows"), - new LongSumAggregatorFactory("imps", "imps"), - new LongSumAggregatorFactory("impers", "imps") + new CountAggregatorFactory("rows", null), + new LongSumAggregatorFactory("imps", "imps", null), + new LongSumAggregatorFactory("impers", "imps", null) ); private static final List POST_AGGS = Arrays.asList( new ArithmeticPostAggregator( @@ -181,9 +181,9 @@ public class CachingClusteredClientTest ) ); private static final List RENAMED_AGGS = Arrays.asList( - new CountAggregatorFactory("rows2"), - new LongSumAggregatorFactory("imps", "imps"), - new LongSumAggregatorFactory("impers2", "imps") + new CountAggregatorFactory("rows2", null), + new LongSumAggregatorFactory("imps", "imps", null), + new LongSumAggregatorFactory("impers2", "imps", null) ); private static final DimFilter DIM_FILTER = null; private static final List RENAMED_POST_AGGS = Arrays.asList(); diff --git a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java index a358ec44798..c64996e6bc3 100644 --- a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java @@ -66,9 +66,9 @@ public class CachingQueryRunnerTest { private static final List AGGS = Arrays.asList( - new CountAggregatorFactory("rows"), - new LongSumAggregatorFactory("imps", "imps"), - new LongSumAggregatorFactory("impers", "imps") + new CountAggregatorFactory("rows", null), + new LongSumAggregatorFactory("imps", "imps", null), + new LongSumAggregatorFactory("impers", "imps", null) ); private static final Object[] objects = new Object[]{ 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..f812c24b974 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -66,7 +66,7 @@ public class FireDepartmentTest null, null, null, null ), new AggregatorFactory[]{ - new CountAggregatorFactory("count") + new CountAggregatorFactory("count", null) }, new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.MINUTE, null, Granularity.HOUR) ), diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index cb464a9ff8f..3142efa29e8 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -73,7 +73,7 @@ public class RealtimeManagerTest schema = new DataSchema( "test", null, - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, + new AggregatorFactory[]{new CountAggregatorFactory("rows", null)}, new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.NONE, null, Granularity.HOUR) ); RealtimeIOConfig ioConfig = new RealtimeIOConfig( 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 b580e83d3c5..9a57b78c462 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 @@ -41,7 +41,6 @@ import io.druid.query.Query; import io.druid.query.QueryRunnerFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; -import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -102,7 +101,7 @@ public class RealtimePlumberSchoolTest return null; } }, - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, + new AggregatorFactory[]{new CountAggregatorFactory("rows", null)}, new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.NONE, null, Granularity.HOUR) ); diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index 0ebb59d4410..c2cc0d9517e 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -49,7 +49,7 @@ public class SinkTest final DataSchema schema = new DataSchema( "test", null, - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, + new AggregatorFactory[]{new CountAggregatorFactory("rows", null)}, new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.MINUTE, null, Granularity.HOUR) ); diff --git a/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java b/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java index 5c2320f9885..0004679b79a 100644 --- a/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java +++ b/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java @@ -113,7 +113,7 @@ public class TieredBrokerHostSelectorTest Druids.newTimeseriesQueryBuilder() .dataSource("test") .granularity("all") - .aggregators(Arrays.asList(new CountAggregatorFactory("rows"))) + .aggregators(Arrays.asList(new CountAggregatorFactory("rows", null))) .intervals(Arrays.asList(new Interval("2011-08-31/2011-09-01"))) .build() ).lhs; @@ -129,7 +129,7 @@ public class TieredBrokerHostSelectorTest Druids.newTimeseriesQueryBuilder() .dataSource("test") .granularity("all") - .aggregators(Arrays.asList(new CountAggregatorFactory("rows"))) + .aggregators(Arrays.asList(new CountAggregatorFactory("rows", null))) .intervals(Arrays.asList(new Interval("2013-08-31/2013-09-01"))) .build() ).lhs; @@ -144,7 +144,7 @@ public class TieredBrokerHostSelectorTest Druids.newTimeseriesQueryBuilder() .dataSource("test") .granularity("all") - .aggregators(Arrays.asList(new CountAggregatorFactory("rows"))) + .aggregators(Arrays.asList(new CountAggregatorFactory("rows", null))) .intervals(Arrays.asList(new Interval("2010-08-31/2010-09-01"))) .build() ).lhs; @@ -158,7 +158,7 @@ public class TieredBrokerHostSelectorTest String brokerName = (String) brokerSelector.select( Druids.newTimeseriesQueryBuilder() .dataSource("test") - .aggregators(Arrays.asList(new CountAggregatorFactory("count"))) + .aggregators(Arrays.asList(new CountAggregatorFactory("count", null))) .intervals( new MultipleIntervalSegmentSpec( Arrays.asList( @@ -179,7 +179,7 @@ public class TieredBrokerHostSelectorTest String brokerName = (String) brokerSelector.select( Druids.newTimeseriesQueryBuilder() .dataSource("test") - .aggregators(Arrays.asList(new CountAggregatorFactory("count"))) + .aggregators(Arrays.asList(new CountAggregatorFactory("count", null))) .intervals( new MultipleIntervalSegmentSpec( Arrays.asList( @@ -200,7 +200,7 @@ public class TieredBrokerHostSelectorTest String brokerName = (String) brokerSelector.select( Druids.newTimeseriesQueryBuilder() .dataSource("test") - .aggregators(Arrays.asList(new CountAggregatorFactory("count"))) + .aggregators(Arrays.asList(new CountAggregatorFactory("count", null))) .intervals( new MultipleIntervalSegmentSpec( Arrays.asList( @@ -223,7 +223,7 @@ public class TieredBrokerHostSelectorTest String brokerName = (String) brokerSelector.select( Druids.newTimeseriesQueryBuilder() .dataSource("test") - .aggregators(Arrays.asList(new CountAggregatorFactory("count"))) + .aggregators(Arrays.asList(new CountAggregatorFactory("count", null))) .intervals( new MultipleIntervalSegmentSpec( Arrays.asList( From 0b42ca4c75a451e221bf74d312693f9eead9f945 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 13 Aug 2014 18:34:00 +0530 Subject: [PATCH 046/107] return correct valueType --- .../java/io/druid/query/aggregation/CountAggregatorFactory.java | 2 +- .../io/druid/query/aggregation/LongSumAggregatorFactory.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java index 83baa4fd41b..5845eda1278 100644 --- a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java @@ -123,7 +123,7 @@ public class CountAggregatorFactory implements AggregatorFactory @Override public String getTypeName() { - return "long"; + return valueType; } @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 bbac473f5de..bf18580ac62 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java @@ -139,7 +139,7 @@ public class LongSumAggregatorFactory implements AggregatorFactory @Override public String getTypeName() { - return "long"; + return valueType; } @Override From 18d3acd3a81359c430fdf7f4a61dea63e5acd9a4 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 14 Aug 2014 11:11:57 -0700 Subject: [PATCH 047/107] fix all manners of brokenness from nulls and empty strings --- .../java/io/druid/segment/IndexMaker.java | 331 +++++++++++++----- .../io/druid/segment/SchemalessTestFull.java | 32 +- 2 files changed, 251 insertions(+), 112 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/IndexMaker.java b/processing/src/main/java/io/druid/segment/IndexMaker.java index f88a25bf573..33be9558174 100644 --- a/processing/src/main/java/io/druid/segment/IndexMaker.java +++ b/processing/src/main/java/io/druid/segment/IndexMaker.java @@ -23,7 +23,6 @@ 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.Predicates; import com.google.common.base.Splitter; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; @@ -91,7 +90,6 @@ import java.nio.LongBuffer; import java.util.AbstractList; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -463,6 +461,7 @@ public class IndexMaker ); final Map dimIndexes = Maps.newHashMap(); + final Map dimensionCardinalities = Maps.newHashMap(); final Map> dimensionValuesLookup = Maps.newHashMap(); final ArrayList> dimConversions = Lists.newArrayListWithCapacity(adapters.size()); final Set skippedDimensions = Sets.newHashSet(); @@ -476,6 +475,7 @@ public class IndexMaker dimConversions, dimIndexes, skippedDimensions, + dimensionCardinalities, dimensionValuesLookup ); @@ -503,6 +503,7 @@ public class IndexMaker skippedDimensions, theRows, columnCapabilities, + dimensionCardinalities, dimensionValuesLookup, rowNumConversions ); @@ -527,6 +528,7 @@ public class IndexMaker final List> dimConversions, final Map dimIndexes, final Set skippedDimensions, + final Map dimensionCardinalities, final Map> dimensionValuesLookup ) { @@ -557,7 +559,28 @@ public class IndexMaker // sort all dimension values and treat all null values as empty strings final Iterable dimensionValues = CombiningIterable.createSplatted( - dimValueLookups, + 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().nullsFirst() ); @@ -572,6 +595,9 @@ public class IndexMaker ++cardinality; } + + dimensionCardinalities.put(dimension, cardinality); + if (cardinality == 0) { log.info("Skipping [%s], it is empty!", dimension); skippedDimensions.add(dimension); @@ -749,6 +775,7 @@ public class IndexMaker final Set skippedDimensions, final Iterable theRows, final Map columnCapabilities, + final Map dimensionCardinalities, final Map> dimensionValuesLookup, final List rowNumConversions ) throws IOException @@ -771,6 +798,7 @@ public class IndexMaker dimIndex, dimension, columnCapabilities, + dimensionCardinalities, dimensionValuesLookup, rowNumConversions ); @@ -788,6 +816,7 @@ public class IndexMaker final int dimIndex, final String dimension, final Map columnCapabilities, + final Map dimensionCardinalities, final Map> dimensionValuesLookup, final List rowNumConversions ) throws IOException @@ -818,6 +847,7 @@ public class IndexMaker ConciseSet nullSet = null; int rowCount = 0; + for (Rowboat theRow : theRows) { if (dimIndex > theRow.getDims().length) { if (nullSet == null) { @@ -838,44 +868,149 @@ public class IndexMaker rowCount++; } - GenericIndexed dictionary = null; final Iterable dimensionValues = dimensionValuesLookup.get(dimension); + GenericIndexed dictionary = GenericIndexed.fromIterable( + dimensionValues, + GenericIndexed.stringStrategy + ); boolean bumpDictionary = false; if (hasMultipleValues) { - List> vals = ((MultiValColumnDictionaryEntryStore) adder).get(); - multiValCol = VSizeIndexed.fromIterable( - FunctionalIterable - .create(vals) - //.filter(Predicates.>notNull()) - .transform( - new Function, VSizeIndexedInts>() - { - @Override - public VSizeIndexedInts apply(List input) - { - if (input == null) { - return VSizeIndexedInts.empty(); + 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 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 + ); } - return VSizeIndexedInts.fromList( - input, - Collections.max(input) - ); } - } - ) - ); - dictionary = GenericIndexed.fromIterable( - dimensionValues, - GenericIndexed.stringStrategy - ); + ) + ); + } } else { final List vals = ((SingleValColumnDictionaryEntryStore) adder).get(); if (nullSet != null) { log.info("Dimension[%s] has null rows.", dimension); - if (Iterables.getFirst(dimensionValues, "") != null) { + if (Iterables.getFirst(dimensionValues, null) != null) { bumpDictionary = true; log.info("Dimension[%s] has no null value in the dictionary, expanding...", dimension); @@ -899,6 +1034,27 @@ public class IndexMaker 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() { @@ -908,10 +1064,6 @@ public class IndexMaker ); } } else { - dictionary = GenericIndexed.fromIterable( - dimensionValues, - GenericIndexed.stringStrategy - ); singleValCol = VSizeIndexedInts.fromList(vals, dictionary.size()); } } @@ -942,65 +1094,49 @@ public class IndexMaker } GenericIndexed bitmaps; - if (!hasMultipleValues) { - 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 { + if (nullSet != null) { + final ImmutableConciseSet theNullSet = ImmutableConciseSet.newImmutableFromMutable(nullSet); + if (bumpDictionary) { bitmaps = GenericIndexed.fromIterable( - Iterables.transform( - conciseSets, - new Function() - { - @Override - public ImmutableConciseSet apply(ConciseSet input) - { - return ImmutableConciseSet.newImmutableFromMutable(input); - } - } + 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 ); @@ -1033,12 +1169,15 @@ public class IndexMaker int dimValIndex = 0; for (String dimVal : dimensionValuesLookup.get(dimension)) { if (hasSpatialIndexes) { - 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)); + 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)); } - tree.insert(coords, conciseSets.get(dimValIndex++)); + dimValIndex++; } } if (hasSpatialIndexes) { @@ -1517,7 +1656,7 @@ public class IndexMaker final Rowboat retVal = new Rowboat( lhs.getTimestamp(), lhs.getDims(), - lhs.getMetrics(), + metrics, lhs.getRowNum() ); diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java index 58a391c7515..0efa20822ca 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("provider", "") + .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("provider", "spot") .put("rows", 4L) @@ -1183,15 +1192,6 @@ public class SchemalessTestFull .put("maxIndex", 100.0) .put("minIndex", 100.0) .build(), - ImmutableMap.builder() - .put("provider", "") - .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("provider", "total_market") .put("rows", 2L) @@ -1386,7 +1386,7 @@ public class SchemalessTestFull .build(); failMsg += " timeseries "; - HashMap context = new HashMap(); + HashMap context = new HashMap(); Iterable> actualResults = Sequences.toList( runner.run(query, context), Lists.>newArrayList() @@ -1420,7 +1420,7 @@ public class SchemalessTestFull .build(); failMsg += " filtered timeseries "; - HashMap context = new HashMap(); + HashMap context = new HashMap(); Iterable> actualResults = Sequences.toList( runner.run(query, context), Lists.>newArrayList() @@ -1453,7 +1453,7 @@ public class SchemalessTestFull .build(); failMsg += " topN "; - HashMap context = new HashMap(); + HashMap context = new HashMap(); Iterable> actualResults = Sequences.toList( runner.run(query, context), Lists.>newArrayList() @@ -1487,7 +1487,7 @@ public class SchemalessTestFull .build(); failMsg += " filtered topN "; - HashMap context = new HashMap(); + HashMap context = new HashMap(); Iterable> actualResults = Sequences.toList( runner.run(query, context), Lists.>newArrayList() @@ -1505,7 +1505,7 @@ public class SchemalessTestFull .build(); failMsg += " search "; - HashMap context = new HashMap(); + HashMap context = new HashMap(); Iterable> actualResults = Sequences.toList( runner.run(query, context), Lists.>newArrayList() @@ -1524,7 +1524,7 @@ public class SchemalessTestFull .build(); failMsg += " filtered search "; - HashMap context = new HashMap(); + HashMap context = new HashMap(); Iterable> actualResults = Sequences.toList( runner.run(query, context), Lists.>newArrayList() @@ -1543,7 +1543,7 @@ public class SchemalessTestFull .build(); failMsg += " timeBoundary "; - HashMap context = new HashMap(); + HashMap context = new HashMap(); Iterable> actualResults = Sequences.toList( runner.run(query, context), Lists.>newArrayList() From 603152eb184eb2fb7eacf4b76247c5cfb27fd62e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 12 Aug 2014 14:02:08 -0700 Subject: [PATCH 048/107] fix compilation with Java 8 --- .../src/test/java/io/druid/query/RetryQueryRunnerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java index 16c8a51504a..9c6329a3df4 100644 --- a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java @@ -131,7 +131,7 @@ public class RetryQueryRunnerTest @Override public Sequence run(Query query, Map context) { - if (context.get("count") == 0) { + if ((int)context.get("count") == 0) { ((List) context.get(RetryQueryRunner.missingSegments)).add( new SegmentDescriptor( new Interval( @@ -368,4 +368,4 @@ public class RetryQueryRunnerTest Assert.assertTrue("Should have one entry in the list of missing segments", ((List) context.get(RetryQueryRunner.missingSegments)).size() == 1); } -} \ No newline at end of file +} From 6f0e768d8260290896c71041b8b59343283cb2d3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 14 Aug 2014 17:09:12 -0700 Subject: [PATCH 049/107] add tools comand to create metadata storage tables --- .../main/java/io/druid/cli/CreateTables.java | 113 ++++++++++++++++++ services/src/main/java/io/druid/cli/Main.java | 2 +- 2 files changed, 114 insertions(+), 1 deletion(-) create mode 100644 services/src/main/java/io/druid/cli/CreateTables.java diff --git a/services/src/main/java/io/druid/cli/CreateTables.java b/services/src/main/java/io/druid/cli/CreateTables.java new file mode 100644 index 00000000000..3c788ab279e --- /dev/null +++ b/services/src/main/java/io/druid/cli/CreateTables.java @@ -0,0 +1,113 @@ +/* + * 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.cli; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.metamx.common.logger.Logger; +import io.airlift.command.Command; +import io.airlift.command.Option; +import io.druid.db.DbConnector; +import io.druid.db.DbConnectorConfig; +import io.druid.db.DbTablesConfig; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.annotations.Self; +import io.druid.server.DruidNode; + +import java.util.List; + +@Command( + name = "metadata-init", + description = "Initialize Metadata Storage" +) +public class CreateTables extends GuiceRunnable +{ + @Option(name = "--connectURI", description = "Database JDBC connection string", required = true) + private String connectURI; + + @Option(name = "--user", description = "Database username", required = true) + private String user; + + @Option(name = "--password", description = "Database password", required = true) + private String password; + + @Option(name = "--base", description = "Base table name") + private String base; + + private static final Logger log = new Logger(CreateTables.class); + + public CreateTables() + { + super(log); + } + + @Override + protected List getModules() + { + return ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bindInstance( + binder, Key.get(DbConnectorConfig.class), new DbConnectorConfig() + { + @Override + public String getConnectURI() + { + return connectURI; + } + + @Override + public String getUser() + { + return user; + } + + @Override + public String getPassword() + { + return password; + } + } + ); + JsonConfigProvider.bindInstance( + binder, Key.get(DbTablesConfig.class), DbTablesConfig.fromBase(base) + ); + JsonConfigProvider.bindInstance( + binder, Key.get(DruidNode.class, Self.class), new DruidNode("tools", "localhost", -1) + ); + } + } + ); + } + + @Override + public void run() + { + final Injector injector = makeInjector(); + DbConnector dbConnector = injector.getInstance(DbConnector.class); + dbConnector.createSegmentTable(); + } +} diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index 0e23a0e81e7..00f48d11f94 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.java @@ -61,7 +61,7 @@ public class Main builder.withGroup("tools") .withDescription("Various tools for working with Druid") .withDefaultCommand(Help.class) - .withCommands(ConvertProperties.class, DruidJsonValidator.class, PullDependencies.class); + .withCommands(ConvertProperties.class, DruidJsonValidator.class, PullDependencies.class, CreateTables.class); builder.withGroup("index") .withDescription("Run indexing for druid") From 17a3e65dc43da8b24e53490fc34a8e37b92ce343 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 15 Aug 2014 12:50:30 -0700 Subject: [PATCH 050/107] add missing tables --- services/src/main/java/io/druid/cli/CreateTables.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/services/src/main/java/io/druid/cli/CreateTables.java b/services/src/main/java/io/druid/cli/CreateTables.java index 3c788ab279e..5dfbf02de45 100644 --- a/services/src/main/java/io/druid/cli/CreateTables.java +++ b/services/src/main/java/io/druid/cli/CreateTables.java @@ -109,5 +109,8 @@ public class CreateTables extends GuiceRunnable final Injector injector = makeInjector(); DbConnector dbConnector = injector.getInstance(DbConnector.class); dbConnector.createSegmentTable(); + dbConnector.createRulesTable(); + dbConnector.createConfigTable(); + dbConnector.createTaskTables(); } } From 1fd30ab58852876808f0b3de8bccb4ab392966fc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 15 Aug 2014 17:14:05 -0700 Subject: [PATCH 051/107] default service/host/port for all nodes --- .../config/ForkingTaskRunnerConfig.java | 2 +- .../main/java/io/druid/server/DruidNode.java | 21 +++++++++++++------ .../server/coordinator/DruidCoordinator.java | 2 +- .../coordinator/DruidCoordinatorConfig.java | 3 --- .../initialization/InitializationTest.java | 2 +- .../coordinator/DruidCoordinatorTest.java | 6 ------ .../src/main/java/io/druid/cli/CliBridge.java | 4 ++++ .../src/main/java/io/druid/cli/CliBroker.java | 4 ++++ .../java/io/druid/cli/CliCoordinator.java | 4 ++++ .../main/java/io/druid/cli/CliHistorical.java | 4 ++++ .../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 ++++ 14 files changed, 58 insertions(+), 19 deletions(-) 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/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/coordinator/DruidCoordinator.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java index 51b1773e26f..3b6a7276d57 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/test/java/io/druid/initialization/InitializationTest.java b/server/src/test/java/io/druid/initialization/InitializationTest.java index f4bcc3708d2..6f60a51954d 100644 --- a/server/src/test/java/io/druid/initialization/InitializationTest.java +++ b/server/src/test/java/io/druid/initialization/InitializationTest.java @@ -109,7 +109,7 @@ public class InitializationTest public void configure(Binder binder) { JsonConfigProvider.bindInstance( - binder, Key.get(DruidNode.class, Self.class), new DruidNode("hadoop-indexer", "localhost", -1) + binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", "localhost", -1) ); } } diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java index c0d6bfa1c36..d3d9d697047 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java @@ -77,12 +77,6 @@ public class DruidCoordinatorTest coordinator = new DruidCoordinator( new DruidCoordinatorConfig() { - @Override - public String getHost() - { - return null; - } - @Override public Duration getCoordinatorStartDelay() { diff --git a/services/src/main/java/io/druid/cli/CliBridge.java b/services/src/main/java/io/druid/cli/CliBridge.java index ffed4789727..6da1ec21c38 100644 --- a/services/src/main/java/io/druid/cli/CliBridge.java +++ b/services/src/main/java/io/druid/cli/CliBridge.java @@ -5,6 +5,7 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; +import com.google.inject.name.Names; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import io.airlift.command.Command; @@ -69,6 +70,9 @@ public class CliBridge extends ServerRunnable @Override public void configure(Binder binder) { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/bridge"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8081); + ConfigProvider.bind(binder, BridgeCuratorConfig.class); binder.bind(BridgeZkCoordinator.class).in(ManageLifecycle.class); diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index 41336ed7c33..af37fa80134 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -22,6 +22,7 @@ package io.druid.cli; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Module; +import com.google.inject.name.Names; import com.metamx.common.logger.Logger; import io.airlift.command.Command; import io.druid.client.BrokerServerView; @@ -78,6 +79,9 @@ public class CliBroker extends ServerRunnable @Override public void configure(Binder binder) { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/broker"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8082); + binder.bind(QueryToolChestWarehouse.class).to(MapQueryToolChestWarehouse.class); binder.bind(CachingClusteredClient.class).in(LazySingleton.class); diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 95919bc3de5..4d060c4acac 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; +import com.google.inject.name.Names; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.logger.Logger; import io.airlift.command.Command; @@ -87,6 +88,9 @@ public class CliCoordinator extends ServerRunnable @Override public void configure(Binder binder) { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/coordinator"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8081); + ConfigProvider.bind(binder, DruidCoordinatorConfig.class); JsonConfigProvider.bind(binder, "druid.manager.segments", DatabaseSegmentManagerConfig.class); diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 6d4152b9226..b1098114e72 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -22,6 +22,7 @@ package io.druid.cli; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Module; +import com.google.inject.name.Names; import com.metamx.common.logger.Logger; import io.airlift.command.Command; import io.druid.client.cache.Cache; @@ -68,6 +69,9 @@ public class CliHistorical extends ServerRunnable @Override public void configure(Binder binder) { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/historical"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8083); + binder.bind(ServerManager.class).in(LazySingleton.class); binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); binder.bind(QuerySegmentWalker.class).to(ServerManager.class).in(LazySingleton.class); diff --git a/services/src/main/java/io/druid/cli/CliMiddleManager.java b/services/src/main/java/io/druid/cli/CliMiddleManager.java index 381b382269a..e458a350fd2 100644 --- a/services/src/main/java/io/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/io/druid/cli/CliMiddleManager.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; +import com.google.inject.name.Names; import com.google.inject.util.Providers; import com.metamx.common.logger.Logger; import io.airlift.command.Command; @@ -74,6 +75,9 @@ public class CliMiddleManager extends ServerRunnable @Override public void configure(Binder binder) { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/middlemanager"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8091); + IndexingServiceModuleHelper.configureTaskRunnerConfigs(binder); JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class); diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 95780b57156..b39d1847858 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -26,6 +26,7 @@ import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; +import com.google.inject.name.Names; import com.google.inject.servlet.GuiceFilter; import com.google.inject.util.Providers; import com.metamx.common.logger.Logger; @@ -114,6 +115,9 @@ public class CliOverlord extends ServerRunnable @Override public void configure(Binder binder) { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/overlord"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8090); + JsonConfigProvider.bind(binder, "druid.indexer.queue", TaskQueueConfig.class); JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class); diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index e80d22242c5..a34e3bd0eda 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -20,6 +20,9 @@ package io.druid.cli; import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.name.Names; import com.metamx.common.logger.Logger; import io.airlift.command.Command; import io.druid.guice.RealtimeModule; @@ -45,7 +48,15 @@ public class CliRealtime extends ServerRunnable protected List getModules() { return ImmutableList.of( - new RealtimeModule() + new RealtimeModule(), + new Module() { + @Override + public void configure(Binder binder) + { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/realtime"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8084); + } + } ); } } diff --git a/services/src/main/java/io/druid/cli/CliRouter.java b/services/src/main/java/io/druid/cli/CliRouter.java index 355e1f993a3..550f94e7c47 100644 --- a/services/src/main/java/io/druid/cli/CliRouter.java +++ b/services/src/main/java/io/druid/cli/CliRouter.java @@ -24,6 +24,7 @@ import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.TypeLiteral; +import com.google.inject.name.Names; import com.metamx.common.logger.Logger; import io.airlift.command.Command; import io.druid.curator.discovery.DiscoveryModule; @@ -72,6 +73,9 @@ public class CliRouter extends ServerRunnable @Override public void configure(Binder binder) { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/router"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8888); + JsonConfigProvider.bind(binder, "druid.router", TieredBrokerConfig.class); binder.bind(CoordinatorRuleManager.class); From 4dd01d4eaee26ab188ec4e8af48a7115e90ffb6e Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Mon, 18 Aug 2014 19:34:56 +0530 Subject: [PATCH 052/107] review comments --- .../io/druid/segment/incremental/IncrementalIndex.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) 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 26985edc024..722153ccbef 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -81,7 +81,7 @@ public class IncrementalIndex implements Iterable, Closeable private final long minTimestamp; private final QueryGranularity gran; - private final Set> rowTransformers; + private final List> rowTransformers; private final AggregatorFactory[] metrics; private final Map metricIndexes; @@ -120,7 +120,7 @@ public class IncrementalIndex implements Iterable, Closeable this.minTimestamp = incrementalIndexSchema.getMinTimestamp(); this.gran = incrementalIndexSchema.getGran(); this.metrics = incrementalIndexSchema.getMetrics(); - this.rowTransformers = Sets.newHashSet(); + this.rowTransformers = Lists.newCopyOnWriteArrayList(); final ImmutableList.Builder metricNamesBuilder = ImmutableList.builder(); final ImmutableMap.Builder metricIndexesBuilder = ImmutableMap.builder(); @@ -447,13 +447,12 @@ public class IncrementalIndex implements Iterable, Closeable if (!facts.containsKey(key)) { int rowOffset = totalAggSize * numEntries.getAndIncrement(); if (rowOffset + totalAggSize > bufferHolder.get().limit()) { - throw new ISE("Buffer Full cannot add more rows current rowSize : %d", numEntries.get()); + throw new ISE("Buffer full, cannot add more rows! Current rowSize[%,d].", numEntries.get()); } for (int i = 0; i < aggs.length; i++) { aggs[i].init(bufferHolder.get(), getMetricPosition(rowOffset, i)); } facts.put(key, rowOffset); - } } in.set(row); From 8f7c6050f97733a3caa6e7f9d61b3c2ed5b3046d Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 18 Aug 2014 12:56:51 -0700 Subject: [PATCH 053/107] fix --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 2a5f1efae96..40734ed8d51 100644 --- a/README.md +++ b/README.md @@ -4,7 +4,7 @@ Druid is a distributed, column-oriented, real-time analytics data store that is commonly used to power exploratory dashboards in multi-tenant environments. Druid excels as a data warehousing solution for fast aggregate queries on petabyte sized data sets. Druid supports a variety of flexible filters, exact -calculations, approximate algorithms, and other useful calculations Druid can +calculations, approximate algorithms, and other useful calculations. Druid can load both streaming and batch data and integrates with Storm and Hadoop. ### More Information From 78054d7f0516424b066fb80208d0c9a59c5d12c9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 18 Aug 2014 13:37:47 -0700 Subject: [PATCH 054/107] fix test initialization --- .../server/initialization/JettyTest.java | 73 +++++++++---------- 1 file changed, 35 insertions(+), 38 deletions(-) diff --git a/server/src/test/java/io/druid/server/initialization/JettyTest.java b/server/src/test/java/io/druid/server/initialization/JettyTest.java index 3d03bf8a22a..a95e883b65d 100644 --- a/server/src/test/java/io/druid/server/initialization/JettyTest.java +++ b/server/src/test/java/io/druid/server/initialization/JettyTest.java @@ -20,11 +20,12 @@ package io.druid.server.initialization; import com.google.api.client.repackaged.com.google.common.base.Throwables; -import com.google.common.collect.Lists; +import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Binder; import com.google.inject.Inject; import com.google.inject.Injector; +import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.servlet.GuiceFilter; import com.metamx.common.lifecycle.Lifecycle; @@ -33,10 +34,14 @@ import com.metamx.http.client.response.InputStreamResponseHandler; import com.metamx.http.client.response.StatusResponseHandler; import com.metamx.http.client.response.StatusResponseHolder; import io.druid.guice.Jerseys; +import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; +import io.druid.guice.LifecycleModule; import io.druid.guice.annotations.Global; import io.druid.guice.GuiceInjectors; +import io.druid.guice.annotations.Self; import io.druid.initialization.Initialization; +import io.druid.server.DruidNode; import org.apache.commons.io.IOUtils; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; @@ -78,10 +83,7 @@ public class JettyTest public static void setProperties() { - System.setProperty("druid.host", "localhost:9999"); - System.setProperty("druid.port", "9999"); System.setProperty("druid.server.http.numThreads", "20"); - System.setProperty("druid.service", "test"); System.setProperty("druid.server.http.maxIdleTime", "PT1S"); System.setProperty("druid.global.http.readTimeout", "PT1S"); } @@ -91,22 +93,24 @@ public class JettyTest { setProperties(); Injector injector = Initialization.makeInjectorWithModules( - GuiceInjectors.makeStartupInjector(), Lists.newArrayList( - new Module() - { - @Override - public void configure(Binder binder) - { - binder.bind(JettyServerInitializer.class).to(JettyServerInit.class).in(LazySingleton.class); - Jerseys.addResource(binder, SlowResource.class); - Jerseys.addResource(binder, ExceptionResource.class); - } - } - ) + GuiceInjectors.makeStartupInjector(), ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bindInstance( + binder, Key.get(DruidNode.class, Self.class), new DruidNode("test", "localhost", 9999) + ); + binder.bind(JettyServerInitializer.class).to(JettyServerInit.class).in(LazySingleton.class); + Jerseys.addResource(binder, SlowResource.class); + Jerseys.addResource(binder, ExceptionResource.class); + LifecycleModule.register(binder, Server.class); + } + } + ) ); lifecycle = injector.getInstance(Lifecycle.class); - // Jetty is Lazy Initialized do a getInstance - injector.getInstance(Server.class); lifecycle.start(); ClientHolder holder = injector.getInstance(ClientHolder.class); client = holder.getClient(); @@ -136,12 +140,9 @@ public class JettyTest long startTime = System.currentTimeMillis(); long startTime2 = 0; try { - ListenableFuture go = client.get( - new URL( - "http://localhost:9999/slow/hello" - ) - ) - .go(new StatusResponseHandler(Charset.defaultCharset())); + ListenableFuture go = + client.get(new URL("http://localhost:9999/slow/hello")) + .go(new StatusResponseHandler(Charset.defaultCharset())); startTime2 = System.currentTimeMillis(); go.get(); } @@ -150,13 +151,13 @@ public class JettyTest } finally { System.out - .println( - "Response time client" - + (System.currentTimeMillis() - startTime) - + "time taken for getting future" - + (System.currentTimeMillis() - startTime2) - + "Counter " + count.incrementAndGet() - ); + .println( + "Response time client" + + (System.currentTimeMillis() - startTime) + + "time taken for getting future" + + (System.currentTimeMillis() - startTime2) + + "Counter " + count.incrementAndGet() + ); latch.countDown(); } @@ -178,13 +179,9 @@ public class JettyTest // above bug is not fixed in jetty for gzip encoding, and the chunk is still finalized instead of throwing exception. public void testChunkNotFinalized() throws Exception { - ListenableFuture go = client.get( - new URL( - "http://localhost:9999/exception/exception" - ) - - ) - .go(new InputStreamResponseHandler()); + ListenableFuture go = + client.get(new URL("http://localhost:9999/exception/exception")) + .go(new InputStreamResponseHandler()); try { StringWriter writer = new StringWriter(); IOUtils.copy(go.get(), writer, "utf-8"); From 3933af79a02cf3bf07a4b71aabdb55bf2401054c Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 18 Aug 2014 17:13:06 -0700 Subject: [PATCH 055/107] fix small bug --- processing/src/main/java/io/druid/segment/IndexMaker.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/IndexMaker.java b/processing/src/main/java/io/druid/segment/IndexMaker.java index 33be9558174..06c92785378 100644 --- a/processing/src/main/java/io/druid/segment/IndexMaker.java +++ b/processing/src/main/java/io/druid/segment/IndexMaker.java @@ -579,9 +579,8 @@ public class IndexMaker ); } } - ) - , - Ordering.natural().nullsFirst() + ), + Ordering.natural() ); int cardinality = 0; @@ -1010,7 +1009,7 @@ public class IndexMaker if (nullSet != null) { log.info("Dimension[%s] has null rows.", dimension); - if (Iterables.getFirst(dimensionValues, null) != null) { + if (Iterables.getFirst(dimensionValues, "") != null) { bumpDictionary = true; log.info("Dimension[%s] has no null value in the dictionary, expanding...", dimension); From 0facb4202db43850f7643a905c2898b5c4bf89de Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 18 Aug 2014 17:22:44 -0700 Subject: [PATCH 056/107] cleanup --- processing/src/main/java/io/druid/segment/IndexMaker.java | 1 - 1 file changed, 1 deletion(-) diff --git a/processing/src/main/java/io/druid/segment/IndexMaker.java b/processing/src/main/java/io/druid/segment/IndexMaker.java index 06c92785378..5c24f2a83da 100644 --- a/processing/src/main/java/io/druid/segment/IndexMaker.java +++ b/processing/src/main/java/io/druid/segment/IndexMaker.java @@ -956,7 +956,6 @@ public class IndexMaker public VSizeIndexedInts apply(List input) { if (input == null) { - //return null; return VSizeIndexedInts.fromList( new AbstractList() { From 4fd547955952b6768b240bec1c1e2705b0e922e9 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 19 Aug 2014 12:34:10 -0700 Subject: [PATCH 057/107] fix typo --- .../java/io/druid/indexing/common/task/MergeTaskBaseTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/MergeTaskBaseTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/MergeTaskBaseTest.java index 328818b8e19..7e8dd86f5a0 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/MergeTaskBaseTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/MergeTaskBaseTest.java @@ -46,7 +46,7 @@ public class MergeTaskBaseTest final MergeTaskBase testMergeTaskBase = new MergeTaskBase(null, "foo", segments) { @Override - protected File merge( Map segments, File outDir) throws Exception + protected File merge(Map segments, File outDir) throws Exception { return null; } From 88a904e0b367296b12a0e95db14b0b93e78392b2 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 19 Aug 2014 12:59:01 -0700 Subject: [PATCH 058/107] address cr about progress ind --- .../io/druid/indexer/IndexGeneratorJob.java | 48 ++++++------------- ...icator.java => BaseProgressIndicator.java} | 2 +- .../segment/LoggingProgressIndicator.java | 2 +- 3 files changed, 16 insertions(+), 36 deletions(-) rename processing/src/main/java/io/druid/segment/{AbstractProgressIndicator.java => BaseProgressIndicator.java} (94%) 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 6e0b4d7496b..5e286ed48ba 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -37,9 +37,10 @@ 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.AbstractProgressIndicator; +import io.druid.segment.BaseProgressIndicator; import io.druid.segment.IndexIO; import io.druid.segment.IndexMaker; +import io.druid.segment.ProgressIndicator; import io.druid.segment.QueryableIndex; import io.druid.segment.SegmentUtils; import io.druid.segment.incremental.IncrementalIndex; @@ -295,6 +296,14 @@ public class IndexGeneratorJob implements Jobby long startTime = System.currentTimeMillis(); Set allDimensionNames = Sets.newHashSet(); + final ProgressIndicator progressIndicator = new BaseProgressIndicator() + { + @Override + public void progress() + { + context.progress(); + } + }; for (final Text value : values) { context.progress(); @@ -318,14 +327,7 @@ public class IndexGeneratorJob implements Jobby context.progress(); IndexMaker.persist( - index, interval, file, new AbstractProgressIndicator() - { - @Override - public void progress() - { - context.progress(); - } - } + index, interval, file, progressIndicator ); // close this index and make a new one index.close(); @@ -348,27 +350,13 @@ public class IndexGeneratorJob implements Jobby mergedBase = new File(baseFlushFile, "merged"); IndexMaker.persist( - index, interval, mergedBase, new AbstractProgressIndicator() - { - @Override - public void progress() - { - context.progress(); - } - } + index, interval, mergedBase, progressIndicator ); } else { if (!index.isEmpty()) { final File finalFile = new File(baseFlushFile, "final"); IndexMaker.persist( - index, interval, finalFile, new AbstractProgressIndicator() - { - @Override - public void progress() - { - context.progress(); - } - } + index, interval, finalFile, progressIndicator ); toMerge.add(finalFile); } @@ -377,14 +365,7 @@ public class IndexGeneratorJob implements Jobby indexes.add(IndexIO.loadIndex(file)); } mergedBase = IndexMaker.mergeQueryableIndex( - indexes, aggs, new File(baseFlushFile, "merged"), new AbstractProgressIndicator() - { - @Override - public void progress() - { - context.progress(); - } - } + indexes, aggs, new File(baseFlushFile, "merged"), progressIndicator ); } serializeOutIndex(context, bucket, mergedBase, Lists.newArrayList(allDimensionNames)); @@ -395,7 +376,6 @@ public class IndexGeneratorJob implements Jobby finally { index.close(); } - } private void serializeOutIndex(Context context, Bucket bucket, File mergedBase, List dimensionNames) diff --git a/processing/src/main/java/io/druid/segment/AbstractProgressIndicator.java b/processing/src/main/java/io/druid/segment/BaseProgressIndicator.java similarity index 94% rename from processing/src/main/java/io/druid/segment/AbstractProgressIndicator.java rename to processing/src/main/java/io/druid/segment/BaseProgressIndicator.java index 567f989767f..7992f3a220c 100644 --- a/processing/src/main/java/io/druid/segment/AbstractProgressIndicator.java +++ b/processing/src/main/java/io/druid/segment/BaseProgressIndicator.java @@ -21,7 +21,7 @@ package io.druid.segment; /** */ -public abstract class AbstractProgressIndicator implements ProgressIndicator +public class BaseProgressIndicator implements ProgressIndicator { @Override public void progress() diff --git a/processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java b/processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java index c40aedfbc69..e86716e8fb7 100644 --- a/processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java +++ b/processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java @@ -29,7 +29,7 @@ import java.util.concurrent.TimeUnit; /** */ -public class LoggingProgressIndicator extends AbstractProgressIndicator +public class LoggingProgressIndicator extends BaseProgressIndicator { private static Logger log = new Logger(LoggingProgressIndicator.class); From 0c77f549ff83ff631b59145902e8c9bac3310233 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 20 Aug 2014 11:54:20 +0530 Subject: [PATCH 059/107] Revert "return correct valueType" This reverts commit 0b42ca4c75a451e221bf74d312693f9eead9f945. --- .../java/io/druid/query/aggregation/CountAggregatorFactory.java | 2 +- .../io/druid/query/aggregation/LongSumAggregatorFactory.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java index 5845eda1278..83baa4fd41b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java @@ -123,7 +123,7 @@ public class CountAggregatorFactory implements AggregatorFactory @Override public String getTypeName() { - return valueType; + return "long"; } @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 bf18580ac62..bbac473f5de 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java @@ -139,7 +139,7 @@ public class LongSumAggregatorFactory implements AggregatorFactory @Override public String getTypeName() { - return valueType; + return "long"; } @Override From 60906c324452ef9eda93f3092f58b8eac440994d Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 20 Aug 2014 11:55:26 +0530 Subject: [PATCH 060/107] Revert "make valueType configurable" This reverts commit 6f60a3f60437ddfff2065110d8622ef218fd4b65. --- .../indexing/common/task/TaskSerdeTest.java | 3 +- .../aggregation/CountAggregatorFactory.java | 16 ++---- .../aggregation/LongSumAggregatorFactory.java | 12 ++--- .../ChainedExecutionQueryRunnerTest.java | 4 +- .../java/io/druid/query/DataSourceTest.java | 2 +- .../test/java/io/druid/query/QueriesTest.java | 8 +-- .../io/druid/query/QueryRunnerTestHelper.java | 4 +- .../io/druid/query/RetryQueryRunnerTest.java | 3 +- .../aggregation/CountAggregatorTest.java | 2 +- .../aggregation/LongSumAggregatorTest.java | 2 +- .../query/groupby/GroupByQueryRunnerTest.java | 49 +++++++++---------- .../druid/query/groupby/GroupByQueryTest.java | 4 +- .../TimeSeriesUnionQueryRunnerTest.java | 3 +- .../timeseries/TimeseriesBinaryFnTest.java | 4 +- .../TimeseriesQueryRunnerBonusTest.java | 2 +- .../timeseries/TimeseriesQueryRunnerTest.java | 24 +++------ .../query/topn/TopNBinaryFnBenchmark.java | 6 +-- .../io/druid/query/topn/TopNBinaryFnTest.java | 4 +- .../java/io/druid/segment/AppendTest.java | 6 +-- .../io/druid/segment/SchemalessIndex.java | 4 +- .../io/druid/segment/SchemalessTestFull.java | 2 +- .../druid/segment/SchemalessTestSimple.java | 2 +- .../segment/data/IncrementalIndexTest.java | 2 +- .../filter/SpatialFilterBonusTest.java | 13 ++--- .../segment/filter/SpatialFilterTest.java | 13 ++--- .../IncrementalIndexStorageAdapterTest.java | 16 +++--- .../antlr4/io/druid/sql/antlr4/DruidSQL.g4 | 2 +- .../client/CachingClusteredClientTest.java | 12 ++--- .../druid/client/CachingQueryRunnerTest.java | 6 +-- .../segment/realtime/FireDepartmentTest.java | 2 +- .../segment/realtime/RealtimeManagerTest.java | 2 +- .../plumber/RealtimePlumberSchoolTest.java | 3 +- .../segment/realtime/plumber/SinkTest.java | 2 +- .../router/TieredBrokerHostSelectorTest.java | 14 +++--- 34 files changed, 118 insertions(+), 135 deletions(-) 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 94c76d3c339..daa35c792fb 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 @@ -19,6 +19,7 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; @@ -103,7 +104,7 @@ public class TaskSerdeTest DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build() ), ImmutableList.of( - new CountAggregatorFactory("cnt", null) + new CountAggregatorFactory("cnt") ) ); diff --git a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java index 83baa4fd41b..9549015f687 100644 --- a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java @@ -35,22 +35,16 @@ import java.util.List; public class CountAggregatorFactory implements AggregatorFactory { private static final byte[] CACHE_KEY = new byte[]{0x0}; - private static final String DEFAULT_VALUE_TYPE = "long"; - private static final List supportedTypes = Arrays.asList("float", "long"); private final String name; - private final String valueType; - @JsonCreator public CountAggregatorFactory( - @JsonProperty("name") String name, - @JsonProperty("valueType") String valueType + @JsonProperty("name") String name ) { Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); + this.name = name; - this.valueType = valueType == null ? DEFAULT_VALUE_TYPE : valueType; - Preconditions.checkArgument(supportedTypes.contains(this.valueType)); } @Override @@ -80,13 +74,13 @@ public class CountAggregatorFactory implements AggregatorFactory @Override public AggregatorFactory getCombiningFactory() { - return new LongSumAggregatorFactory(name, name, valueType); + return new LongSumAggregatorFactory(name, name); } @Override public List getRequiredColumns() { - return Arrays.asList(new CountAggregatorFactory(name, valueType)); + return Arrays.asList(new CountAggregatorFactory(name)); } @Override @@ -123,7 +117,7 @@ public class CountAggregatorFactory implements AggregatorFactory @Override public String getTypeName() { - return "long"; + return "float"; } @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 bbac473f5de..acf16e9e7af 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java @@ -35,18 +35,14 @@ import java.util.List; public class LongSumAggregatorFactory implements AggregatorFactory { private static final byte CACHE_TYPE_ID = 0x1; - private static final String DEFAULT_VALUE_TYPE = "long"; - private static final List supportedTypes = Arrays.asList("float", "long"); private final String fieldName; private final String name; - private final String valueType; @JsonCreator public LongSumAggregatorFactory( @JsonProperty("name") String name, - @JsonProperty("fieldName") final String fieldName, - @JsonProperty("valueType") String valueType + @JsonProperty("fieldName") final String fieldName ) { Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); @@ -54,8 +50,6 @@ public class LongSumAggregatorFactory implements AggregatorFactory this.name = name; this.fieldName = fieldName; - this.valueType = valueType == null ? DEFAULT_VALUE_TYPE : valueType; - Preconditions.checkArgument(supportedTypes.contains(this.valueType)); } @Override @@ -88,13 +82,13 @@ public class LongSumAggregatorFactory implements AggregatorFactory @Override public AggregatorFactory getCombiningFactory() { - return new LongSumAggregatorFactory(name, name, valueType); + return new LongSumAggregatorFactory(name, name); } @Override public List getRequiredColumns() { - return Arrays.asList(new LongSumAggregatorFactory(fieldName, fieldName, valueType)); + return Arrays.asList(new LongSumAggregatorFactory(fieldName, fieldName)); } @Override diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index 8a704e635a9..489c36e799f 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -107,7 +107,7 @@ public class ChainedExecutionQueryRunnerTest Druids.newTimeseriesQueryBuilder() .dataSource("test") .intervals("2014/2015") - .aggregators(Lists.newArrayList(new CountAggregatorFactory("count", null))) + .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) .build(), context ); @@ -215,7 +215,7 @@ public class ChainedExecutionQueryRunnerTest Druids.newTimeseriesQueryBuilder() .dataSource("test") .intervals("2014/2015") - .aggregators(Lists.newArrayList(new CountAggregatorFactory("count", null))) + .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) .context(ImmutableMap.of("timeout", 100, "queryId", "test")) .build(), context diff --git a/processing/src/test/java/io/druid/query/DataSourceTest.java b/processing/src/test/java/io/druid/query/DataSourceTest.java index 1c48ff6231f..368ece5634c 100644 --- a/processing/src/test/java/io/druid/query/DataSourceTest.java +++ b/processing/src/test/java/io/druid/query/DataSourceTest.java @@ -73,7 +73,7 @@ public class DataSourceTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index", null) + new LongSumAggregatorFactory("idx", "index") ) ) .setGranularity(QueryRunnerTestHelper.dayGran) diff --git a/processing/src/test/java/io/druid/query/QueriesTest.java b/processing/src/test/java/io/druid/query/QueriesTest.java index d212f660529..763aeb135fd 100644 --- a/processing/src/test/java/io/druid/query/QueriesTest.java +++ b/processing/src/test/java/io/druid/query/QueriesTest.java @@ -40,7 +40,7 @@ public class QueriesTest public void testVerifyAggregations() throws Exception { List aggFactories = Arrays.asList( - new CountAggregatorFactory("count", null), + new CountAggregatorFactory("count"), new DoubleSumAggregatorFactory("idx", "index"), new DoubleSumAggregatorFactory("rev", "revenue") ); @@ -72,7 +72,7 @@ public class QueriesTest public void testVerifyAggregationsMissingVal() throws Exception { List aggFactories = Arrays.asList( - new CountAggregatorFactory("count", null), + new CountAggregatorFactory("count"), new DoubleSumAggregatorFactory("idx", "index"), new DoubleSumAggregatorFactory("rev", "revenue") ); @@ -104,7 +104,7 @@ public class QueriesTest public void testVerifyAggregationsMultiLevel() throws Exception { List aggFactories = Arrays.asList( - new CountAggregatorFactory("count", null), + new CountAggregatorFactory("count"), new DoubleSumAggregatorFactory("idx", "index"), new DoubleSumAggregatorFactory("rev", "revenue") ); @@ -158,7 +158,7 @@ public class QueriesTest public void testVerifyAggregationsMultiLevelMissingVal() throws Exception { List aggFactories = Arrays.asList( - new CountAggregatorFactory("count", null), + new CountAggregatorFactory("count"), new DoubleSumAggregatorFactory("idx", "index"), new DoubleSumAggregatorFactory("rev", "revenue") ); diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index 5592a299b4a..deeaff563e8 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -90,8 +90,8 @@ public class QueryRunnerTestHelper public static final String uniqueMetric = "uniques"; public static final String addRowsIndexConstantMetric = "addRowsIndexConstant"; public static String dependentPostAggMetric = "dependentPostAgg"; - public static final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows", null); - public static final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index", null); + public static final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); + public static final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index"); public static final DoubleSumAggregatorFactory indexDoubleSum = new DoubleSumAggregatorFactory("index", "index"); public static final JavaScriptAggregatorFactory jsIndexSumIfPlacementishA = new JavaScriptAggregatorFactory( "nindex", diff --git a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java index 76e31558674..16c8a51504a 100644 --- a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java @@ -36,8 +36,7 @@ public class RetryQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index", - null + "index" ), QueryRunnerTestHelper.qualityUniques ) diff --git a/processing/src/test/java/io/druid/query/aggregation/CountAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/CountAggregatorTest.java index 24fe633f67c..2ca8584c939 100644 --- a/processing/src/test/java/io/druid/query/aggregation/CountAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/CountAggregatorTest.java @@ -56,7 +56,7 @@ public class CountAggregatorTest Object first = agg.get(); agg.aggregate(); - Comparator comp = new CountAggregatorFactory("null", null).getComparator(); + Comparator comp = new CountAggregatorFactory("null").getComparator(); Assert.assertEquals(-1, comp.compare(first, agg.get())); Assert.assertEquals(0, comp.compare(first, first)); 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 0e57cce691f..046e836248b 100644 --- a/processing/src/test/java/io/druid/query/aggregation/LongSumAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/LongSumAggregatorTest.java @@ -66,7 +66,7 @@ public class LongSumAggregatorTest Object first = agg.get(); agg.aggregate(); - Comparator comp = new LongSumAggregatorFactory("null", "null", null).getComparator(); + Comparator comp = new LongSumAggregatorFactory("null", "null").getComparator(); Assert.assertEquals(-1, comp.compare(first, agg.get())); Assert.assertEquals(0, comp.compare(first, first)); 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 0009173d5b8..4d7f285d13f 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -191,7 +191,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index", null) + new LongSumAggregatorFactory("idx", "index") ) ) .setGranularity(QueryRunnerTestHelper.dayGran) @@ -302,7 +302,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index", null) + new LongSumAggregatorFactory("idx", "index") ) ) .setGranularity(QueryRunnerTestHelper.dayGran) @@ -353,8 +353,7 @@ public class GroupByQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index", - null + "index" ) ) ) @@ -404,7 +403,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index", null) + new LongSumAggregatorFactory("idx", "index") ) ) .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)); @@ -480,7 +479,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index", null) + new LongSumAggregatorFactory("idx", "index") ) ) .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)) @@ -518,7 +517,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index", null) + new LongSumAggregatorFactory("idx", "index") ) ) .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)) @@ -597,7 +596,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index", null) + new LongSumAggregatorFactory("idx", "index") ) ) .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)) @@ -637,7 +636,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index", null) + new LongSumAggregatorFactory("idx", "index") ) ) .addOrderByColumn("rows") @@ -678,7 +677,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index", null) + new LongSumAggregatorFactory("idx", "index") ) ) .addOrderByColumn("rows", "desc") @@ -764,7 +763,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index", null) + new LongSumAggregatorFactory("idx", "index") ) ) .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)) @@ -955,7 +954,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index", null) + new LongSumAggregatorFactory("idx", "index") ) ) .setGranularity(QueryRunnerTestHelper.dayGran) @@ -968,8 +967,8 @@ public class GroupByQueryRunnerTest .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias"))) .setAggregatorSpecs( Arrays.asList( - new LongSumAggregatorFactory("rows", "rows", null), - new LongSumAggregatorFactory("idx", "idx", null) + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx") ) ) .setGranularity(QueryRunnerTestHelper.dayGran) @@ -1013,7 +1012,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index", null) + new LongSumAggregatorFactory("idx", "index") ) ) .setGranularity(QueryRunnerTestHelper.dayGran) @@ -1051,7 +1050,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index", null) + new LongSumAggregatorFactory("idx", "index") ) ) .setGranularity(QueryRunnerTestHelper.dayGran) @@ -1088,7 +1087,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index", null) + new LongSumAggregatorFactory("idx", "index") ) ) .setGranularity(QueryRunnerTestHelper.dayGran) @@ -1122,7 +1121,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx_subagg", "index", null) + new LongSumAggregatorFactory("idx_subagg", "index") ) ) .setPostAggregatorSpecs( @@ -1146,8 +1145,8 @@ public class GroupByQueryRunnerTest .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias"))) .setAggregatorSpecs( Arrays.asList( - new LongSumAggregatorFactory("rows", "rows", null), - new LongSumAggregatorFactory("idx", "idx_subpostagg", null) + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx_subpostagg") ) ) .setPostAggregatorSpecs( @@ -1203,7 +1202,7 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx_subagg", "index", null) + new LongSumAggregatorFactory("idx_subagg", "index") ) ) .setPostAggregatorSpecs( @@ -1246,8 +1245,8 @@ public class GroupByQueryRunnerTest .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias"))) .setAggregatorSpecs( Arrays.asList( - new LongSumAggregatorFactory("rows", "rows", null), - new LongSumAggregatorFactory("idx", "idx_subpostagg", null) + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx_subpostagg") ) ) .setPostAggregatorSpecs( @@ -1351,8 +1350,8 @@ public class GroupByQueryRunnerTest .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias"))) .setAggregatorSpecs( Arrays.asList( - new LongSumAggregatorFactory("rows", "rows", null), - new LongSumAggregatorFactory("idx", "idx_subpostagg", null), + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx_subpostagg"), new DoubleSumAggregatorFactory("js_outer_agg", "js_agg") ) ) diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java index 036f4a5ed9a..8557ba04c24 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java @@ -24,10 +24,12 @@ package io.druid.query.groupby; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.Druids; import io.druid.query.Query; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; import org.junit.Assert; @@ -51,7 +53,7 @@ public class GroupByQueryTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx", "index", null) + new LongSumAggregatorFactory("idx", "index") ) ) .setGranularity(QueryRunnerTestHelper.dayGran) 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 f091134f7d7..8ad6c40dbec 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java @@ -77,8 +77,7 @@ public class TimeSeriesUnionQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index", - null + "index" ), QueryRunnerTestHelper.qualityUniques ) diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java index 90fb7c327d9..ff3266f303c 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java @@ -36,8 +36,8 @@ import java.util.List; */ public class TimeseriesBinaryFnTest { - final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows", null); - final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index", null); + final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); + final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index"); final List aggregatorFactories = Arrays.asList( rowsCount, indexLongSum 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 ae1a6f8e3c5..95d654b9153 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java @@ -109,7 +109,7 @@ public class TimeseriesQueryRunnerBonusTest .intervals(ImmutableList.of(new Interval("2012-01-01T00:00:00Z/P1D"))) .aggregators( ImmutableList.of( - new CountAggregatorFactory("rows", null) + new CountAggregatorFactory("rows") ) ) .build(); 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 88f2fbb51e1..1d1fcb21b00 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -260,8 +260,7 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index", - null + "index" ), QueryRunnerTestHelper.qualityUniques ) @@ -302,8 +301,7 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index", - null + "index" ) ) ) @@ -357,8 +355,7 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index", - null + "index" ), QueryRunnerTestHelper.qualityUniques ) @@ -395,8 +392,7 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index", - null + "index" ), QueryRunnerTestHelper.qualityUniques ) @@ -444,8 +440,7 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index", - null + "index" ) ) ) @@ -492,8 +487,7 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index", - null + "index" ), QueryRunnerTestHelper.qualityUniques ) @@ -531,8 +525,7 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index", - null + "index" ), QueryRunnerTestHelper.qualityUniques ) @@ -575,8 +568,7 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory( "idx", - "index", - null + "index" ) ) ) diff --git a/processing/src/test/java/io/druid/query/topn/TopNBinaryFnBenchmark.java b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnBenchmark.java index eb97d72e4c5..65ab16ad19b 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNBinaryFnBenchmark.java +++ b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnBenchmark.java @@ -67,10 +67,10 @@ public class TopNBinaryFnBenchmark extends SimpleBenchmark final List aggregatorFactories = new ArrayList<>(); - aggregatorFactories.add(new CountAggregatorFactory("rows", null)); - aggregatorFactories.add(new LongSumAggregatorFactory("index", "index", null)); + aggregatorFactories.add(new CountAggregatorFactory("rows")); + aggregatorFactories.add(new LongSumAggregatorFactory("index", "index")); for (int i = 1; i < aggCount; i++) { - aggregatorFactories.add(new CountAggregatorFactory("rows" + i, null)); + aggregatorFactories.add(new CountAggregatorFactory("rows" + i)); } final List postAggregators = new ArrayList<>(); for (int i = 0; i < postAggCount; i++) { diff --git a/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java index 0b8d0c93c08..c8958dc6580 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java @@ -45,8 +45,8 @@ import java.util.Map; */ public class TopNBinaryFnTest { - final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows", null); - final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index", null); + final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); + final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index"); final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null); final FieldAccessPostAggregator rowsPostAgg = new FieldAccessPostAggregator("rows", "rows"); final FieldAccessPostAggregator indexPostAgg = new FieldAccessPostAggregator("index", "index"); diff --git a/processing/src/test/java/io/druid/segment/AppendTest.java b/processing/src/test/java/io/druid/segment/AppendTest.java index e32d5a11410..017f99c293e 100644 --- a/processing/src/test/java/io/druid/segment/AppendTest.java +++ b/processing/src/test/java/io/druid/segment/AppendTest.java @@ -69,12 +69,12 @@ public class AppendTest { private static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ new DoubleSumAggregatorFactory("index", "index"), - new CountAggregatorFactory("count", null), + new CountAggregatorFactory("count"), new HyperUniquesAggregatorFactory("quality_uniques", "quality") }; private static final AggregatorFactory[] METRIC_AGGS_NO_UNIQ = new AggregatorFactory[]{ new DoubleSumAggregatorFactory("index", "index"), - new CountAggregatorFactory("count", null) + new CountAggregatorFactory("count") }; final String dataSource = "testing"; @@ -86,7 +86,7 @@ public class AppendTest final String placementDimension = "placement"; final String placementishDimension = "placementish"; final String indexMetric = "index"; - final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows", null); + final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); final DoubleSumAggregatorFactory indexDoubleSum = new DoubleSumAggregatorFactory("index", "index"); final HyperUniquesAggregatorFactory uniques = new HyperUniquesAggregatorFactory("uniques", "quality_uniques"); final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null); diff --git a/processing/src/test/java/io/druid/segment/SchemalessIndex.java b/processing/src/test/java/io/druid/segment/SchemalessIndex.java index 9c03f107658..823fd83774a 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessIndex.java +++ b/processing/src/test/java/io/druid/segment/SchemalessIndex.java @@ -68,12 +68,12 @@ public class SchemalessIndex private static final List METRICS = Arrays.asList("index"); private static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ new DoubleSumAggregatorFactory("index", "index"), - new CountAggregatorFactory("count", null), + new CountAggregatorFactory("count"), new HyperUniquesAggregatorFactory("quality_uniques", "quality") }; private static final AggregatorFactory[] METRIC_AGGS_NO_UNIQ = new AggregatorFactory[]{ new DoubleSumAggregatorFactory("index", "index"), - new CountAggregatorFactory("count", null) + new CountAggregatorFactory("count") }; private static final List> events = Lists.newArrayList(); diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java index 1f3dbd3af24..58a391c7515 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java @@ -76,7 +76,7 @@ public class SchemalessTestFull final String placementDimension = "placement"; final String placementishDimension = "placementish"; final String indexMetric = "index"; - final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows", null); + final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); final DoubleSumAggregatorFactory indexDoubleSum = new DoubleSumAggregatorFactory("index", "index"); final HyperUniquesAggregatorFactory uniques = new HyperUniquesAggregatorFactory("uniques", "quality_uniques"); final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null); diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java b/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java index 14909e3a5fe..153a1226619 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java @@ -103,7 +103,7 @@ public class SchemalessTestSimple final String placementDimension = "placement"; final String placementishDimension = "placementish"; final String indexMetric = "index"; - final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows", null); + final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); final DoubleSumAggregatorFactory indexDoubleSum = new DoubleSumAggregatorFactory("index", "index"); final HyperUniquesAggregatorFactory uniques = new HyperUniquesAggregatorFactory("uniques", "quality_uniques"); final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null); 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 4bf0638f0fd..7a131901501 100644 --- a/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java @@ -109,7 +109,7 @@ public class IncrementalIndexTest final IncrementalIndex index = new IncrementalIndex( 0L, QueryGranularity.NONE, - new AggregatorFactory[]{new CountAggregatorFactory("count", null)}, + new AggregatorFactory[]{new CountAggregatorFactory("count")}, TestQueryRunners.pool ); final int threadCount = 10; 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 6992c531105..12c3ca633f1 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java @@ -51,6 +51,7 @@ import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; import io.druid.segment.TestHelper; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import org.joda.time.DateTime; @@ -74,8 +75,8 @@ 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", null), - new LongSumAggregatorFactory("val", "val", null) + new CountAggregatorFactory("rows"), + new LongSumAggregatorFactory("val", "val") }; private static List DIMS = Lists.newArrayList("dim", "dim.geo"); private final Segment segment; @@ -444,8 +445,8 @@ public class SpatialFilterBonusTest ) .aggregators( Arrays.asList( - new CountAggregatorFactory("rows", null), - new LongSumAggregatorFactory("val", "val", null) + new CountAggregatorFactory("rows"), + new LongSumAggregatorFactory("val", "val") ) ) .build(); @@ -495,8 +496,8 @@ public class SpatialFilterBonusTest ) .aggregators( Arrays.asList( - new CountAggregatorFactory("rows", null), - new LongSumAggregatorFactory("val", "val", null) + new CountAggregatorFactory("rows"), + new LongSumAggregatorFactory("val", "val") ) ) .build(); 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 d9b815a3235..f9693bdabf8 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java @@ -51,6 +51,7 @@ import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; import io.druid.segment.TestHelper; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import org.joda.time.DateTime; @@ -74,8 +75,8 @@ public class SpatialFilterTest { private static Interval DATA_INTERVAL = new Interval("2013-01-01/2013-01-07"); private static AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ - new CountAggregatorFactory("rows", null), - new LongSumAggregatorFactory("val", "val", null) + new CountAggregatorFactory("rows"), + new LongSumAggregatorFactory("val", "val") }; private static List DIMS = Lists.newArrayList("dim", "lat", "long"); private final Segment segment; @@ -472,8 +473,8 @@ public class SpatialFilterTest ) .aggregators( Arrays.asList( - new CountAggregatorFactory("rows", null), - new LongSumAggregatorFactory("val", "val", null) + new CountAggregatorFactory("rows"), + new LongSumAggregatorFactory("val", "val") ) ) .build(); @@ -523,8 +524,8 @@ public class SpatialFilterTest ) .aggregators( Arrays.asList( - new CountAggregatorFactory("rows", null), - new LongSumAggregatorFactory("val", "val", null) + new CountAggregatorFactory("rows"), + new LongSumAggregatorFactory("val", "val") ) ) .build(); 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 8fd1446a8d3..1eee18fe782 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -53,6 +53,7 @@ import org.junit.Test; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.List; /** */ @@ -62,7 +63,7 @@ public class IncrementalIndexStorageAdapterTest public void testSanity() throws Exception { IncrementalIndex index = new IncrementalIndex( - 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt", null)}, + 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, TestQueryRunners.pool ); @@ -109,7 +110,7 @@ public class IncrementalIndexStorageAdapterTest .setInterval(new Interval(0, new DateTime().getMillis())) .addDimension("billy") .addDimension("sally") - .addAggregator(new LongSumAggregatorFactory("cnt", "cnt", null)) + .addAggregator(new LongSumAggregatorFactory("cnt", "cnt")) .build(), new IncrementalIndexStorageAdapter(index) ); @@ -128,7 +129,7 @@ public class IncrementalIndexStorageAdapterTest @Test public void testResetSanity() { IncrementalIndex index = new IncrementalIndex( - 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt", null)}, + 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, TestQueryRunners.pool ); @@ -181,7 +182,7 @@ public class IncrementalIndexStorageAdapterTest public void testSingleValueTopN() { IncrementalIndex index = new IncrementalIndex( - 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt", null)}, + 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, TestQueryRunners.pool ); @@ -219,8 +220,7 @@ public class IncrementalIndexStorageAdapterTest Lists.newArrayList( new LongSumAggregatorFactory( "cnt", - "cnt", - null + "cnt" ) ) ) @@ -238,7 +238,7 @@ public class IncrementalIndexStorageAdapterTest public void testFilterByNull() throws Exception { IncrementalIndex index = new IncrementalIndex( - 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt", null)}, + 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, TestQueryRunners.pool ); @@ -285,7 +285,7 @@ public class IncrementalIndexStorageAdapterTest .setInterval(new Interval(0, new DateTime().getMillis())) .addDimension("billy") .addDimension("sally") - .addAggregator(new LongSumAggregatorFactory("cnt", "cnt", null)) + .addAggregator(new LongSumAggregatorFactory("cnt", "cnt")) .setDimFilter(DimFilters.dimEquals("sally", (String) null)) .build(), new IncrementalIndexStorageAdapter(index) diff --git a/server/src/main/antlr4/io/druid/sql/antlr4/DruidSQL.g4 b/server/src/main/antlr4/io/druid/sql/antlr4/DruidSQL.g4 index ed810051070..37716ab6673 100644 --- a/server/src/main/antlr4/io/druid/sql/antlr4/DruidSQL.g4 +++ b/server/src/main/antlr4/io/druid/sql/antlr4/DruidSQL.g4 @@ -72,7 +72,7 @@ import java.util.Map; case SUM: return new DoubleSumAggregatorFactory("sum("+name+")", name); case MIN: return new MinAggregatorFactory("min("+name+")", name); case MAX: return new MaxAggregatorFactory("max("+name+")", name); - case COUNT: return new CountAggregatorFactory(name, null); + case COUNT: return new CountAggregatorFactory(name); } throw new IllegalArgumentException("Unknown function [" + fn + "]"); } diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index edd1e307179..0834a0d9f8b 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -150,9 +150,9 @@ public class CachingClusteredClientTest */ private static final int RANDOMNESS = 10; private static final List AGGS = Arrays.asList( - new CountAggregatorFactory("rows", null), - new LongSumAggregatorFactory("imps", "imps", null), - new LongSumAggregatorFactory("impers", "imps", null) + new CountAggregatorFactory("rows"), + new LongSumAggregatorFactory("imps", "imps"), + new LongSumAggregatorFactory("impers", "imps") ); private static final List POST_AGGS = Arrays.asList( new ArithmeticPostAggregator( @@ -181,9 +181,9 @@ public class CachingClusteredClientTest ) ); private static final List RENAMED_AGGS = Arrays.asList( - new CountAggregatorFactory("rows2", null), - new LongSumAggregatorFactory("imps", "imps", null), - new LongSumAggregatorFactory("impers2", "imps", null) + new CountAggregatorFactory("rows2"), + new LongSumAggregatorFactory("imps", "imps"), + new LongSumAggregatorFactory("impers2", "imps") ); private static final DimFilter DIM_FILTER = null; private static final List RENAMED_POST_AGGS = Arrays.asList(); diff --git a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java index c64996e6bc3..a358ec44798 100644 --- a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java @@ -66,9 +66,9 @@ public class CachingQueryRunnerTest { private static final List AGGS = Arrays.asList( - new CountAggregatorFactory("rows", null), - new LongSumAggregatorFactory("imps", "imps", null), - new LongSumAggregatorFactory("impers", "imps", null) + new CountAggregatorFactory("rows"), + new LongSumAggregatorFactory("imps", "imps"), + new LongSumAggregatorFactory("impers", "imps") ); private static final Object[] objects = new Object[]{ 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 f812c24b974..5fca643fc08 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -66,7 +66,7 @@ public class FireDepartmentTest null, null, null, null ), new AggregatorFactory[]{ - new CountAggregatorFactory("count", null) + new CountAggregatorFactory("count") }, new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.MINUTE, null, Granularity.HOUR) ), diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index 3142efa29e8..cb464a9ff8f 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -73,7 +73,7 @@ public class RealtimeManagerTest schema = new DataSchema( "test", null, - new AggregatorFactory[]{new CountAggregatorFactory("rows", null)}, + new AggregatorFactory[]{new CountAggregatorFactory("rows")}, new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.NONE, null, Granularity.HOUR) ); RealtimeIOConfig ioConfig = new RealtimeIOConfig( 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 9a57b78c462..b580e83d3c5 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 @@ -41,6 +41,7 @@ import io.druid.query.Query; import io.druid.query.QueryRunnerFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -101,7 +102,7 @@ public class RealtimePlumberSchoolTest return null; } }, - new AggregatorFactory[]{new CountAggregatorFactory("rows", null)}, + new AggregatorFactory[]{new CountAggregatorFactory("rows")}, new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.NONE, null, Granularity.HOUR) ); diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index c2cc0d9517e..0ebb59d4410 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -49,7 +49,7 @@ public class SinkTest final DataSchema schema = new DataSchema( "test", null, - new AggregatorFactory[]{new CountAggregatorFactory("rows", null)}, + new AggregatorFactory[]{new CountAggregatorFactory("rows")}, new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.MINUTE, null, Granularity.HOUR) ); diff --git a/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java b/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java index 0004679b79a..5c2320f9885 100644 --- a/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java +++ b/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java @@ -113,7 +113,7 @@ public class TieredBrokerHostSelectorTest Druids.newTimeseriesQueryBuilder() .dataSource("test") .granularity("all") - .aggregators(Arrays.asList(new CountAggregatorFactory("rows", null))) + .aggregators(Arrays.asList(new CountAggregatorFactory("rows"))) .intervals(Arrays.asList(new Interval("2011-08-31/2011-09-01"))) .build() ).lhs; @@ -129,7 +129,7 @@ public class TieredBrokerHostSelectorTest Druids.newTimeseriesQueryBuilder() .dataSource("test") .granularity("all") - .aggregators(Arrays.asList(new CountAggregatorFactory("rows", null))) + .aggregators(Arrays.asList(new CountAggregatorFactory("rows"))) .intervals(Arrays.asList(new Interval("2013-08-31/2013-09-01"))) .build() ).lhs; @@ -144,7 +144,7 @@ public class TieredBrokerHostSelectorTest Druids.newTimeseriesQueryBuilder() .dataSource("test") .granularity("all") - .aggregators(Arrays.asList(new CountAggregatorFactory("rows", null))) + .aggregators(Arrays.asList(new CountAggregatorFactory("rows"))) .intervals(Arrays.asList(new Interval("2010-08-31/2010-09-01"))) .build() ).lhs; @@ -158,7 +158,7 @@ public class TieredBrokerHostSelectorTest String brokerName = (String) brokerSelector.select( Druids.newTimeseriesQueryBuilder() .dataSource("test") - .aggregators(Arrays.asList(new CountAggregatorFactory("count", null))) + .aggregators(Arrays.asList(new CountAggregatorFactory("count"))) .intervals( new MultipleIntervalSegmentSpec( Arrays.asList( @@ -179,7 +179,7 @@ public class TieredBrokerHostSelectorTest String brokerName = (String) brokerSelector.select( Druids.newTimeseriesQueryBuilder() .dataSource("test") - .aggregators(Arrays.asList(new CountAggregatorFactory("count", null))) + .aggregators(Arrays.asList(new CountAggregatorFactory("count"))) .intervals( new MultipleIntervalSegmentSpec( Arrays.asList( @@ -200,7 +200,7 @@ public class TieredBrokerHostSelectorTest String brokerName = (String) brokerSelector.select( Druids.newTimeseriesQueryBuilder() .dataSource("test") - .aggregators(Arrays.asList(new CountAggregatorFactory("count", null))) + .aggregators(Arrays.asList(new CountAggregatorFactory("count"))) .intervals( new MultipleIntervalSegmentSpec( Arrays.asList( @@ -223,7 +223,7 @@ public class TieredBrokerHostSelectorTest String brokerName = (String) brokerSelector.select( Druids.newTimeseriesQueryBuilder() .dataSource("test") - .aggregators(Arrays.asList(new CountAggregatorFactory("count", null))) + .aggregators(Arrays.asList(new CountAggregatorFactory("count"))) .intervals( new MultipleIntervalSegmentSpec( Arrays.asList( From e525562767edcd45863a350c34372e6446615762 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 20 Aug 2014 15:04:43 +0530 Subject: [PATCH 061/107] review comments - cleanup ColumnSelectorFactory --- .../IngestSegmentFirehoseFactory.java | 3 +- .../druid/query/metadata/SegmentAnalyzer.java | 2 +- .../druid/query/select/SelectQueryEngine.java | 3 +- .../java/io/druid/segment/ColumnSelector.java | 1 - .../ColumnSelectorBitmapIndexSelector.java | 2 +- .../druid/segment/ColumnSelectorFactory.java | 1 - .../main/java/io/druid/segment/IndexIO.java | 13 +++---- .../QueryableIndexIndexableAdapter.java | 2 +- .../segment/QueryableIndexStorageAdapter.java | 34 +++---------------- .../druid/segment/SimpleQueryableIndex.java | 21 ++++-------- .../java/io/druid/segment/column/Column.java | 1 + .../segment/incremental/IncrementalIndex.java | 24 ++++++------- .../IncrementalIndexStorageAdapter.java | 25 +++++++------- .../java/io/druid/segment/EmptyIndexTest.java | 3 +- .../java/io/druid/segment/IndexMakerTest.java | 15 ++++---- 15 files changed, 59 insertions(+), 91 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index dfb6f4c2a2d..4e668905f20 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 @@ -55,6 +55,7 @@ import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.QueryableIndexStorageAdapter; import io.druid.segment.StorageAdapter; +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(@Nullable final Cursor cursor) { - final LongColumnSelector timestampColumnSelector = cursor.makeTimestampColumnSelector(); + final LongColumnSelector timestampColumnSelector = cursor.makeLongColumnSelector(Column.TIME_COLUMN_NAME); final Map dimSelectors = Maps.newHashMap(); for (String dim : dims) { 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/select/SelectQueryEngine.java b/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java index 9e51ca2bd46..b5482fa6a11 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java @@ -32,6 +32,7 @@ import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; +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 LongColumnSelector timestampColumnSelector = cursor.makeTimestampColumnSelector(); + final LongColumnSelector timestampColumnSelector = cursor.makeLongColumnSelector(Column.TIME_COLUMN_NAME); final Map dimSelectors = Maps.newHashMap(); for (String dim : dims) { 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 c7c6a54289e..48613d16903 100644 --- a/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java +++ b/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java @@ -24,7 +24,6 @@ package io.druid.segment; */ public interface ColumnSelectorFactory { - public LongColumnSelector makeTimestampColumnSelector(); public DimensionSelector makeDimensionSelector(String dimensionName); public FloatColumnSelector makeFloatColumnSelector(String columnName); public LongColumnSelector makeLongColumnSelector(String columnName); diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index b4163aa6aed..dca9723b6c8 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -715,15 +715,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), index.getAvailableDimensions(), - new ColumnBuilder() - .setType(ValueType.LONG) - .setGenericColumn(new LongGenericColumnSupplier(index.timestamps)) - .build(), columns, index.getFileMapper() ); @@ -756,8 +755,10 @@ public class IndexIO 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/QueryableIndexIndexableAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java index 7e082eea805..6b20722360f 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java @@ -171,7 +171,7 @@ 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; diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index 6e3386e36af..aa0299c90d8 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 { @@ -192,7 +192,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( @@ -255,19 +255,6 @@ public class QueryableIndexStorageAdapter implements StorageAdapter cursorOffset = initOffset.clone(); } - @Override - public LongColumnSelector makeTimestampColumnSelector() - { - return new LongColumnSelector() - { - @Override - public long get() - { - return timestamps.getLongSingleValueRow(cursorOffset.getOffset()); - } - }; - } - @Override public DimensionSelector makeDimensionSelector(String dimension) { @@ -690,7 +677,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( @@ -746,19 +733,6 @@ public class QueryableIndexStorageAdapter implements StorageAdapter currRow = initRow; } - @Override - public LongColumnSelector makeTimestampColumnSelector() - { - return new LongColumnSelector() - { - @Override - public long get() - { - return timestamps.getLongSingleValueRow(currRow); - } - }; - } - @Override public DimensionSelector makeDimensionSelector(String dimension) { 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..4e6fc10709f 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 String TIME_COLUMN_NAME = "__time"; public ColumnCapabilities getCapabilities(); public int getLength(); 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 b2f05b51235..54d73feddce 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -48,6 +48,7 @@ import io.druid.segment.DimensionSelector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; +import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ValueType; @@ -133,22 +134,19 @@ public class IncrementalIndex implements Iterable, Closeable aggs[i] = agg.factorizeBuffered( new ColumnSelectorFactory() { - @Override - public LongColumnSelector makeTimestampColumnSelector() - { - return new LongColumnSelector() - { - @Override - public long get() - { - return in.get().getTimestampFromEpoch(); - } - }; - } - @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() { 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 142d634b628..cc33d21c11f 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -41,6 +41,7 @@ import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.StorageAdapter; +import io.druid.segment.column.Column; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.ListIndexed; @@ -261,19 +262,6 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter done = !foundMatched && (cursorMap.size() == 0 || !baseIter.hasNext()); } - @Override - public LongColumnSelector makeTimestampColumnSelector() - { - return new LongColumnSelector() - { - @Override - public long get() - { - return currEntry.getKey().getTimestamp(); - } - }; - } - @Override public DimensionSelector makeDimensionSelector(String dimension) { @@ -382,6 +370,17 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter 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() diff --git a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java index 486f7b492b5..10b58340c98 100644 --- a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java +++ b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java @@ -24,6 +24,7 @@ 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.column.ColumnConfig; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexAdapter; @@ -60,6 +61,6 @@ public class EmptyIndexTest 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/IndexMakerTest.java b/processing/src/test/java/io/druid/segment/IndexMakerTest.java index 96eedeb0284..bfa1fa0e1e5 100644 --- a/processing/src/test/java/io/druid/segment/IndexMakerTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMakerTest.java @@ -27,6 +27,7 @@ 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.column.ColumnConfig; import io.druid.segment.data.IncrementalIndexTest; import io.druid.segment.incremental.IncrementalIndex; @@ -52,7 +53,7 @@ public class IndexMakerTest try { 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()); } @@ -91,13 +92,13 @@ public class IndexMakerTest try { 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(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()); @@ -109,7 +110,7 @@ public class IndexMakerTest ) ); - 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()); } @@ -152,13 +153,13 @@ public class IndexMakerTest 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 { From 33354cf7fe4b18b1c69c3177b06c079afef2e284 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 20 Aug 2014 20:59:44 +0530 Subject: [PATCH 062/107] replace maxRowsInMemory with BufferSize --- docs/content/Realtime-ingestion.md | 8 +- docs/content/Tasks.md | 2 +- .../examples/rand/RandomFirehoseFactory.java | 2 +- .../io/druid/indexer/HadoopTuningConfig.java | 20 ++-- .../io/druid/indexer/IndexGeneratorJob.java | 9 +- .../common/task/RealtimeIndexTask.java | 92 +++++++++---------- .../segment/incremental/IncrementalIndex.java | 9 ++ .../indexing/RealtimeTuningConfig.java | 18 ++-- .../segment/realtime/RealtimeManager.java | 4 +- .../segment/realtime/plumber/Plumber.java | 2 + .../druid/segment/realtime/plumber/Sink.java | 14 +-- .../segment/realtime/RealtimeManagerTest.java | 14 +-- 12 files changed, 102 insertions(+), 92 deletions(-) diff --git a/docs/content/Realtime-ingestion.md b/docs/content/Realtime-ingestion.md index 9849946f0a8..a2aa1854fec 100644 --- a/docs/content/Realtime-ingestion.md +++ b/docs/content/Realtime-ingestion.md @@ -41,7 +41,7 @@ The property `druid.realtime.specFile` has the path of a file (absolute or relat } }, "config": { - "maxRowsInMemory": 500000, + "bufferSize": 500000000, "intermediatePersistPeriod": "PT10m" }, "firehose": { @@ -104,7 +104,7 @@ This provides configuration for the data processing portion of the realtime stre |Field|Type|Description|Required| |-----|----|-----------|--------| |intermediatePersistPeriod|ISO8601 Period String|The period that determines the rate at which intermediate persists occur. These persists determine how often commits happen against the incoming realtime stream. If the realtime data loading process is interrupted at time T, it should be restarted to re-read data that arrived at T minus this period.|yes| -|maxRowsInMemory|Number|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size.|yes| +|bufferSize|Number|The size in bytes of buffer to be used for ingestion. When the buffer is full intermediate rows will be persisted to disk.|yes| ### Firehose @@ -132,8 +132,8 @@ The following table summarizes constraints between settings in the spec file for | windowPeriod| when reading an InputRow, events with timestamp older than now minus this window are discarded | time jitter tolerance | use this to reject outliers | | segmentGranularity| time granularity (minute, hour, day, week, month) for loading data at query time | equal to indexGranularity| more than indexGranularity| | indexGranularity| time granularity (minute, hour, day, week, month) of indexes | less than segmentGranularity| minute, hour, day, week, month | -| intermediatePersistPeriod| the max real time (ISO8601 Period) between flushes of InputRows from memory to disk | avoid excessive flushing | number of un-persisted rows in memory also constrained by maxRowsInMemory | -| maxRowsInMemory| the max number of InputRows to hold in memory before a flush to disk | number of un-persisted post-aggregation rows in memory is also constrained by intermediatePersistPeriod | use this to avoid running out of heap if too many rows in an intermediatePersistPeriod | +| intermediatePersistPeriod| the max real time (ISO8601 Period) between flushes of InputRows from memory to disk | avoid excessive flushing | size of un-persisted rows in memory also constrained by bufferSize | +| bufferSize| size of offheap buffer to be used to hold Input Rows before a flush to disk | The normal, expected use cases have the following overall constraints: `indexGranularity < intermediatePersistPeriod =< windowPeriod < segmentGranularity` diff --git a/docs/content/Tasks.md b/docs/content/Tasks.md index 6c5df35d25f..bce0f549e1a 100644 --- a/docs/content/Tasks.md +++ b/docs/content/Tasks.md @@ -57,7 +57,7 @@ The Index Task is a simpler variation of the Index Hadoop task that is designed |indexGranularity|The rollup granularity for timestamps. See [Realtime Ingestion](Realtime-ingestion.html) for more information. |no| |targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|no| |firehose|The input source of data. For more info, see [Firehose](Firehose.html).|yes| -|rowFlushBoundary|Used in determining when intermediate persist should occur to disk.|no| +|bufferSize|Used in determining the size of offheap buffer to be used to store intermediate rows. When the buffer gets full, rows are persisted to disk.|no| ### Index Hadoop Task diff --git a/examples/src/main/java/io/druid/examples/rand/RandomFirehoseFactory.java b/examples/src/main/java/io/druid/examples/rand/RandomFirehoseFactory.java index e395cf02dcb..023b6ed6681 100644 --- a/examples/src/main/java/io/druid/examples/rand/RandomFirehoseFactory.java +++ b/examples/src/main/java/io/druid/examples/rand/RandomFirehoseFactory.java @@ -65,7 +65,7 @@ import static java.lang.Thread.sleep; * {"type":"doubleSum","name":"outColumn","fieldName":"inColumn"} ], * "indexGranularity":"minute", * "shardSpec" : { "type": "none" } }, - * "config" : { "maxRowsInMemory" : 50000, + * "config" : { "bufferSize" : 50000000, * "intermediatePersistPeriod" : "PT2m" }, * * "firehose" : { "type" : "rand", diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java index b8f9fb7fe5c..a93c7653964 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java @@ -38,7 +38,7 @@ public class HadoopTuningConfig implements TuningConfig { private static final PartitionsSpec defaultPartitionsSpec = HashedPartitionsSpec.makeDefaultHashedPartitionsSpec(); private static final Map> defaultShardSpecs = ImmutableMap.>of(); - private static final int defaultRowFlushBoundary = 80000; + private static final int defaultBufferSize = 256 * 1024 * 1024; public static HadoopTuningConfig makeDefaultTuningConfig() { @@ -47,7 +47,7 @@ public class HadoopTuningConfig implements TuningConfig new DateTime().toString(), defaultPartitionsSpec, defaultShardSpecs, - defaultRowFlushBoundary, + defaultBufferSize, false, true, false, @@ -61,7 +61,7 @@ public class HadoopTuningConfig implements TuningConfig private final String version; private final PartitionsSpec partitionsSpec; private final Map> shardSpecs; - private final int rowFlushBoundary; + private final int bufferSize; private final boolean leaveIntermediate; private final Boolean cleanupOnFailure; private final boolean overwriteFiles; @@ -75,7 +75,7 @@ public class HadoopTuningConfig implements TuningConfig final @JsonProperty("version") String version, final @JsonProperty("partitionsSpec") PartitionsSpec partitionsSpec, final @JsonProperty("shardSpecs") Map> shardSpecs, - final @JsonProperty("rowFlushBoundary") Integer rowFlushBoundary, + final @JsonProperty("bufferSize") Integer bufferSize, final @JsonProperty("leaveIntermediate") boolean leaveIntermediate, final @JsonProperty("cleanupOnFailure") Boolean cleanupOnFailure, final @JsonProperty("overwriteFiles") boolean overwriteFiles, @@ -88,7 +88,7 @@ public class HadoopTuningConfig implements TuningConfig this.version = version == null ? new DateTime().toString() : version; this.partitionsSpec = partitionsSpec == null ? defaultPartitionsSpec : partitionsSpec; this.shardSpecs = shardSpecs == null ? defaultShardSpecs : shardSpecs; - this.rowFlushBoundary = rowFlushBoundary == null ? defaultRowFlushBoundary : rowFlushBoundary; + this.bufferSize = bufferSize == null ? defaultBufferSize : bufferSize; this.leaveIntermediate = leaveIntermediate; this.cleanupOnFailure = cleanupOnFailure == null ? true : cleanupOnFailure; this.overwriteFiles = overwriteFiles; @@ -124,9 +124,9 @@ public class HadoopTuningConfig implements TuningConfig } @JsonProperty - public int getRowFlushBoundary() + public int getBufferSize() { - return rowFlushBoundary; + return bufferSize; } @JsonProperty @@ -172,7 +172,7 @@ public class HadoopTuningConfig implements TuningConfig version, partitionsSpec, shardSpecs, - rowFlushBoundary, + bufferSize, leaveIntermediate, cleanupOnFailure, overwriteFiles, @@ -189,7 +189,7 @@ public class HadoopTuningConfig implements TuningConfig ver, partitionsSpec, shardSpecs, - rowFlushBoundary, + bufferSize, leaveIntermediate, cleanupOnFailure, overwriteFiles, @@ -206,7 +206,7 @@ public class HadoopTuningConfig implements TuningConfig version, partitionsSpec, specs, - rowFlushBoundary, + bufferSize, leaveIntermediate, cleanupOnFailure, overwriteFiles, 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 5e286ed48ba..1fda38cccf1 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -313,7 +313,7 @@ public class IndexGeneratorJob implements Jobby int numRows = index.add(inputRow); ++lineCount; - if (numRows >= config.getSchema().getTuningConfig().getRowFlushBoundary()) { + if (index.isFull()) { log.info( "%,d lines to %,d rows in %,d millis", lineCount - runningTotalLineCount, @@ -602,11 +602,6 @@ public class IndexGeneratorJob implements Jobby private IncrementalIndex makeIncrementalIndex(Bucket theBucket, AggregatorFactory[] aggs) { - int aggsSize = 0; - for (AggregatorFactory agg : aggs) { - aggsSize += agg.getMaxIntermediateSize(); - } - int bufferSize = aggsSize * config.getSchema().getTuningConfig().getRowFlushBoundary(); return new IncrementalIndex( new IncrementalIndexSchema.Builder() .withMinTimestamp(theBucket.time.getMillis()) @@ -614,7 +609,7 @@ public class IndexGeneratorJob implements Jobby .withQueryGranularity(config.getSchema().getDataSchema().getGranularitySpec().getQueryGranularity()) .withMetrics(aggs) .build(), - new OffheapBufferPool(bufferSize) + new OffheapBufferPool(config.getSchema().getTuningConfig().getBufferSize()) ); } 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 37055fb4af9..a1b737da7e1 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 @@ -19,7 +19,6 @@ package io.druid.indexing.common.task; -import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; @@ -44,7 +43,6 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QueryToolChest; -import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeIOConfig; import io.druid.segment.indexing.RealtimeTuningConfig; @@ -57,6 +55,7 @@ import io.druid.segment.realtime.SegmentPublisher; import io.druid.segment.realtime.plumber.Plumber; import io.druid.segment.realtime.plumber.RealtimePlumberSchool; import io.druid.segment.realtime.plumber.RejectionPolicyFactory; +import io.druid.segment.realtime.plumber.Sink; import io.druid.segment.realtime.plumber.VersioningPolicy; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.timeline.DataSegment; @@ -70,36 +69,10 @@ import java.io.IOException; public class RealtimeIndexTask extends AbstractTask { private static final EmittingLogger log = new EmittingLogger(RealtimeIndexTask.class); - - private static String makeTaskId(FireDepartment fireDepartment, Schema schema) - { - // Backwards compatible - if (fireDepartment == null) { - return String.format( - "index_realtime_%s_%d_%s", - schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime().toString() - ); - } else { - return String.format( - "index_realtime_%s_%d_%s", - fireDepartment.getDataSchema().getDataSource(), - fireDepartment.getTuningConfig().getShardSpec().getPartitionNum(), - new DateTime().toString() - ); - } - } - - private static String makeDatasource(FireDepartment fireDepartment, Schema schema) - { - return (fireDepartment != null) ? fireDepartment.getDataSchema().getDataSource() : schema.getDataSource(); - } - @JsonIgnore private final FireDepartment spec; - @JsonIgnore private volatile Plumber plumber = null; - @JsonIgnore private volatile QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate = null; @@ -152,6 +125,29 @@ public class RealtimeIndexTask extends AbstractTask } } + private static String makeTaskId(FireDepartment fireDepartment, Schema schema) + { + // Backwards compatible + if (fireDepartment == null) { + return String.format( + "index_realtime_%s_%d_%s", + schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime().toString() + ); + } else { + return String.format( + "index_realtime_%s_%d_%s", + fireDepartment.getDataSchema().getDataSource(), + fireDepartment.getTuningConfig().getShardSpec().getPartitionNum(), + new DateTime().toString() + ); + } + } + + private static String makeDatasource(FireDepartment fireDepartment, Schema schema) + { + return (fireDepartment != null) ? fireDepartment.getDataSchema().getDataSource() : schema.getDataSource(); + } + @Override public String getType() { @@ -263,29 +259,30 @@ public class RealtimeIndexTask extends AbstractTask // Shouldn't usually happen, since we don't expect people to submit tasks that intersect with the // realtime window, but if they do it can be problematic. If we decide to care, we can use more threads in // the plumber such that waiting for the coordinator doesn't block data processing. - final VersioningPolicy versioningPolicy = new VersioningPolicy() - { - @Override - public String getVersion(final Interval interval) - { - try { - // Side effect: Calling getVersion causes a lock to be acquired - final TaskLock myLock = toolbox.getTaskActionClient() - .submit(new LockAcquireAction(interval)); + final VersioningPolicy versioningPolicy = new + VersioningPolicy() + { + @Override + public String getVersion(final Interval interval) + { + try { + // Side effect: Calling getVersion causes a lock to be acquired + final TaskLock myLock = toolbox.getTaskActionClient() + .submit(new LockAcquireAction(interval)); - return myLock.getVersion(); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } - }; + return myLock.getVersion(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + }; DataSchema dataSchema = spec.getDataSchema(); RealtimeIOConfig realtimeIOConfig = spec.getIOConfig(); RealtimeTuningConfig tuningConfig = spec.getTuningConfig() - .withBasePersistDirectory(new File(toolbox.getTaskWorkDir(), "persist")) - .withVersioningPolicy(versioningPolicy); + .withBasePersistDirectory(new File(toolbox.getTaskWorkDir(), "persist")) + .withVersioningPolicy(versioningPolicy); final FireDepartment fireDepartment = new FireDepartment( dataSchema, @@ -352,7 +349,8 @@ public class RealtimeIndexTask extends AbstractTask } fireDepartment.getMetrics().incrementProcessed(); - if (currCount >= tuningConfig.getMaxRowsInMemory() || System.currentTimeMillis() > nextFlush) { + final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch()); + if ((sink != null && sink.isFull()) || System.currentTimeMillis() > nextFlush) { plumber.persist(firehose.commit()); nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis(); } 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 722153ccbef..af72bf3a0b2 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -471,6 +471,15 @@ public class IncrementalIndex implements Iterable, Closeable return numEntries.get() == 0; } + /** + * + * @return true if the underlying buffer for IncrementalIndex is full and cannot accomodate more rows. + */ + public boolean isFull() + { + return (numEntries.get() + 1) * totalAggSize > bufferHolder.get().limit(); + } + public int size() { return numEntries.get(); 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..52df7cafbe4 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -36,7 +36,7 @@ import java.io.File; */ public class RealtimeTuningConfig implements TuningConfig { - private static final int defaultMaxRowsInMemory = 500000; + private static final int defaultBufferSize = 256 * 1024 * 1024; private static final Period defaultIntermediatePersistPeriod = new Period("PT10M"); private static final Period defaultWindowPeriod = new Period("PT10M"); private static final File defaultBasePersistDirectory = Files.createTempDir(); @@ -49,7 +49,7 @@ public class RealtimeTuningConfig implements TuningConfig public static RealtimeTuningConfig makeDefaultTuningConfig() { return new RealtimeTuningConfig( - defaultMaxRowsInMemory, + defaultBufferSize, defaultIntermediatePersistPeriod, defaultWindowPeriod, defaultBasePersistDirectory, @@ -60,7 +60,7 @@ public class RealtimeTuningConfig implements TuningConfig ); } - private final int maxRowsInMemory; + private final int bufferSize; private final Period intermediatePersistPeriod; private final Period windowPeriod; private final File basePersistDirectory; @@ -71,7 +71,7 @@ public class RealtimeTuningConfig implements TuningConfig @JsonCreator public RealtimeTuningConfig( - @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, + @JsonProperty("bufferSize") Integer bufferSize, @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, @JsonProperty("windowPeriod") Period windowPeriod, @JsonProperty("basePersistDirectory") File basePersistDirectory, @@ -81,7 +81,7 @@ public class RealtimeTuningConfig implements TuningConfig @JsonProperty("shardSpec") ShardSpec shardSpec ) { - this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory; + this.bufferSize = bufferSize == null ? defaultBufferSize : bufferSize; this.intermediatePersistPeriod = intermediatePersistPeriod == null ? defaultIntermediatePersistPeriod : intermediatePersistPeriod; @@ -96,9 +96,9 @@ public class RealtimeTuningConfig implements TuningConfig } @JsonProperty - public int getMaxRowsInMemory() + public int getBufferSize() { - return maxRowsInMemory; + return bufferSize; } @JsonProperty @@ -146,7 +146,7 @@ public class RealtimeTuningConfig implements TuningConfig public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy) { return new RealtimeTuningConfig( - maxRowsInMemory, + bufferSize, intermediatePersistPeriod, windowPeriod, basePersistDirectory, @@ -160,7 +160,7 @@ public class RealtimeTuningConfig implements TuningConfig public RealtimeTuningConfig withBasePersistDirectory(File dir) { return new RealtimeTuningConfig( - maxRowsInMemory, + bufferSize, intermediatePersistPeriod, windowPeriod, dir, diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java index a1cfb220972..5ba960925aa 100644 --- a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java +++ b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java @@ -43,6 +43,7 @@ import io.druid.query.SegmentDescriptor; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.realtime.plumber.Plumber; +import io.druid.segment.realtime.plumber.Sink; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; @@ -208,7 +209,8 @@ public class RealtimeManager implements QuerySegmentWalker continue; } - if (currCount >= config.getMaxRowsInMemory() || System.currentTimeMillis() > nextFlush) { + final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch()); + if ((sink != null && sink.isFull()) || System.currentTimeMillis() > nextFlush) { plumber.persist(firehose.commit()); nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis(); } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Plumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/Plumber.java index 136d3a8a253..3824d30d1eb 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Plumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Plumber.java @@ -52,4 +52,6 @@ public interface Plumber * fed into sinks and persisted. */ public void finishJob(); + + public Sink getSink(long timeStamp); } 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 6a557b44822..87f2ef4a9ba 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 @@ -133,6 +133,13 @@ public class Sink implements Iterable } } + public boolean isFull() + { + synchronized (currHydrant){ + return currHydrant.getIndex().isFull(); + } + } + /** * If currHydrant is A, creates a new index B, sets currHydrant to B and returns A. * @@ -176,11 +183,6 @@ public class Sink implements Iterable private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) { - int aggsSize = 0; - for (AggregatorFactory agg : schema.getAggregators()) { - aggsSize += agg.getMaxIntermediateSize(); - } - int bufferSize = aggsSize * config.getMaxRowsInMemory(); IncrementalIndex newIndex = new IncrementalIndex( new IncrementalIndexSchema.Builder() .withMinTimestamp(minTimestamp) @@ -188,7 +190,7 @@ public class Sink implements Iterable .withDimensionsSpec(schema.getParser()) .withMetrics(schema.getAggregators()) .build(), - new OffheapBufferPool(bufferSize) + new OffheapBufferPool(config.getBufferSize()) ); FireHydrant old; diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index 43350445823..53e24e9c4a6 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -69,11 +69,11 @@ public class RealtimeManagerTest final List rows = Arrays.asList( makeRow(new DateTime("9000-01-01").getMillis()), makeRow(new DateTime().getMillis()) ); - + final AggregatorFactory[] aggs = {new CountAggregatorFactory("rows")}; schema = new DataSchema( "test", null, - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, + aggs, new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.NONE, null, Granularity.HOUR) ); RealtimeIOConfig ioConfig = new RealtimeIOConfig( @@ -108,8 +108,13 @@ public class RealtimeManagerTest } } ); + int rowSize = 0; + for (AggregatorFactory agg : aggs) { + rowSize += agg.getMaxIntermediateSize(); + } + RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( - 1, + rowSize, new Period("P1Y"), null, null, @@ -201,7 +206,6 @@ public class RealtimeManagerTest }; } - private static class TestFirehose implements Firehose { private final Iterator rows; @@ -238,8 +242,6 @@ public class RealtimeManagerTest private static class TestPlumber implements Plumber { private final Sink sink; - - private volatile boolean startedJob = false; private volatile boolean finishedJob = false; private volatile int persistCount = 0; From 79ff993b319209afe506be7b946db2ce17bf5957 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 20 Aug 2014 22:15:06 +0530 Subject: [PATCH 063/107] increase default buffer size to 512m --- .../src/main/java/io/druid/indexer/HadoopTuningConfig.java | 2 +- .../java/io/druid/segment/indexing/RealtimeTuningConfig.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java index a93c7653964..be70ef462e5 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java @@ -38,7 +38,7 @@ public class HadoopTuningConfig implements TuningConfig { private static final PartitionsSpec defaultPartitionsSpec = HashedPartitionsSpec.makeDefaultHashedPartitionsSpec(); private static final Map> defaultShardSpecs = ImmutableMap.>of(); - private static final int defaultBufferSize = 256 * 1024 * 1024; + private static final int defaultBufferSize = 512 * 1024 * 1024; public static HadoopTuningConfig makeDefaultTuningConfig() { 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 52df7cafbe4..29d93deb5a0 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -36,7 +36,7 @@ import java.io.File; */ public class RealtimeTuningConfig implements TuningConfig { - private static final int defaultBufferSize = 256 * 1024 * 1024; + private static final int defaultBufferSize = 512 * 1024 * 1024; private static final Period defaultIntermediatePersistPeriod = new Period("PT10M"); private static final Period defaultWindowPeriod = new Period("PT10M"); private static final File defaultBasePersistDirectory = Files.createTempDir(); From fd09b1b5f187d9ba34a5651b3dd2d653b7caea9d Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 20 Aug 2014 09:49:21 -0700 Subject: [PATCH 064/107] cleanup things according to cr --- .../java/io/druid/segment/IndexMaker.java | 27 +++++++++---------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/IndexMaker.java b/processing/src/main/java/io/druid/segment/IndexMaker.java index 5c24f2a83da..4cbb61b35b9 100644 --- a/processing/src/main/java/io/druid/segment/IndexMaker.java +++ b/processing/src/main/java/io/druid/segment/IndexMaker.java @@ -33,6 +33,7 @@ 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; @@ -45,9 +46,11 @@ 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.jackson.DefaultObjectMapper; +import io.druid.guice.GuiceInjectors; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.ToLowerCaseAggregatorFactory; import io.druid.segment.column.ColumnCapabilities; @@ -81,6 +84,7 @@ 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; @@ -105,8 +109,12 @@ public class IndexMaker private static final SerializerUtils serializerUtils = new SerializerUtils(); private static final int INVALID_ROW = -1; private static final Splitter SPLITTER = Splitter.on(","); - // This should really be provided by DI, should be changed once we switch around to using a DI framework - private static final ObjectMapper mapper = new DefaultObjectMapper(); + private static 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 @@ -461,7 +469,6 @@ public class IndexMaker ); final Map dimIndexes = Maps.newHashMap(); - final Map dimensionCardinalities = Maps.newHashMap(); final Map> dimensionValuesLookup = Maps.newHashMap(); final ArrayList> dimConversions = Lists.newArrayListWithCapacity(adapters.size()); final Set skippedDimensions = Sets.newHashSet(); @@ -475,7 +482,6 @@ public class IndexMaker dimConversions, dimIndexes, skippedDimensions, - dimensionCardinalities, dimensionValuesLookup ); @@ -503,7 +509,6 @@ public class IndexMaker skippedDimensions, theRows, columnCapabilities, - dimensionCardinalities, dimensionValuesLookup, rowNumConversions ); @@ -528,7 +533,6 @@ public class IndexMaker final List> dimConversions, final Map dimIndexes, final Set skippedDimensions, - final Map dimensionCardinalities, final Map> dimensionValuesLookup ) { @@ -595,8 +599,6 @@ public class IndexMaker ++cardinality; } - dimensionCardinalities.put(dimension, cardinality); - if (cardinality == 0) { log.info("Skipping [%s], it is empty!", dimension); skippedDimensions.add(dimension); @@ -774,7 +776,6 @@ public class IndexMaker final Set skippedDimensions, final Iterable theRows, final Map columnCapabilities, - final Map dimensionCardinalities, final Map> dimensionValuesLookup, final List rowNumConversions ) throws IOException @@ -797,7 +798,6 @@ public class IndexMaker dimIndex, dimension, columnCapabilities, - dimensionCardinalities, dimensionValuesLookup, rowNumConversions ); @@ -815,7 +815,6 @@ public class IndexMaker final int dimIndex, final String dimension, final Map columnCapabilities, - final Map dimensionCardinalities, final Map> dimensionValuesLookup, final List rowNumConversions ) throws IOException @@ -1405,9 +1404,9 @@ public class IndexMaker ) { this.dimSet = dimSet; - conversionBuf = ByteBuffer.allocateDirect(dimSet.size() * Ints.BYTES).asIntBuffer(); + this.conversionBuf = ByteBuffer.allocateDirect(dimSet.size() * Ints.BYTES).asIntBuffer(); - currIndex = 0; + this.currIndex = 0; } public void convert(String value, int index) From fe105d52ee57d4a2939225e0a781d7fb59b4be22 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 20 Aug 2014 22:39:23 +0530 Subject: [PATCH 065/107] use bufferSize for IndexTask --- .../druid/indexing/common/task/IndexTask.java | 25 +- .../indexing/overlord/TaskLifecycleTest.java | 352 +++++++++--------- 2 files changed, 193 insertions(+), 184 deletions(-) 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 4ab040d947d..c9022fa765b 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 @@ -55,6 +55,7 @@ import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.realtime.FireDepartmentMetrics; import io.druid.segment.realtime.plumber.Plumber; +import io.druid.segment.realtime.plumber.Sink; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.NoneShardSpec; @@ -372,7 +373,6 @@ public class IndexTask extends AbstractFixedIntervalTask ); final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); - final int rowFlushBoundary = ingestionSchema.getTuningConfig().getRowFlushBoundary(); // We need to track published segments. final List pushedSegments = new CopyOnWriteArrayList(); @@ -403,15 +403,10 @@ public class IndexTask extends AbstractFixedIntervalTask tmpDir ).findPlumber( schema, - new RealtimeTuningConfig(null, null, null, null, null, null, null, shardSpec), + new RealtimeTuningConfig(ingestionSchema.getTuningConfig().getBufferSize(), null, null, null, null, null, null, shardSpec), metrics ); - // rowFlushBoundary for this job - final int myRowFlushBoundary = rowFlushBoundary > 0 - ? rowFlushBoundary - : toolbox.getConfig().getDefaultRowFlushBoundary(); - try { plumber.startJob(); @@ -429,8 +424,8 @@ public class IndexTask extends AbstractFixedIntervalTask ); } metrics.incrementProcessed(); - - if (numRows >= myRowFlushBoundary) { + Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch()); + if (sink != null && sink.isFull()) { plumber.persist(firehose.commit()); } } else { @@ -548,21 +543,21 @@ public class IndexTask extends AbstractFixedIntervalTask public static class IndexTuningConfig implements TuningConfig { private static final int DEFAULT_TARGET_PARTITION_SIZE = 5000000; - private static final int DEFAULT_ROW_FLUSH_BOUNDARY = 500000; + private static final int DEFAULT_BUFFER_SIZE = 512 * 1024 * 1024; private final int targetPartitionSize; - private final int rowFlushBoundary; + private final int bufferSize; private final int numShards; @JsonCreator public IndexTuningConfig( @JsonProperty("targetPartitionSize") int targetPartitionSize, - @JsonProperty("rowFlushBoundary") int rowFlushBoundary, + @JsonProperty("bufferSize") int bufferSize, @JsonProperty("numShards") @Nullable Integer numShards ) { this.targetPartitionSize = targetPartitionSize == 0 ? DEFAULT_TARGET_PARTITION_SIZE : targetPartitionSize; - this.rowFlushBoundary = rowFlushBoundary == 0 ? DEFAULT_ROW_FLUSH_BOUNDARY : rowFlushBoundary; + this.bufferSize = bufferSize == 0 ? DEFAULT_BUFFER_SIZE : bufferSize; this.numShards = numShards == null ? -1 : numShards; Preconditions.checkArgument( this.targetPartitionSize == -1 || this.numShards == -1, @@ -577,9 +572,9 @@ public class IndexTask extends AbstractFixedIntervalTask } @JsonProperty - public int getRowFlushBoundary() + public int getBufferSize() { - return rowFlushBoundary; + return bufferSize; } @JsonProperty diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 33411036e11..a8a0e7d526d 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -43,14 +43,12 @@ import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.InputRowParser; import io.druid.granularity.QueryGranularity; -import io.druid.indexing.common.TestUtils; -import io.druid.segment.column.ColumnConfig; -import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.TaskToolboxFactory; +import io.druid.indexing.common.TestUtils; import io.druid.indexing.common.actions.LocalTaskActionClientFactory; import io.druid.indexing.common.actions.LockListAction; import io.druid.indexing.common.actions.SegmentInsertAction; @@ -67,6 +65,8 @@ import io.druid.indexing.overlord.config.TaskQueueConfig; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentMover; @@ -96,16 +96,6 @@ import java.util.Set; public class TaskLifecycleTest { - private File tmp = null; - private TaskStorage ts = null; - private TaskLockbox tl = null; - private TaskQueue tq = null; - private TaskRunner tr = null; - private MockIndexerDBCoordinator mdc = null; - private TaskActionClientFactory tac = null; - private TaskToolboxFactory tb = null; - TaskStorageQueryAdapter tsqa = null; - private static final Ordering byIntervalOrdering = new Ordering() { @Override @@ -114,6 +104,153 @@ public class TaskLifecycleTest return Comparators.intervalsByStartThenEnd().compare(dataSegment.getInterval(), dataSegment2.getInterval()); } }; + TaskStorageQueryAdapter tsqa = null; + private File tmp = null; + private TaskStorage ts = null; + private TaskLockbox tl = null; + private TaskQueue tq = null; + private TaskRunner tr = null; + private MockIndexerDBCoordinator mdc = null; + private TaskActionClientFactory tac = null; + private TaskToolboxFactory tb = null; + + private static MockIndexerDBCoordinator newMockMDC() + { + return new MockIndexerDBCoordinator(); + } + + private static ServiceEmitter newMockEmitter() + { + return new ServiceEmitter(null, null, null) + { + @Override + public void emit(Event event) + { + + } + + @Override + public void emit(ServiceEventBuilder builder) + { + + } + }; + } + + private static InputRow IR(String dt, String dim1, String dim2, float met) + { + return new MapBasedInputRow( + new DateTime(dt).getMillis(), + ImmutableList.of("dim1", "dim2"), + ImmutableMap.of( + "dim1", dim1, + "dim2", dim2, + "met", met + ) + ); + } + + private static FirehoseFactory newMockExceptionalFirehoseFactory() + { + return new FirehoseFactory() + { + @Override + public Firehose connect(InputRowParser parser) throws IOException + { + return new Firehose() + { + @Override + public boolean hasMore() + { + return true; + } + + @Override + public InputRow nextRow() + { + throw new RuntimeException("HA HA HA"); + } + + @Override + public Runnable commit() + { + return new Runnable() + { + @Override + public void run() + { + + } + }; + } + + @Override + public void close() throws IOException + { + + } + }; + } + + @Override + public InputRowParser getParser() + { + return null; + } + }; + } + + private static FirehoseFactory newMockFirehoseFactory(final Iterable inputRows) + { + return new FirehoseFactory() + { + @Override + public Firehose connect(InputRowParser parser) throws IOException + { + final Iterator inputRowIterator = inputRows.iterator(); + + return new Firehose() + { + @Override + public boolean hasMore() + { + return inputRowIterator.hasNext(); + } + + @Override + public InputRow nextRow() + { + return inputRowIterator.next(); + } + + @Override + public Runnable commit() + { + return new Runnable() + { + @Override + public void run() + { + + } + }; + } + + @Override + public void close() throws IOException + { + + } + }; + } + + @Override + public InputRowParser getParser() + { + return null; + } + }; + } @Before public void setUp() throws Exception @@ -231,25 +368,35 @@ public class TaskLifecycleTest { final Task indexTask = new IndexTask( null, - null, - "foo", - new UniformGranularitySpec( - Granularity.DAY, - null, - ImmutableList.of(new Interval("2010-01-01/P2D")), - Granularity.DAY - ), - new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, - QueryGranularity.NONE, - 10000, - newMockFirehoseFactory( - ImmutableList.of( - IR("2010-01-01T01", "x", "y", 1), - IR("2010-01-01T01", "x", "z", 1), - IR("2010-01-02T01", "a", "b", 2), - IR("2010-01-02T01", "a", "c", 1) + new IndexTask.IndexIngestionSpec( + new DataSchema( + "foo", + null, + new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, + new UniformGranularitySpec( + Granularity.DAY, + null, + ImmutableList.of(new Interval("2010-01-01/P2D")), + Granularity.DAY + ) + ), new IndexTask.IndexIOConfig( + newMockFirehoseFactory( + ImmutableList.of( + IR("2010-01-01T01", "x", "y", 1), + IR("2010-01-01T01", "x", "z", 1), + IR("2010-01-02T01", "a", "b", 2), + IR("2010-01-02T01", "a", "c", 1) + ) ) ), + new IndexTask.IndexTuningConfig(10000, 5 * 1024 * 1024, -1) + ), + null, + null, + null, + null, + -1, + null, -1, TestUtils.MAPPER ); @@ -294,7 +441,12 @@ public class TaskLifecycleTest null, null, "foo", - new UniformGranularitySpec(Granularity.DAY, null, ImmutableList.of(new Interval("2010-01-01/P1D")), Granularity.DAY), + new UniformGranularitySpec( + Granularity.DAY, + null, + ImmutableList.of(new Interval("2010-01-01/P1D")), + Granularity.DAY + ), new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, QueryGranularity.NONE, 10000, @@ -561,142 +713,4 @@ public class TaskLifecycleTest return ImmutableSet.copyOf(nuked); } } - - private static MockIndexerDBCoordinator newMockMDC() - { - return new MockIndexerDBCoordinator(); - } - - private static ServiceEmitter newMockEmitter() - { - return new ServiceEmitter(null, null, null) - { - @Override - public void emit(Event event) - { - - } - - @Override - public void emit(ServiceEventBuilder builder) - { - - } - }; - } - - private static InputRow IR(String dt, String dim1, String dim2, float met) - { - return new MapBasedInputRow( - new DateTime(dt).getMillis(), - ImmutableList.of("dim1", "dim2"), - ImmutableMap.of( - "dim1", dim1, - "dim2", dim2, - "met", met - ) - ); - } - - private static FirehoseFactory newMockExceptionalFirehoseFactory() - { - return new FirehoseFactory() - { - @Override - public Firehose connect(InputRowParser parser) throws IOException - { - return new Firehose() - { - @Override - public boolean hasMore() - { - return true; - } - - @Override - public InputRow nextRow() - { - throw new RuntimeException("HA HA HA"); - } - - @Override - public Runnable commit() - { - return new Runnable() - { - @Override - public void run() - { - - } - }; - } - - @Override - public void close() throws IOException - { - - } - }; - } - - @Override - public InputRowParser getParser() - { - return null; - } - }; - } - - private static FirehoseFactory newMockFirehoseFactory(final Iterable inputRows) - { - return new FirehoseFactory() - { - @Override - public Firehose connect(InputRowParser parser) throws IOException - { - final Iterator inputRowIterator = inputRows.iterator(); - - return new Firehose() - { - @Override - public boolean hasMore() - { - return inputRowIterator.hasNext(); - } - - @Override - public InputRow nextRow() - { - return inputRowIterator.next(); - } - - @Override - public Runnable commit() - { - return new Runnable() - { - @Override - public void run() - { - - } - }; - } - - @Override - public void close() throws IOException - { - - } - }; - } - - @Override - public InputRowParser getParser() - { - return null; - } - }; - } } From 8933685884244fb0311d636c211dac5ad8aacad9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 20 Aug 2014 12:41:47 -0700 Subject: [PATCH 066/107] log allocation of conversionBuf --- processing/src/main/java/io/druid/segment/IndexMaker.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/segment/IndexMaker.java b/processing/src/main/java/io/druid/segment/IndexMaker.java index 4cbb61b35b9..34fda27702e 100644 --- a/processing/src/main/java/io/druid/segment/IndexMaker.java +++ b/processing/src/main/java/io/druid/segment/IndexMaker.java @@ -1404,7 +1404,9 @@ public class IndexMaker ) { this.dimSet = dimSet; - this.conversionBuf = ByteBuffer.allocateDirect(dimSet.size() * Ints.BYTES).asIntBuffer(); + 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; } From 92f26d9a1fc7fc59eb06ff4612dca98bb5465cac Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 20 Aug 2014 13:09:37 -0700 Subject: [PATCH 067/107] cleanup rowflushboundary --- .../java/io/druid/indexer/rollup/DataRollupSpec.java | 3 --- .../io/druid/indexing/common/config/TaskConfig.java | 10 ---------- 2 files changed, 13 deletions(-) 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..e06932d0311 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 @@ -38,9 +38,6 @@ public class DataRollupSpec @JsonProperty public QueryGranularity rollupGranularity = QueryGranularity.NONE; - @JsonProperty - public int rowFlushBoundary = 500000; - public DataRollupSpec() {} public DataRollupSpec(List aggs, QueryGranularity rollupGranularity) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java index 1671fb84bd3..6d60d002484 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java @@ -41,9 +41,6 @@ public class TaskConfig @JsonProperty private final String hadoopWorkingPath; - @JsonProperty - private final int defaultRowFlushBoundary; - @JsonProperty private final List defaultHadoopCoordinates; @@ -59,7 +56,6 @@ public class TaskConfig this.baseDir = baseDir == null ? "/tmp" : baseDir; this.baseTaskDir = new File(defaultDir(baseTaskDir, "persistent/task")); this.hadoopWorkingPath = defaultDir(hadoopWorkingPath, "druid-indexing"); - this.defaultRowFlushBoundary = defaultRowFlushBoundary == null ? 500000 : defaultRowFlushBoundary; this.defaultHadoopCoordinates = defaultHadoopCoordinates == null ? DEFAULT_DEFAULT_HADOOP_COORDINATES : defaultHadoopCoordinates; @@ -83,12 +79,6 @@ public class TaskConfig return hadoopWorkingPath; } - @JsonProperty - public int getDefaultRowFlushBoundary() - { - return defaultRowFlushBoundary; - } - @JsonProperty public List getDefaultHadoopCoordinates() { From 1614f40f1abb87a72a1882b1e3e05a5f1a83c654 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 20 Aug 2014 13:11:00 -0700 Subject: [PATCH 068/107] fix index task --- .../main/java/io/druid/indexing/common/task/IndexTask.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 c9022fa765b..dc6e334aece 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 @@ -148,7 +148,7 @@ public class IndexTask extends AbstractFixedIntervalTask granularitySpec.withQueryGranularity(indexGranularity == null ? QueryGranularity.NONE : indexGranularity) ), new IndexIOConfig(firehoseFactory), - new IndexTuningConfig(targetPartitionSize, rowFlushBoundary, null) + new IndexTuningConfig(targetPartitionSize, null, null) ); } this.jsonMapper = jsonMapper; @@ -552,12 +552,12 @@ public class IndexTask extends AbstractFixedIntervalTask @JsonCreator public IndexTuningConfig( @JsonProperty("targetPartitionSize") int targetPartitionSize, - @JsonProperty("bufferSize") int bufferSize, + @JsonProperty("bufferSize") @Nullable Integer bufferSize, @JsonProperty("numShards") @Nullable Integer numShards ) { this.targetPartitionSize = targetPartitionSize == 0 ? DEFAULT_TARGET_PARTITION_SIZE : targetPartitionSize; - this.bufferSize = bufferSize == 0 ? DEFAULT_BUFFER_SIZE : bufferSize; + this.bufferSize = bufferSize == null ? DEFAULT_BUFFER_SIZE : bufferSize; this.numShards = numShards == null ? -1 : numShards; Preconditions.checkArgument( this.targetPartitionSize == -1 || this.numShards == -1, From bb73b2556e95ba75ad6e660ebc11d5b97aee60d5 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 20 Aug 2014 13:17:00 -0700 Subject: [PATCH 069/107] fix compilation --- .../src/main/java/io/druid/indexer/HadoopIngestionSpec.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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..d759ca26074 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopIngestionSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopIngestionSpec.java @@ -159,7 +159,7 @@ public class HadoopIngestionSpec extends IngestionSpec Date: Wed, 20 Aug 2014 13:22:42 -0700 Subject: [PATCH 070/107] more cleanup --- .../io/druid/indexer/rollup/DataRollupSpec.java | 14 +------------- .../druid/indexing/common/task/TaskSerdeTest.java | 2 +- 2 files changed, 2 insertions(+), 14 deletions(-) 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 e06932d0311..9f5048559d1 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 @@ -30,6 +30,7 @@ import java.util.List; * * Adjust to JsonCreator and final fields when resolved. */ +@Deprecated public class DataRollupSpec { @JsonProperty @@ -38,21 +39,8 @@ public class DataRollupSpec @JsonProperty public QueryGranularity rollupGranularity = QueryGranularity.NONE; - 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-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 daa35c792fb..c6bd1658830 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 @@ -369,7 +369,7 @@ public class TaskSerdeTest true, null, false, - new DataRollupSpec(ImmutableList.of(), QueryGranularity.NONE), + null, null, false, ImmutableMap.of("foo", "bar"), From 67f4bbae74b45af24146551a3a6208475e3d8f34 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Thu, 21 Aug 2014 09:24:04 +0530 Subject: [PATCH 071/107] fixes from review comments fix sync of aggs, fix NPE in sink.isFull, RealtimeTuningConfig lower the bufferSize to 256m --- .../segment/incremental/IncrementalIndex.java | 285 ++++++++++-------- .../indexing/RealtimeTuningConfig.java | 2 +- .../druid/segment/realtime/plumber/Sink.java | 2 +- 3 files changed, 165 insertions(+), 124 deletions(-) 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 af72bf3a0b2..124035e612f 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -130,146 +130,148 @@ public class IncrementalIndex implements Iterable, Closeable int currAggSize = 0; for (int i = 0; i < metrics.length; i++) { final AggregatorFactory agg = metrics[i]; - aggs[i] = agg.factorizeBuffered( - new ColumnSelectorFactory() - { - @Override - public TimestampColumnSelector makeTimestampColumnSelector() - { - return new TimestampColumnSelector() + aggs[i] = new ThreadSafeAggregator( + agg.factorizeBuffered( + new ColumnSelectorFactory() { @Override - public long getTimestamp() + public TimestampColumnSelector makeTimestampColumnSelector() { - return in.get().getTimestampFromEpoch(); - } - }; - } - - @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() + return new TimestampColumnSelector() { @Override - public int size() + public long getTimestamp() { - return vals.size(); - } - - @Override - public int get(int index) - { - return vals.get(index); - } - - @Override - public Iterator iterator() - { - return vals.iterator(); + return in.get().getTimestampFromEpoch(); } }; } @Override - public int getValueCardinality() + public FloatColumnSelector makeFloatColumnSelector(String columnName) { - throw new UnsupportedOperationException("value cardinality is unknown in incremental index"); + final String metricName = columnName.toLowerCase(); + return new FloatColumnSelector() + { + @Override + public float get() + { + return in.get().getFloatMetric(metricName); + } + }; } @Override - public String lookupName(int id) + public ObjectColumnSelector makeObjectColumnSelector(String column) { - return in.get().getDimension(dimensionName).get(id); + 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 int lookupId(String name) + public DimensionSelector makeDimensionSelector(final String dimension) { - return in.get().getDimension(dimensionName).indexOf(name); + 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(); @@ -458,9 +460,7 @@ public class IncrementalIndex implements Iterable, Closeable in.set(row); int rowOffset = facts.get(key); 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(); @@ -473,7 +473,7 @@ public class IncrementalIndex implements Iterable, Closeable /** * - * @return true if the underlying buffer for IncrementalIndex is full and cannot accomodate more rows. + * @return true if the underlying buffer for IncrementalIndex is full and cannot accommodate more rows. */ public boolean isFull() { @@ -861,4 +861,45 @@ public class IncrementalIndex implements Iterable, Closeable } } } + + private static class ThreadSafeAggregator implements BufferAggregator + { + + private final BufferAggregator delegate; + + public ThreadSafeAggregator(BufferAggregator delegate) + { + this.delegate = delegate; + } + + @Override + public synchronized void init(ByteBuffer buf, int position) + { + delegate.init(buf, position); + } + + @Override + public synchronized void aggregate(ByteBuffer buf, int position) + { + delegate.aggregate(buf, position); + } + + @Override + public synchronized Object get(ByteBuffer buf, int position) + { + return delegate.get(buf, position); + } + + @Override + public synchronized float getFloat(ByteBuffer buf, int position) + { + return delegate.getFloat(buf, position); + } + + @Override + public synchronized void close() + { + delegate.close(); + } + } } 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 29d93deb5a0..52df7cafbe4 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -36,7 +36,7 @@ import java.io.File; */ public class RealtimeTuningConfig implements TuningConfig { - private static final int defaultBufferSize = 512 * 1024 * 1024; + private static final int defaultBufferSize = 256 * 1024 * 1024; private static final Period defaultIntermediatePersistPeriod = new Period("PT10M"); private static final Period defaultWindowPeriod = new Period("PT10M"); private static final File defaultBasePersistDirectory = Files.createTempDir(); 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 87f2ef4a9ba..03315201520 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 @@ -136,7 +136,7 @@ public class Sink implements Iterable public boolean isFull() { synchronized (currHydrant){ - return currHydrant.getIndex().isFull(); + return currHydrant != null && currHydrant.getIndex().isFull(); } } From c216eb73407bc6b5534f93825d41ce69542e4095 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Thu, 21 Aug 2014 23:56:07 +0530 Subject: [PATCH 072/107] remove ThreadSafeAggregator --- .../segment/incremental/IncrementalIndex.java | 269 ++++++++---------- 1 file changed, 111 insertions(+), 158 deletions(-) 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 124035e612f..da9fe613f42 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -80,9 +80,7 @@ public class IncrementalIndex implements Iterable, Closeable { private final long minTimestamp; private final QueryGranularity gran; - private final List> rowTransformers; - private final AggregatorFactory[] metrics; private final Map metricIndexes; private final Map metricTypes; @@ -93,9 +91,7 @@ public class IncrementalIndex implements Iterable, Closeable private final LinkedHashMap dimensionOrder; private final CopyOnWriteArrayList dimensions; private final DimensionHolder dimValues; - private final Map columnCapabilities; - private final ConcurrentSkipListMap facts; private final ResourceHolder bufferHolder; private volatile AtomicInteger numEntries = new AtomicInteger(); @@ -130,148 +126,146 @@ public class IncrementalIndex implements Iterable, Closeable int currAggSize = 0; for (int i = 0; i < metrics.length; i++) { final AggregatorFactory agg = metrics[i]; - aggs[i] = new ThreadSafeAggregator( - agg.factorizeBuffered( - new ColumnSelectorFactory() + aggs[i] = agg.factorizeBuffered( + new ColumnSelectorFactory() + { + @Override + public TimestampColumnSelector makeTimestampColumnSelector() + { + return new TimestampColumnSelector() { @Override - public TimestampColumnSelector makeTimestampColumnSelector() + public long getTimestamp() { - return new TimestampColumnSelector() - { - @Override - public long getTimestamp() - { - return in.get().getTimestampFromEpoch(); - } - }; + return in.get().getTimestampFromEpoch(); + } + }; + } + + @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 FloatColumnSelector makeFloatColumnSelector(String columnName) + public Object get() { - final String metricName = columnName.toLowerCase(); - return new FloatColumnSelector() - { - @Override - public float get() - { - return in.get().getFloatMetric(metricName); - } - }; + return in.get().getRaw(columnName); + } + }; + + if (!deserializeComplexMetrics) { + return rawColumnSelector; + } else { + if (typeName.equals("float")) { + return rawColumnSelector; } - @Override - public ObjectColumnSelector makeObjectColumnSelector(String column) + 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() { - final String typeName = agg.getTypeName(); - final String columnName = column.toLowerCase(); - - final ObjectColumnSelector rawColumnSelector = new ObjectColumnSelector() + @Override + public Class classOfObject() { - @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); - } - }; + 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 DimensionSelector makeDimensionSelector(final String dimension) + public IndexedInts getRow() { - final String dimensionName = dimension.toLowerCase(); - return new DimensionSelector() + 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 IndexedInts getRow() + public int size() { - 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(); - } - }; + return vals.size(); } @Override - public int getValueCardinality() + public int get(int index) { - throw new UnsupportedOperationException("value cardinality is unknown in incremental index"); + return vals.get(index); } @Override - public String lookupName(int id) + public Iterator iterator() { - return in.get().getDimension(dimensionName).get(id); - } - - @Override - public int lookupId(String name) - { - return in.get().getDimension(dimensionName).indexOf(name); + 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(); @@ -861,45 +855,4 @@ public class IncrementalIndex implements Iterable, Closeable } } } - - private static class ThreadSafeAggregator implements BufferAggregator - { - - private final BufferAggregator delegate; - - public ThreadSafeAggregator(BufferAggregator delegate) - { - this.delegate = delegate; - } - - @Override - public synchronized void init(ByteBuffer buf, int position) - { - delegate.init(buf, position); - } - - @Override - public synchronized void aggregate(ByteBuffer buf, int position) - { - delegate.aggregate(buf, position); - } - - @Override - public synchronized Object get(ByteBuffer buf, int position) - { - return delegate.get(buf, position); - } - - @Override - public synchronized float getFloat(ByteBuffer buf, int position) - { - return delegate.getFloat(buf, position); - } - - @Override - public synchronized void close() - { - delegate.close(); - } - } } From e5d1c390fa3fc815ded6e3605e3b5cc51da370ea Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 21 Aug 2014 13:17:29 -0700 Subject: [PATCH 073/107] add extra logging --- processing/src/main/java/io/druid/segment/IndexMaker.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/segment/IndexMaker.java b/processing/src/main/java/io/druid/segment/IndexMaker.java index 34fda27702e..74911a01777 100644 --- a/processing/src/main/java/io/druid/segment/IndexMaker.java +++ b/processing/src/main/java/io/druid/segment/IndexMaker.java @@ -819,7 +819,6 @@ public class IndexMaker final List rowNumConversions ) throws IOException { - final String section = String.format("make %s", dimension); progress.startSection(section); @@ -1181,6 +1180,8 @@ public class IndexMaker spatialIndex = ImmutableRTree.newImmutableFromMutable(tree); } + log.info("Completed dimension[%s] with cardinality[%,d]. Starting write.", dimension, dictionary.size()); + writeColumn( v9Smoosher, new DictionaryEncodedColumnPartSerde( From 64ff5f0880525110d2357b5851fb7cbeaae39c91 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 21 Aug 2014 13:20:23 -0700 Subject: [PATCH 074/107] add back broken synch --- .../java/io/druid/segment/incremental/IncrementalIndex.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 da9fe613f42..c1ab7d49327 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -28,7 +28,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.metamx.common.IAE; @@ -454,7 +453,9 @@ public class IncrementalIndex implements Iterable, Closeable in.set(row); int rowOffset = facts.get(key); 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(); @@ -466,7 +467,6 @@ public class IncrementalIndex implements Iterable, Closeable } /** - * * @return true if the underlying buffer for IncrementalIndex is full and cannot accommodate more rows. */ public boolean isFull() From 12f4147df57593736fb69c17cfad3089eee7f543 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 21 Aug 2014 13:28:15 -0700 Subject: [PATCH 075/107] switch index gen job to use logging indicator --- .../src/main/java/io/druid/indexer/IndexGeneratorJob.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 1fda38cccf1..bdd25f8e78d 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -40,6 +40,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.BaseProgressIndicator; import io.druid.segment.IndexIO; 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; @@ -296,7 +297,7 @@ public class IndexGeneratorJob implements Jobby long startTime = System.currentTimeMillis(); Set allDimensionNames = Sets.newHashSet(); - final ProgressIndicator progressIndicator = new BaseProgressIndicator() + final ProgressIndicator progressIndicator = new LoggingProgressIndicator("IndexGeneratorJob") { @Override public void progress() From 197c80a69418a88d3baa4d0b2c8b056c7b411320 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 21 Aug 2014 13:40:55 -0700 Subject: [PATCH 076/107] fix logging --- .../main/java/io/druid/segment/LoggingProgressIndicator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java b/processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java index e86716e8fb7..39b1bcbd976 100644 --- a/processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java +++ b/processing/src/main/java/io/druid/segment/LoggingProgressIndicator.java @@ -76,7 +76,7 @@ public class LoggingProgressIndicator extends BaseProgressIndicator @Override public void progressSection(String section, String message) { - Stopwatch sectionWatch = sections.remove(section); + Stopwatch sectionWatch = sections.get(section); if (sectionWatch == null) { throw new ISE("[%s]: Cannot progress tracker for [%s]. Nothing started.", progressName, section); } From 88de45c6955a006c65af3c98f6edba0117fb91a4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 29 Aug 2014 15:13:52 -0700 Subject: [PATCH 077/107] ensure jackson injection is available on startup --- common/src/main/java/io/druid/guice/DruidSecondaryModule.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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); } From 75134335de5cb732f541f3c2eb1e5cc9a42b5b58 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Sat, 30 Aug 2014 22:45:36 -0700 Subject: [PATCH 078/107] fix last merge from master not being forward-compatible --- .../java/io/druid/query/TimewarpOperator.java | 7 +- .../io/druid/query/TimewarpOperatorTest.java | 12 +- .../timeseries/TimeseriesQueryRunnerTest.java | 130 +++++++++--------- 3 files changed, 78 insertions(+), 71 deletions(-) diff --git a/processing/src/main/java/io/druid/query/TimewarpOperator.java b/processing/src/main/java/io/druid/query/TimewarpOperator.java index 49a8fb1ed85..3f197795d6d 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/test/java/io/druid/query/TimewarpOperatorTest.java b/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java index f1703e8099c..f3aff876944 100644 --- a/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java +++ b/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java @@ -35,9 +35,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 +79,7 @@ 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 +124,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,7 +138,7 @@ public class TimewarpOperatorTest new QueryRunner>() { @Override - public Sequence> run(Query> query) + public Sequence> run(Query> query, Map context) { return Sequences.simple( ImmutableList.of( @@ -161,7 +165,7 @@ 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()) ); } 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 635e49ccd1a..ec2ff8e277d 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -54,7 +54,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.Map; import java.util.List; /** @@ -62,6 +62,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,10 +106,9 @@ public class TimeseriesQueryRunnerTest DateTime expectedEarliest = new DateTime("2011-01-12"); DateTime expectedLast = new DateTime("2011-04-15"); - HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); @@ -170,9 +172,8 @@ public class TimeseriesQueryRunnerTest DateTime expectedEarliest = new DateTime("2011-01-12"); DateTime expectedLast = new DateTime("2011-04-15"); - HashMap context = new HashMap(); Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); Result result = results.iterator().next(); @@ -217,9 +218,9 @@ public class TimeseriesQueryRunnerTest DateTime expectedEarliest = new DateTime("2011-01-12"); DateTime expectedLast = new DateTime("2011-04-15"); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); @@ -283,9 +284,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); @@ -330,9 +331,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); @@ -372,9 +373,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results1 = Sequences.toList( - runner.run(query1, context), + runner.run(query1, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults1, results1); @@ -411,7 +412,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results2 = Sequences.toList( - runner.run(query2, context), + runner.run(query2, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults2, results2); @@ -462,9 +463,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results1 = Sequences.toList( - runner.run(query1, context), + runner.run(query1, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults1, results1); @@ -532,7 +533,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results1 = Sequences.toList( - runner.run(query1), + runner.run(query1, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults1, results1); @@ -579,7 +580,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results1 = Sequences.toList( - runner.run(query1), + runner.run(query1, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults1, results1); @@ -619,9 +620,8 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); Iterable> results1 = Sequences.toList( - runner.run(query1, context), + runner.run(query1, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults1, results1); @@ -659,7 +659,7 @@ public class TimeseriesQueryRunnerTest ); Iterable> results2 = Sequences.toList( - runner.run(query2, context), + runner.run(query2, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults2, results2); @@ -692,9 +692,9 @@ public class TimeseriesQueryRunnerTest .build(); List> expectedResults = Arrays.asList(); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -742,9 +742,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -792,9 +792,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -842,9 +842,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -892,9 +892,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -942,9 +942,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1000,9 +1000,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1058,9 +1058,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1116,9 +1116,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1172,9 +1172,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1234,9 +1234,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1278,9 +1278,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1322,9 +1322,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1380,9 +1380,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> results = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, results); @@ -1420,9 +1420,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1461,9 +1461,9 @@ public class TimeseriesQueryRunnerTest ) ) ); - HashMap context = new HashMap(); + Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1480,7 +1480,7 @@ public class TimeseriesQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - HashMap context = new HashMap(); + Iterable> expectedResults = Sequences.toList( runner.run( Druids.newTimeseriesQueryBuilder() @@ -1490,12 +1490,12 @@ public class TimeseriesQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(), - context + CONTEXT ), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1512,7 +1512,7 @@ public class TimeseriesQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - HashMap context = new HashMap(); + Iterable> expectedResults = Sequences.toList( runner.run( Druids.newTimeseriesQueryBuilder() @@ -1523,12 +1523,12 @@ public class TimeseriesQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(), - context + CONTEXT ), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1574,7 +1574,7 @@ public class TimeseriesQueryRunnerTest ) ) .build(); - HashMap context = new HashMap(); + Iterable> expectedResults = Sequences.toList( runner.run( Druids.newTimeseriesQueryBuilder() @@ -1585,12 +1585,12 @@ public class TimeseriesQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(), - context + CONTEXT ), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); @@ -1638,7 +1638,7 @@ public class TimeseriesQueryRunnerTest ) ) .build(); - HashMap context = new HashMap(); + Iterable> expectedResults = Sequences.toList( runner.run( Druids.newTimeseriesQueryBuilder() @@ -1649,12 +1649,12 @@ public class TimeseriesQueryRunnerTest .aggregators(QueryRunnerTestHelper.commonAggregators) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(), - context + CONTEXT ), Lists.>newArrayList() ); Iterable> actualResults = Sequences.toList( - runner.run(query, context), + runner.run(query, CONTEXT), Lists.>newArrayList() ); TestHelper.assertExpectedResults(expectedResults, actualResults); From 178f002f05d7a52a5a583b336a8a498dea8924a8 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Tue, 9 Sep 2014 15:59:33 +0530 Subject: [PATCH 079/107] indexing working with mapdb --- pom.xml | 5 + processing/pom.xml | 4 + .../segment/incremental/IncrementalIndex.java | 117 +++++++++--------- .../IncrementalIndexStorageAdapter.java | 9 +- 4 files changed, 71 insertions(+), 64 deletions(-) diff --git a/pom.xml b/pom.xml index ce074209e92..c097a937158 100644 --- a/pom.xml +++ b/pom.xml @@ -429,6 +429,11 @@ 2.3.0 provided + + org.mapdb + mapdb + 1.0.6 + diff --git a/processing/pom.xml b/processing/pom.xml index 8b04ea52763..7ccbed8b4da 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -86,6 +86,10 @@ net.jpountz.lz4 lz4 + + org.mapdb + mapdb + 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 c1ab7d49327..4b22b71ee36 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -21,8 +21,6 @@ package io.druid.segment.incremental; import com.google.common.base.Function; import com.google.common.base.Throwables; -import com.google.common.collect.BiMap; -import com.google.common.collect.HashBiMap; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterators; @@ -56,10 +54,13 @@ import io.druid.segment.serde.ComplexMetricSerde; import io.druid.segment.serde.ComplexMetrics; import org.joda.time.DateTime; import org.joda.time.Interval; +import org.mapdb.DB; +import org.mapdb.DBMaker; import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; +import java.io.Serializable; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; @@ -69,7 +70,6 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentNavigableMap; -import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicInteger; @@ -91,8 +91,9 @@ public class IncrementalIndex implements Iterable, Closeable private final CopyOnWriteArrayList dimensions; private final DimensionHolder dimValues; private final Map columnCapabilities; - private final ConcurrentSkipListMap facts; + private final ConcurrentNavigableMap facts; private final ResourceHolder bufferHolder; + private final DB db; private volatile AtomicInteger numEntries = new AtomicInteger(); // This is modified on add() in a critical section. private ThreadLocal in = new ThreadLocal<>(); @@ -312,7 +313,8 @@ public class IncrementalIndex implements Iterable, Closeable } this.bufferHolder = bufferPool.take(); this.dimValues = new DimensionHolder(); - this.facts = new ConcurrentSkipListMap<>(); + db = DBMaker.newMemoryDirectDB().transactionDisable().cacheWeakRefEnable().make(); + this.facts = db.createTreeMap("facts").make(); } public IncrementalIndex( @@ -495,13 +497,10 @@ public class IncrementalIndex implements Iterable, Closeable int count = 0; for (String dimValue : dimValues) { - String canonicalDimValue = dimLookup.get(dimValue); - if (canonicalDimValue == null) { - canonicalDimValue = dimValue; + if (!dimLookup.contains(dimValue)) { dimLookup.add(dimValue); } - - retVal[count] = canonicalDimValue; + retVal[count] = dimValue; count++; } Arrays.sort(retVal); @@ -589,7 +588,7 @@ public class IncrementalIndex implements Iterable, Closeable return columnCapabilities.get(column); } - ConcurrentSkipListMap getFacts() + ConcurrentNavigableMap getFacts() { return facts; } @@ -655,45 +654,14 @@ public class IncrementalIndex implements Iterable, Closeable { try { bufferHolder.close(); + db.close(); } catch (IOException e) { throw Throwables.propagate(e); } } - static class DimensionHolder - { - private final Map dimensions; - - DimensionHolder() - { - dimensions = Maps.newConcurrentMap(); - } - - void reset() - { - dimensions.clear(); - } - - DimDim add(String dimension) - { - DimDim holder = dimensions.get(dimension); - if (holder == null) { - holder = new DimDim(); - dimensions.put(dimension, holder); - } else { - throw new ISE("dimension[%s] already existed even though add() was called!?", dimension); - } - return holder; - } - - DimDim get(String dimension) - { - return dimensions.get(dimension); - } - } - - static class TimeAndDims implements Comparable + static class TimeAndDims implements Comparable, Serializable { private final long timestamp; private final String[][] dims; @@ -778,23 +746,48 @@ public class IncrementalIndex implements Iterable, Closeable } } - static class DimDim + class DimensionHolder + { + private final Map dimensions; + + DimensionHolder() + { + dimensions = Maps.newConcurrentMap(); + } + + void reset() + { + dimensions.clear(); + } + + DimDim add(String dimension) + { + DimDim holder = dimensions.get(dimension); + if (holder == null) { + holder = new DimDim(dimension); + dimensions.put(dimension, holder); + } else { + throw new ISE("dimension[%s] already existed even though add() was called!?", dimension); + } + return holder; + } + + DimDim get(String dimension) + { + return dimensions.get(dimension); + } + } + + class DimDim { - private final Map poorMansInterning = Maps.newConcurrentMap(); private final Map falseIds; private final Map falseIdsReverse; private volatile String[] sortedVals = null; - public DimDim() + public DimDim(String dimName) { - BiMap biMap = Maps.synchronizedBiMap(HashBiMap.create()); - falseIds = biMap; - falseIdsReverse = biMap.inverse(); - } - - public String get(String value) - { - return value == null ? null : poorMansInterning.get(value); + falseIds = db.createHashMap(dimName).make(); + falseIdsReverse = db.createHashMap(dimName + "_inverse").make(); } public int getId(String value) @@ -807,20 +800,26 @@ public class IncrementalIndex implements Iterable, Closeable 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() { - return poorMansInterning.keySet(); + return falseIds.keySet(); } public synchronized void add(String value) { - poorMansInterning.put(value, value); - falseIds.put(value, falseIds.size()); + final int id = falseIds.size(); + falseIds.put(value, id); + falseIdsReverse.put(id, value); } public int getSortedId(String 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 d02a8c361b4..214bba923f3 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -488,14 +488,14 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter } @Override - public ValueMatcher makeValueMatcher(String dimension, String value) + public ValueMatcher makeValueMatcher(final String dimension,final String value) { Integer dimIndexObject = index.getDimensionIndex(dimension.toLowerCase()); if (dimIndexObject == null) { return new BooleanValueMatcher(false); } - String idObject = index.getDimension(dimension.toLowerCase()).get(value); - if (idObject == null) { + + if (!index.getDimension(dimension.toLowerCase()).contains(value)) { if (value == null || "".equals(value)) { final int dimIndex = dimIndexObject; @@ -516,7 +516,6 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter } final int dimIndex = dimIndexObject; - final String id = idObject; return new ValueMatcher() { @@ -529,7 +528,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter } for (String dimVal : dims[dimIndex]) { - if (id == dimVal) { + if (value.equals(dimVal)) { return true; } } From e7a5a01f7bb005278a20d4564f4b7727ea74c361 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Tue, 9 Sep 2014 16:02:50 +0530 Subject: [PATCH 080/107] cache size locally --- .../io/druid/segment/incremental/IncrementalIndex.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) 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 4b22b71ee36..c34c521c731 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -783,6 +783,8 @@ public class IncrementalIndex implements Iterable, Closeable private final Map falseIds; private final Map falseIdsReverse; private volatile String[] sortedVals = null; + // size on MapDB.HTreeMap is slow so maintain a count here + private volatile int size=0; public DimDim(String dimName) { @@ -807,7 +809,7 @@ public class IncrementalIndex implements Iterable, Closeable public int size() { - return falseIds.size(); + return size; } public Set keySet() @@ -817,7 +819,7 @@ public class IncrementalIndex implements Iterable, Closeable public synchronized void add(String value) { - final int id = falseIds.size(); + final int id = size++; falseIds.put(value, id); falseIdsReverse.put(id, value); } @@ -837,7 +839,7 @@ public class IncrementalIndex implements Iterable, Closeable public void sort() { if (sortedVals == null) { - sortedVals = new String[falseIds.size()]; + sortedVals = new String[size]; int index = 0; for (String value : falseIds.keySet()) { From 2b3ad4d742b1001fbfb469747c27e538a1a78f72 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Tue, 9 Sep 2014 23:20:19 +0530 Subject: [PATCH 081/107] append a random UUID to map name avoid collision with dimension name --- .../java/io/druid/segment/incremental/IncrementalIndex.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index c34c521c731..0e94e8b22e3 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -69,6 +69,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.UUID; import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicInteger; @@ -314,7 +315,7 @@ public class IncrementalIndex implements Iterable, Closeable this.bufferHolder = bufferPool.take(); this.dimValues = new DimensionHolder(); db = DBMaker.newMemoryDirectDB().transactionDisable().cacheWeakRefEnable().make(); - this.facts = db.createTreeMap("facts").make(); + this.facts = db.createTreeMap("__facts" + UUID.randomUUID()).make(); } public IncrementalIndex( From 47b52aaae17e4301d421e5b8abc8027e3e9ca204 Mon Sep 17 00:00:00 2001 From: Tarek Rached Date: Tue, 9 Sep 2014 12:44:30 -0700 Subject: [PATCH 082/107] moved old console to old-console --- .../resources/static/{ => old-console}/cluster.html | 0 .../resources/static/{ => old-console}/config.html | 0 .../static/{ => old-console}/css/config.css | 0 .../static/{ => old-console}/css/demo_table.css | 0 .../static/{ => old-console}/css/enable.css | 0 .../css/images/ui-bg_flat_0_aaaaaa_40x100.png | Bin .../css/images/ui-bg_flat_75_ffffff_40x100.png | Bin .../css/images/ui-bg_glass_55_fbf9ee_1x400.png | Bin .../css/images/ui-bg_glass_65_ffffff_1x400.png | Bin .../css/images/ui-bg_glass_75_dadada_1x400.png | Bin .../css/images/ui-bg_glass_75_e6e6e6_1x400.png | Bin .../css/images/ui-bg_glass_95_fef1ec_1x400.png | Bin .../images/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 .../resources/static/{ => old-console}/enable.html | 0 .../{ => old-console}/images/back_disabled.jpg | Bin .../{ => old-console}/images/back_enabled.jpg | Bin .../static/{ => old-console}/images/favicon.ico | Bin .../{ => old-console}/images/forward_disabled.jpg | Bin .../{ => old-console}/images/forward_enabled.jpg | Bin .../static/{ => old-console}/images/sort_asc.png | Bin .../{ => old-console}/images/sort_asc_disabled.png | Bin .../static/{ => old-console}/images/sort_both.png | Bin .../static/{ => old-console}/images/sort_desc.png | Bin .../{ => old-console}/images/sort_desc_disabled.png | Bin .../resources/static/{ => old-console}/index.html | 0 .../static/{ => old-console}/js/config-0.0.2.js | 0 .../static/{ => old-console}/js/druidTable-0.0.1.js | 0 .../static/{ => old-console}/js/enable-0.0.1.js | 0 .../static/{ => 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 .../static/{ => old-console}/js/jquery-ui-1.9.2.js | 0 .../{ => old-console}/js/jquery.dataTables-1.8.2.js | 0 .../static/{ => old-console}/js/kill-0.0.1.js | 0 .../static/{ => old-console}/js/rules-0.0.2.js | 0 .../{ => old-console}/js/tablehelper-0.0.2.js | 0 .../static/{ => old-console}/js/underscore-1.2.2.js | 0 .../resources/static/{ => old-console}/kill.html | 0 .../resources/static/{ => old-console}/rules.html | 0 .../resources/static/{ => old-console}/view.html | 0 49 files changed, 0 insertions(+), 0 deletions(-) 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%) rename server/src/main/resources/static/{ => old-console}/index.html (100%) 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%) diff --git a/server/src/main/resources/static/cluster.html b/server/src/main/resources/static/old-console/cluster.html similarity index 100% rename from server/src/main/resources/static/cluster.html rename to server/src/main/resources/static/old-console/cluster.html diff --git a/server/src/main/resources/static/config.html b/server/src/main/resources/static/old-console/config.html similarity index 100% rename from server/src/main/resources/static/config.html rename to server/src/main/resources/static/old-console/config.html diff --git a/server/src/main/resources/static/css/config.css b/server/src/main/resources/static/old-console/css/config.css similarity index 100% rename from server/src/main/resources/static/css/config.css rename to server/src/main/resources/static/old-console/css/config.css diff --git a/server/src/main/resources/static/css/demo_table.css b/server/src/main/resources/static/old-console/css/demo_table.css similarity index 100% rename from server/src/main/resources/static/css/demo_table.css rename to server/src/main/resources/static/old-console/css/demo_table.css diff --git a/server/src/main/resources/static/css/enable.css b/server/src/main/resources/static/old-console/css/enable.css similarity index 100% rename from server/src/main/resources/static/css/enable.css rename to server/src/main/resources/static/old-console/css/enable.css diff --git a/server/src/main/resources/static/css/images/ui-bg_flat_0_aaaaaa_40x100.png b/server/src/main/resources/static/old-console/css/images/ui-bg_flat_0_aaaaaa_40x100.png similarity index 100% rename from server/src/main/resources/static/css/images/ui-bg_flat_0_aaaaaa_40x100.png rename to server/src/main/resources/static/old-console/css/images/ui-bg_flat_0_aaaaaa_40x100.png diff --git a/server/src/main/resources/static/css/images/ui-bg_flat_75_ffffff_40x100.png b/server/src/main/resources/static/old-console/css/images/ui-bg_flat_75_ffffff_40x100.png similarity index 100% rename from server/src/main/resources/static/css/images/ui-bg_flat_75_ffffff_40x100.png rename to server/src/main/resources/static/old-console/css/images/ui-bg_flat_75_ffffff_40x100.png diff --git a/server/src/main/resources/static/css/images/ui-bg_glass_55_fbf9ee_1x400.png b/server/src/main/resources/static/old-console/css/images/ui-bg_glass_55_fbf9ee_1x400.png similarity index 100% rename from server/src/main/resources/static/css/images/ui-bg_glass_55_fbf9ee_1x400.png rename to server/src/main/resources/static/old-console/css/images/ui-bg_glass_55_fbf9ee_1x400.png diff --git a/server/src/main/resources/static/css/images/ui-bg_glass_65_ffffff_1x400.png b/server/src/main/resources/static/old-console/css/images/ui-bg_glass_65_ffffff_1x400.png similarity index 100% rename from server/src/main/resources/static/css/images/ui-bg_glass_65_ffffff_1x400.png rename to server/src/main/resources/static/old-console/css/images/ui-bg_glass_65_ffffff_1x400.png diff --git a/server/src/main/resources/static/css/images/ui-bg_glass_75_dadada_1x400.png b/server/src/main/resources/static/old-console/css/images/ui-bg_glass_75_dadada_1x400.png similarity index 100% rename from server/src/main/resources/static/css/images/ui-bg_glass_75_dadada_1x400.png rename to server/src/main/resources/static/old-console/css/images/ui-bg_glass_75_dadada_1x400.png diff --git a/server/src/main/resources/static/css/images/ui-bg_glass_75_e6e6e6_1x400.png b/server/src/main/resources/static/old-console/css/images/ui-bg_glass_75_e6e6e6_1x400.png similarity index 100% rename from server/src/main/resources/static/css/images/ui-bg_glass_75_e6e6e6_1x400.png rename to server/src/main/resources/static/old-console/css/images/ui-bg_glass_75_e6e6e6_1x400.png diff --git a/server/src/main/resources/static/css/images/ui-bg_glass_95_fef1ec_1x400.png b/server/src/main/resources/static/old-console/css/images/ui-bg_glass_95_fef1ec_1x400.png similarity index 100% rename from server/src/main/resources/static/css/images/ui-bg_glass_95_fef1ec_1x400.png rename to server/src/main/resources/static/old-console/css/images/ui-bg_glass_95_fef1ec_1x400.png diff --git a/server/src/main/resources/static/css/images/ui-bg_highlight-soft_75_cccccc_1x100.png b/server/src/main/resources/static/old-console/css/images/ui-bg_highlight-soft_75_cccccc_1x100.png similarity index 100% rename from server/src/main/resources/static/css/images/ui-bg_highlight-soft_75_cccccc_1x100.png rename to server/src/main/resources/static/old-console/css/images/ui-bg_highlight-soft_75_cccccc_1x100.png diff --git a/server/src/main/resources/static/css/images/ui-icons_222222_256x240.png b/server/src/main/resources/static/old-console/css/images/ui-icons_222222_256x240.png similarity index 100% rename from server/src/main/resources/static/css/images/ui-icons_222222_256x240.png rename to server/src/main/resources/static/old-console/css/images/ui-icons_222222_256x240.png diff --git a/server/src/main/resources/static/css/images/ui-icons_2e83ff_256x240.png b/server/src/main/resources/static/old-console/css/images/ui-icons_2e83ff_256x240.png similarity index 100% rename from server/src/main/resources/static/css/images/ui-icons_2e83ff_256x240.png rename to server/src/main/resources/static/old-console/css/images/ui-icons_2e83ff_256x240.png diff --git a/server/src/main/resources/static/css/images/ui-icons_454545_256x240.png b/server/src/main/resources/static/old-console/css/images/ui-icons_454545_256x240.png similarity index 100% rename from server/src/main/resources/static/css/images/ui-icons_454545_256x240.png rename to server/src/main/resources/static/old-console/css/images/ui-icons_454545_256x240.png diff --git a/server/src/main/resources/static/css/images/ui-icons_888888_256x240.png b/server/src/main/resources/static/old-console/css/images/ui-icons_888888_256x240.png similarity index 100% rename from server/src/main/resources/static/css/images/ui-icons_888888_256x240.png rename to server/src/main/resources/static/old-console/css/images/ui-icons_888888_256x240.png diff --git a/server/src/main/resources/static/css/images/ui-icons_cd0a0a_256x240.png b/server/src/main/resources/static/old-console/css/images/ui-icons_cd0a0a_256x240.png similarity index 100% rename from server/src/main/resources/static/css/images/ui-icons_cd0a0a_256x240.png rename to server/src/main/resources/static/old-console/css/images/ui-icons_cd0a0a_256x240.png diff --git a/server/src/main/resources/static/css/index.css b/server/src/main/resources/static/old-console/css/index.css similarity index 100% rename from server/src/main/resources/static/css/index.css rename to server/src/main/resources/static/old-console/css/index.css diff --git a/server/src/main/resources/static/css/jquery-ui-1.9.2.css b/server/src/main/resources/static/old-console/css/jquery-ui-1.9.2.css similarity index 100% rename from server/src/main/resources/static/css/jquery-ui-1.9.2.css rename to server/src/main/resources/static/old-console/css/jquery-ui-1.9.2.css diff --git a/server/src/main/resources/static/css/rules.css b/server/src/main/resources/static/old-console/css/rules.css similarity index 100% rename from server/src/main/resources/static/css/rules.css rename to server/src/main/resources/static/old-console/css/rules.css diff --git a/server/src/main/resources/static/css/style.css b/server/src/main/resources/static/old-console/css/style.css similarity index 100% rename from server/src/main/resources/static/css/style.css rename to server/src/main/resources/static/old-console/css/style.css diff --git a/server/src/main/resources/static/enable.html b/server/src/main/resources/static/old-console/enable.html similarity index 100% rename from server/src/main/resources/static/enable.html rename to server/src/main/resources/static/old-console/enable.html diff --git a/server/src/main/resources/static/images/back_disabled.jpg b/server/src/main/resources/static/old-console/images/back_disabled.jpg similarity index 100% rename from server/src/main/resources/static/images/back_disabled.jpg rename to server/src/main/resources/static/old-console/images/back_disabled.jpg diff --git a/server/src/main/resources/static/images/back_enabled.jpg b/server/src/main/resources/static/old-console/images/back_enabled.jpg similarity index 100% rename from server/src/main/resources/static/images/back_enabled.jpg rename to server/src/main/resources/static/old-console/images/back_enabled.jpg diff --git a/server/src/main/resources/static/images/favicon.ico b/server/src/main/resources/static/old-console/images/favicon.ico similarity index 100% rename from server/src/main/resources/static/images/favicon.ico rename to server/src/main/resources/static/old-console/images/favicon.ico diff --git a/server/src/main/resources/static/images/forward_disabled.jpg b/server/src/main/resources/static/old-console/images/forward_disabled.jpg similarity index 100% rename from server/src/main/resources/static/images/forward_disabled.jpg rename to server/src/main/resources/static/old-console/images/forward_disabled.jpg diff --git a/server/src/main/resources/static/images/forward_enabled.jpg b/server/src/main/resources/static/old-console/images/forward_enabled.jpg similarity index 100% rename from server/src/main/resources/static/images/forward_enabled.jpg rename to server/src/main/resources/static/old-console/images/forward_enabled.jpg diff --git a/server/src/main/resources/static/images/sort_asc.png b/server/src/main/resources/static/old-console/images/sort_asc.png similarity index 100% rename from server/src/main/resources/static/images/sort_asc.png rename to server/src/main/resources/static/old-console/images/sort_asc.png diff --git a/server/src/main/resources/static/images/sort_asc_disabled.png b/server/src/main/resources/static/old-console/images/sort_asc_disabled.png similarity index 100% rename from server/src/main/resources/static/images/sort_asc_disabled.png rename to server/src/main/resources/static/old-console/images/sort_asc_disabled.png diff --git a/server/src/main/resources/static/images/sort_both.png b/server/src/main/resources/static/old-console/images/sort_both.png similarity index 100% rename from server/src/main/resources/static/images/sort_both.png rename to server/src/main/resources/static/old-console/images/sort_both.png diff --git a/server/src/main/resources/static/images/sort_desc.png b/server/src/main/resources/static/old-console/images/sort_desc.png similarity index 100% rename from server/src/main/resources/static/images/sort_desc.png rename to server/src/main/resources/static/old-console/images/sort_desc.png diff --git a/server/src/main/resources/static/images/sort_desc_disabled.png b/server/src/main/resources/static/old-console/images/sort_desc_disabled.png similarity index 100% rename from server/src/main/resources/static/images/sort_desc_disabled.png rename to server/src/main/resources/static/old-console/images/sort_desc_disabled.png diff --git a/server/src/main/resources/static/index.html b/server/src/main/resources/static/old-console/index.html similarity index 100% rename from server/src/main/resources/static/index.html rename to server/src/main/resources/static/old-console/index.html diff --git a/server/src/main/resources/static/js/config-0.0.2.js b/server/src/main/resources/static/old-console/js/config-0.0.2.js similarity index 100% rename from server/src/main/resources/static/js/config-0.0.2.js rename to server/src/main/resources/static/old-console/js/config-0.0.2.js diff --git a/server/src/main/resources/static/js/druidTable-0.0.1.js b/server/src/main/resources/static/old-console/js/druidTable-0.0.1.js similarity index 100% rename from server/src/main/resources/static/js/druidTable-0.0.1.js rename to server/src/main/resources/static/old-console/js/druidTable-0.0.1.js diff --git a/server/src/main/resources/static/js/enable-0.0.1.js b/server/src/main/resources/static/old-console/js/enable-0.0.1.js similarity index 100% rename from server/src/main/resources/static/js/enable-0.0.1.js rename to server/src/main/resources/static/old-console/js/enable-0.0.1.js diff --git a/server/src/main/resources/static/js/handlers-0.0.2.js b/server/src/main/resources/static/old-console/js/handlers-0.0.2.js similarity index 100% rename from server/src/main/resources/static/js/handlers-0.0.2.js rename to server/src/main/resources/static/old-console/js/handlers-0.0.2.js diff --git a/server/src/main/resources/static/js/init-0.0.2.js b/server/src/main/resources/static/old-console/js/init-0.0.2.js similarity index 100% rename from server/src/main/resources/static/js/init-0.0.2.js rename to server/src/main/resources/static/old-console/js/init-0.0.2.js diff --git a/server/src/main/resources/static/js/jquery-1.11.0.min.js b/server/src/main/resources/static/old-console/js/jquery-1.11.0.min.js similarity index 100% rename from server/src/main/resources/static/js/jquery-1.11.0.min.js rename to server/src/main/resources/static/old-console/js/jquery-1.11.0.min.js diff --git a/server/src/main/resources/static/js/jquery-ui-1.9.2.js b/server/src/main/resources/static/old-console/js/jquery-ui-1.9.2.js similarity index 100% rename from server/src/main/resources/static/js/jquery-ui-1.9.2.js rename to server/src/main/resources/static/old-console/js/jquery-ui-1.9.2.js diff --git a/server/src/main/resources/static/js/jquery.dataTables-1.8.2.js b/server/src/main/resources/static/old-console/js/jquery.dataTables-1.8.2.js similarity index 100% rename from server/src/main/resources/static/js/jquery.dataTables-1.8.2.js rename to server/src/main/resources/static/old-console/js/jquery.dataTables-1.8.2.js diff --git a/server/src/main/resources/static/js/kill-0.0.1.js b/server/src/main/resources/static/old-console/js/kill-0.0.1.js similarity index 100% rename from server/src/main/resources/static/js/kill-0.0.1.js rename to server/src/main/resources/static/old-console/js/kill-0.0.1.js diff --git a/server/src/main/resources/static/js/rules-0.0.2.js b/server/src/main/resources/static/old-console/js/rules-0.0.2.js similarity index 100% rename from server/src/main/resources/static/js/rules-0.0.2.js rename to server/src/main/resources/static/old-console/js/rules-0.0.2.js diff --git a/server/src/main/resources/static/js/tablehelper-0.0.2.js b/server/src/main/resources/static/old-console/js/tablehelper-0.0.2.js similarity index 100% rename from server/src/main/resources/static/js/tablehelper-0.0.2.js rename to server/src/main/resources/static/old-console/js/tablehelper-0.0.2.js diff --git a/server/src/main/resources/static/js/underscore-1.2.2.js b/server/src/main/resources/static/old-console/js/underscore-1.2.2.js similarity index 100% rename from server/src/main/resources/static/js/underscore-1.2.2.js rename to server/src/main/resources/static/old-console/js/underscore-1.2.2.js diff --git a/server/src/main/resources/static/kill.html b/server/src/main/resources/static/old-console/kill.html similarity index 100% rename from server/src/main/resources/static/kill.html rename to server/src/main/resources/static/old-console/kill.html diff --git a/server/src/main/resources/static/rules.html b/server/src/main/resources/static/old-console/rules.html similarity index 100% rename from server/src/main/resources/static/rules.html rename to server/src/main/resources/static/old-console/rules.html diff --git a/server/src/main/resources/static/view.html b/server/src/main/resources/static/old-console/view.html similarity index 100% rename from server/src/main/resources/static/view.html rename to server/src/main/resources/static/old-console/view.html From ae2a09ab5e24c7768e69f80793f5d6eaf03e424f Mon Sep 17 00:00:00 2001 From: Tarek Rached Date: Tue, 9 Sep 2014 12:44:57 -0700 Subject: [PATCH 083/107] added new console --- .../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 | 14 + .../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 + 18 files changed, 22344 insertions(+) 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 create mode 100644 server/src/main/resources/static/index.html 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 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("