diff --git a/benchmarks/src/main/java/io/druid/benchmark/DimensionPredicateFilterBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/DimensionPredicateFilterBenchmark.java index 48799cd01676..0a22e8c1267f 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/DimensionPredicateFilterBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/DimensionPredicateFilterBenchmark.java @@ -29,6 +29,7 @@ import io.druid.collections.bitmap.RoaringBitmapFactory; import io.druid.collections.spatial.ImmutableRTree; import io.druid.query.filter.BitmapIndexSelector; +import io.druid.query.filter.DruidFloatPredicate; import io.druid.query.filter.DruidLongPredicate; import io.druid.query.filter.DruidPredicateFactory; import io.druid.segment.column.BitmapIndex; @@ -86,14 +87,13 @@ public boolean apply(String input) @Override public DruidLongPredicate makeLongPredicate() { - return new DruidLongPredicate() - { - @Override - public boolean applyLong(long input) - { - return false; - } - }; + return DruidLongPredicate.ALWAYS_FALSE; + } + + @Override + public DruidFloatPredicate makeFloatPredicate() + { + return DruidFloatPredicate.ALWAYS_FALSE; } }, null diff --git a/benchmarks/src/main/java/io/druid/benchmark/FilterPartitionBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/FilterPartitionBenchmark.java index 825940f221c2..a3da969c812e 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/FilterPartitionBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/FilterPartitionBenchmark.java @@ -46,6 +46,7 @@ import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.BoundDimFilter; import io.druid.query.filter.DimFilter; +import io.druid.query.filter.DruidFloatPredicate; import io.druid.query.filter.DruidLongPredicate; import io.druid.query.filter.DruidPredicateFactory; import io.druid.query.filter.Filter; @@ -626,14 +627,13 @@ public boolean apply(String input) @Override public DruidLongPredicate makeLongPredicate() { - return new DruidLongPredicate() - { - @Override - public boolean applyLong(long input) - { - return false; - } - }; + return DruidLongPredicate.ALWAYS_FALSE; + } + + @Override + public DruidFloatPredicate makeFloatPredicate() + { + return DruidFloatPredicate.ALWAYS_FALSE; } }; diff --git a/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java new file mode 100644 index 000000000000..e7d46cdd99e0 --- /dev/null +++ b/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java @@ -0,0 +1,860 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.benchmark; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.hash.Hashing; +import com.google.common.io.Files; +import io.druid.benchmark.datagen.BenchmarkDataGenerator; +import io.druid.benchmark.datagen.BenchmarkSchemaInfo; +import io.druid.benchmark.datagen.BenchmarkSchemas; +import io.druid.benchmark.query.QueryBenchmarkUtil; +import io.druid.collections.BlockingPool; +import io.druid.collections.StupidPool; +import io.druid.concurrent.Execs; +import io.druid.data.input.InputRow; +import io.druid.data.input.Row; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.granularity.QueryGranularities; +import io.druid.granularity.QueryGranularity; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.Sequences; +import io.druid.java.util.common.logger.Logger; +import io.druid.offheap.OffheapBufferGenerator; +import io.druid.query.DruidProcessingConfig; +import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.groupby.GroupByQueryConfig; +import io.druid.query.groupby.GroupByQueryEngine; +import io.druid.query.groupby.GroupByQueryQueryToolChest; +import io.druid.query.groupby.GroupByQueryRunnerFactory; +import io.druid.query.groupby.strategy.GroupByStrategySelector; +import io.druid.query.groupby.strategy.GroupByStrategyV1; +import io.druid.query.groupby.strategy.GroupByStrategyV2; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.query.spec.QuerySegmentSpec; +import io.druid.segment.IndexIO; +import io.druid.segment.IndexMergerV9; +import io.druid.segment.IndexSpec; +import io.druid.segment.QueryableIndex; +import io.druid.segment.QueryableIndexSegment; +import io.druid.segment.column.ColumnConfig; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.segment.incremental.OnheapIncrementalIndex; +import io.druid.segment.serde.ComplexMetrics; +import org.apache.commons.io.FileUtils; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; + +// Benchmark for determining the interface overhead of GroupBy with multiple type implementations + +@State(Scope.Benchmark) +@Fork(jvmArgsPrepend = "-server", value = 1) +@Warmup(iterations = 15) +@Measurement(iterations = 30) +public class GroupByTypeInterfaceBenchmark +{ + @Param({"4"}) + private int numSegments; + + @Param({"4"}) + private int numProcessingThreads; + + @Param({"-1"}) + private int initialBuckets; + + @Param({"100000"}) + private int rowsPerSegment; + + @Param({"v2"}) + private String defaultStrategy; + + @Param({"all"}) + private String queryGranularity; + + private static final Logger log = new Logger(GroupByTypeInterfaceBenchmark.class); + private static final int RNG_SEED = 9999; + private static final IndexMergerV9 INDEX_MERGER_V9; + private static final IndexIO INDEX_IO; + public static final ObjectMapper JSON_MAPPER; + + private File tmpDir; + private IncrementalIndex anIncrementalIndex; + private List queryableIndexes; + + private QueryRunnerFactory factory; + + private BenchmarkSchemaInfo schemaInfo; + private GroupByQuery stringQuery; + private GroupByQuery longFloatQuery; + private GroupByQuery floatQuery; + private GroupByQuery longQuery; + + private ExecutorService executorService; + + static { + JSON_MAPPER = new DefaultObjectMapper(); + INDEX_IO = new IndexIO( + JSON_MAPPER, + new ColumnConfig() + { + @Override + public int columnCacheSizeBytes() + { + return 0; + } + } + ); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + } + + private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); + + private void setupQueries() + { + // queries for the basic schema + Map basicQueries = new LinkedHashMap<>(); + BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + + { // basic.A + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + List queryAggs = new ArrayList<>(); + queryAggs.add(new LongSumAggregatorFactory( + "sumLongSequential", + "sumLongSequential" + )); + GroupByQuery queryString = GroupByQuery + .builder() + .setDataSource("blah") + .setQuerySegmentSpec(intervalSpec) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("dimSequential", null) + )) + .setAggregatorSpecs( + queryAggs + ) + .setGranularity(QueryGranularity.fromString(queryGranularity)) + .build(); + + GroupByQuery queryLongFloat = GroupByQuery + .builder() + .setDataSource("blah") + .setQuerySegmentSpec(intervalSpec) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("metLongUniform", null), + new DefaultDimensionSpec("metFloatNormal", null) + )) + .setAggregatorSpecs( + queryAggs + ) + .setGranularity(QueryGranularity.fromString(queryGranularity)) + .build(); + + GroupByQuery queryLong = GroupByQuery + .builder() + .setDataSource("blah") + .setQuerySegmentSpec(intervalSpec) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("metLongUniform", null) + )) + .setAggregatorSpecs( + queryAggs + ) + .setGranularity(QueryGranularity.fromString(queryGranularity)) + .build(); + + GroupByQuery queryFloat = GroupByQuery + .builder() + .setDataSource("blah") + .setQuerySegmentSpec(intervalSpec) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("metFloatNormal", null) + )) + .setAggregatorSpecs( + queryAggs + ) + .setGranularity(QueryGranularity.fromString(queryGranularity)) + .build(); + + basicQueries.put("string", queryString); + basicQueries.put("longFloat", queryLongFloat); + basicQueries.put("long", queryLong); + basicQueries.put("float", queryFloat); + } + + { // basic.nested + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + List queryAggs = new ArrayList<>(); + queryAggs.add(new LongSumAggregatorFactory( + "sumLongSequential", + "sumLongSequential" + )); + + GroupByQuery subqueryA = GroupByQuery + .builder() + .setDataSource("blah") + .setQuerySegmentSpec(intervalSpec) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("dimSequential", null), + new DefaultDimensionSpec("dimZipf", null) + )) + .setAggregatorSpecs( + queryAggs + ) + .setGranularity(QueryGranularities.DAY) + .build(); + + GroupByQuery queryA = GroupByQuery + .builder() + .setDataSource(subqueryA) + .setQuerySegmentSpec(intervalSpec) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("dimSequential", null) + )) + .setAggregatorSpecs( + queryAggs + ) + .setGranularity(QueryGranularities.WEEK) + .build(); + + basicQueries.put("nested", queryA); + } + + SCHEMA_QUERY_MAP.put("basic", basicQueries); + } + + @Setup(Level.Trial) + public void setup() throws IOException + { + log.info("SETUP CALLED AT %d", System.currentTimeMillis()); + + if (ComplexMetrics.getSerdeForType("hyperUnique") == null) { + ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(Hashing.murmur3_128())); + } + executorService = Execs.multiThreaded(numProcessingThreads, "GroupByThreadPool[%d]"); + + setupQueries(); + + String schemaName = "basic"; + + schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schemaName); + stringQuery = SCHEMA_QUERY_MAP.get(schemaName).get("string"); + longFloatQuery = SCHEMA_QUERY_MAP.get(schemaName).get("longFloat"); + longQuery = SCHEMA_QUERY_MAP.get(schemaName).get("long"); + floatQuery = SCHEMA_QUERY_MAP.get(schemaName).get("float"); + + final BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator( + schemaInfo.getColumnSchemas(), + RNG_SEED + 1, + schemaInfo.getDataInterval(), + rowsPerSegment + ); + + tmpDir = Files.createTempDir(); + log.info("Using temp dir: %s", tmpDir.getAbsolutePath()); + + // queryableIndexes -> numSegments worth of on-disk segments + // anIncrementalIndex -> the last incremental index + anIncrementalIndex = null; + queryableIndexes = new ArrayList<>(numSegments); + + for (int i = 0; i < numSegments; i++) { + log.info("Generating rows for segment %d/%d", i + 1, numSegments); + + final IncrementalIndex index = makeIncIndex(); + + for (int j = 0; j < rowsPerSegment; j++) { + final InputRow row = dataGenerator.nextRow(); + if (j % 20000 == 0) { + log.info("%,d/%,d rows generated.", i * rowsPerSegment + j, rowsPerSegment * numSegments); + } + index.add(row); + } + + log.info( + "%,d/%,d rows generated, persisting segment %d/%d.", + (i + 1) * rowsPerSegment, + rowsPerSegment * numSegments, + i + 1, + numSegments + ); + + final File file = INDEX_MERGER_V9.persist( + index, + new File(tmpDir, String.valueOf(i)), + new IndexSpec() + ); + + queryableIndexes.add(INDEX_IO.loadIndex(file)); + + if (i == numSegments - 1) { + anIncrementalIndex = index; + } else { + index.close(); + } + } + + StupidPool bufferPool = new StupidPool<>( + "GroupByBenchmark-computeBufferPool", + new OffheapBufferGenerator("compute", 250_000_000), + 0, + Integer.MAX_VALUE + ); + + // limit of 2 is required since we simulate both historical merge and broker merge in the same process + BlockingPool mergePool = new BlockingPool<>( + new OffheapBufferGenerator("merge", 250_000_000), + 2 + ); + final GroupByQueryConfig config = new GroupByQueryConfig() + { + @Override + public String getDefaultStrategy() + { + return defaultStrategy; + } + + @Override + public int getBufferGrouperInitialBuckets() + { + return initialBuckets; + } + + @Override + public long getMaxOnDiskStorage() + { + return 1_000_000_000L; + } + }; + config.setSingleThreaded(false); + config.setMaxIntermediateRows(Integer.MAX_VALUE); + config.setMaxResults(Integer.MAX_VALUE); + + DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig() + { + @Override + public int getNumThreads() + { + // Used by "v2" strategy for concurrencyHint + return numProcessingThreads; + } + + @Override + public String getFormatString() + { + return null; + } + }; + + final Supplier configSupplier = Suppliers.ofInstance(config); + final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + configSupplier, + new GroupByStrategyV1( + configSupplier, + new GroupByQueryEngine(configSupplier, bufferPool), + QueryBenchmarkUtil.NOOP_QUERYWATCHER, + bufferPool + ), + new GroupByStrategyV2( + druidProcessingConfig, + configSupplier, + bufferPool, + mergePool, + new ObjectMapper(new SmileFactory()), + QueryBenchmarkUtil.NOOP_QUERYWATCHER + ) + ); + + factory = new GroupByQueryRunnerFactory( + strategySelector, + new GroupByQueryQueryToolChest( + configSupplier, + strategySelector, + bufferPool, + QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator() + ) + ); + } + + private IncrementalIndex makeIncIndex() + { + return new OnheapIncrementalIndex( + new IncrementalIndexSchema.Builder() + .withQueryGranularity(QueryGranularities.NONE) + .withMetrics(schemaInfo.getAggsArray()) + .withDimensionsSpec(new DimensionsSpec(null, null, null)) + .build(), + true, + false, + true, + rowsPerSegment + ); + } + + @TearDown(Level.Trial) + public void tearDown() + { + try { + if (anIncrementalIndex != null) { + anIncrementalIndex.close(); + } + + if (queryableIndexes != null) { + for (QueryableIndex index : queryableIndexes) { + index.close(); + } + } + + if (tmpDir != null) { + FileUtils.deleteDirectory(tmpDir); + } + } + catch (IOException e) { + log.warn(e, "Failed to tear down, temp dir was: %s", tmpDir); + throw Throwables.propagate(e); + } + } + + private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) + { + QueryToolChest toolChest = factory.getToolchest(); + QueryRunner theRunner = new FinalizeResultsQueryRunner<>( + toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)), + toolChest + ); + + Sequence queryResult = theRunner.run(query, Maps.newHashMap()); + return Sequences.toList(queryResult, Lists.newArrayList()); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexStringOnly(Blackhole blackhole) throws Exception + { + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + + for (Row result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexLongOnly(Blackhole blackhole) throws Exception + { + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + + for (Row result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexFloatOnly(Blackhole blackhole) throws Exception + { + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + + for (Row result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexNumericOnly(Blackhole blackhole) throws Exception + { + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery); + + for (Row result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexNumericThenString(Blackhole blackhole) throws Exception + { + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery); + + for (Row result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + + for (Row result : results) { + blackhole.consume(result); + } + } + + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexLongThenString(Blackhole blackhole) throws Exception + { + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + + for (Row result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + + for (Row result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexLongThenFloat(Blackhole blackhole) throws Exception + { + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + + for (Row result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + + for (Row result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexStringThenNumeric(Blackhole blackhole) throws Exception + { + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + + for (Row result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery); + + for (Row result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexStringThenLong(Blackhole blackhole) throws Exception + { + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + + for (Row result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + + for (Row result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexStringTwice(Blackhole blackhole) throws Exception + { + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + + for (Row result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + + for (Row result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexLongTwice(Blackhole blackhole) throws Exception + { + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + + for (Row result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + + for (Row result : results) { + blackhole.consume(result); + } + } + + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexFloatTwice(Blackhole blackhole) throws Exception + { + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + + for (Row result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + + for (Row result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexFloatThenLong(Blackhole blackhole) throws Exception + { + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + + for (Row result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + + for (Row result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexFloatThenString(Blackhole blackhole) throws Exception + { + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + List results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + + for (Row result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", queryableIndexes.get(0)) + ); + + results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + + for (Row result : results) { + blackhole.consume(result); + } + } +} diff --git a/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java new file mode 100644 index 000000000000..689d8db688f5 --- /dev/null +++ b/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java @@ -0,0 +1,645 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.benchmark; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.hash.Hashing; +import com.google.common.io.Files; +import io.druid.benchmark.datagen.BenchmarkDataGenerator; +import io.druid.benchmark.datagen.BenchmarkSchemaInfo; +import io.druid.benchmark.datagen.BenchmarkSchemas; +import io.druid.benchmark.query.QueryBenchmarkUtil; +import io.druid.collections.StupidPool; +import io.druid.concurrent.Execs; +import io.druid.data.input.InputRow; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.granularity.QueryGranularities; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.Sequences; +import io.druid.java.util.common.logger.Logger; +import io.druid.offheap.OffheapBufferGenerator; +import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; +import io.druid.query.Result; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.DoubleMinAggregatorFactory; +import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.LongMaxAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; +import io.druid.query.dimension.ExtractionDimensionSpec; +import io.druid.query.extraction.IdentityExtractionFn; +import io.druid.query.ordering.StringComparators; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.query.spec.QuerySegmentSpec; +import io.druid.query.topn.DimensionTopNMetricSpec; +import io.druid.query.topn.TopNQuery; +import io.druid.query.topn.TopNQueryBuilder; +import io.druid.query.topn.TopNQueryConfig; +import io.druid.query.topn.TopNQueryQueryToolChest; +import io.druid.query.topn.TopNQueryRunnerFactory; +import io.druid.query.topn.TopNResultValue; +import io.druid.segment.IndexIO; +import io.druid.segment.IndexMergerV9; +import io.druid.segment.IndexSpec; +import io.druid.segment.QueryableIndex; +import io.druid.segment.QueryableIndexSegment; +import io.druid.segment.column.ColumnConfig; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.segment.incremental.OnheapIncrementalIndex; +import io.druid.segment.serde.ComplexMetrics; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; + +// Benchmark for determining the interface overhead of TopN with multiple type implementations + +@State(Scope.Benchmark) +@Fork(jvmArgsPrepend = "-server", value = 1) +@Warmup(iterations = 10) +@Measurement(iterations = 25) +public class TopNTypeInterfaceBenchmark +{ + @Param({"1"}) + private int numSegments; + + @Param({"750000"}) + private int rowsPerSegment; + + @Param({"basic.A"}) + private String schemaAndQuery; + + @Param({"10"}) + private int threshold; + + private static final Logger log = new Logger(TopNTypeInterfaceBenchmark.class); + private static final int RNG_SEED = 9999; + private static final IndexMergerV9 INDEX_MERGER_V9; + private static final IndexIO INDEX_IO; + public static final ObjectMapper JSON_MAPPER; + + private List incIndexes; + private List qIndexes; + + private QueryRunnerFactory factory; + private BenchmarkSchemaInfo schemaInfo; + private TopNQueryBuilder queryBuilder; + private TopNQuery stringQuery; + private TopNQuery longQuery; + private TopNQuery floatQuery; + + private ExecutorService executorService; + + static { + JSON_MAPPER = new DefaultObjectMapper(); + INDEX_IO = new IndexIO( + JSON_MAPPER, + new ColumnConfig() + { + @Override + public int columnCacheSizeBytes() + { + return 0; + } + } + ); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + } + + private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); + + private void setupQueries() + { + // queries for the basic schema + Map basicQueries = new LinkedHashMap<>(); + BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + + { // basic.A + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + + List queryAggs = new ArrayList<>(); + queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); + queryAggs.add(new LongMaxAggregatorFactory("maxLongUniform", "maxLongUniform")); + queryAggs.add(new DoubleSumAggregatorFactory("sumFloatNormal", "sumFloatNormal")); + queryAggs.add(new DoubleMinAggregatorFactory("minFloatZipf", "minFloatZipf")); + queryAggs.add(new HyperUniquesAggregatorFactory("hyperUniquesMet", "hyper")); + + // Use an IdentityExtractionFn to force usage of DimExtractionTopNAlgorithm + TopNQueryBuilder queryBuilderString = new TopNQueryBuilder() + .dataSource("blah") + .granularity(QueryGranularities.ALL) + .dimension(new ExtractionDimensionSpec("dimSequential", "dimSequential", IdentityExtractionFn.getInstance())) + .metric("sumFloatNormal") + .intervals(intervalSpec) + .aggregators(queryAggs); + + // DimExtractionTopNAlgorithm is always used for numeric columns + TopNQueryBuilder queryBuilderLong = new TopNQueryBuilder() + .dataSource("blah") + .granularity(QueryGranularities.ALL) + .dimension("metLongUniform") + .metric("sumFloatNormal") + .intervals(intervalSpec) + .aggregators(queryAggs); + + TopNQueryBuilder queryBuilderFloat = new TopNQueryBuilder() + .dataSource("blah") + .granularity(QueryGranularities.ALL) + .dimension("metFloatNormal") + .metric("sumFloatNormal") + .intervals(intervalSpec) + .aggregators(queryAggs); + + basicQueries.put("string", queryBuilderString); + basicQueries.put("long", queryBuilderLong); + basicQueries.put("float", queryBuilderFloat); + } + { // basic.numericSort + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + + List queryAggs = new ArrayList<>(); + queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); + + TopNQueryBuilder queryBuilderA = new TopNQueryBuilder() + .dataSource("blah") + .granularity(QueryGranularities.ALL) + .dimension("dimUniform") + .metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC)) + .intervals(intervalSpec) + .aggregators(queryAggs); + + basicQueries.put("numericSort", queryBuilderA); + } + { // basic.alphanumericSort + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + + List queryAggs = new ArrayList<>(); + queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); + + TopNQueryBuilder queryBuilderA = new TopNQueryBuilder() + .dataSource("blah") + .granularity(QueryGranularities.ALL) + .dimension("dimUniform") + .metric(new DimensionTopNMetricSpec(null, StringComparators.ALPHANUMERIC)) + .intervals(intervalSpec) + .aggregators(queryAggs); + + basicQueries.put("alphanumericSort", queryBuilderA); + } + + SCHEMA_QUERY_MAP.put("basic", basicQueries); + } + + + @Setup + public void setup() throws IOException + { + log.info("SETUP CALLED AT " + System.currentTimeMillis()); + + if (ComplexMetrics.getSerdeForType("hyperUnique") == null) { + ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(Hashing.murmur3_128())); + } + + executorService = Execs.multiThreaded(numSegments, "TopNThreadPool"); + + setupQueries(); + + schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + queryBuilder = SCHEMA_QUERY_MAP.get("basic").get("string"); + queryBuilder.threshold(threshold); + stringQuery = queryBuilder.build(); + + TopNQueryBuilder longBuilder = SCHEMA_QUERY_MAP.get("basic").get("long"); + longBuilder.threshold(threshold); + longQuery = longBuilder.build(); + + TopNQueryBuilder floatBuilder = SCHEMA_QUERY_MAP.get("basic").get("float"); + floatBuilder.threshold(threshold); + floatQuery = floatBuilder.build(); + + incIndexes = new ArrayList<>(); + for (int i = 0; i < numSegments; i++) { + log.info("Generating rows for segment " + i); + + BenchmarkDataGenerator gen = new BenchmarkDataGenerator( + schemaInfo.getColumnSchemas(), + RNG_SEED + i, + schemaInfo.getDataInterval(), + rowsPerSegment + ); + + IncrementalIndex incIndex = makeIncIndex(); + + for (int j = 0; j < rowsPerSegment; j++) { + InputRow row = gen.nextRow(); + if (j % 10000 == 0) { + log.info(j + " rows generated."); + } + incIndex.add(row); + } + incIndexes.add(incIndex); + } + + File tmpFile = Files.createTempDir(); + log.info("Using temp dir: " + tmpFile.getAbsolutePath()); + tmpFile.deleteOnExit(); + + qIndexes = new ArrayList<>(); + for (int i = 0; i < numSegments; i++) { + File indexFile = INDEX_MERGER_V9.persist( + incIndexes.get(i), + tmpFile, + new IndexSpec() + ); + + QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile); + qIndexes.add(qIndex); + } + + factory = new TopNQueryRunnerFactory( + new StupidPool<>( + "TopNBenchmark-compute-bufferPool", + new OffheapBufferGenerator("compute", 250000000), + 0, + Integer.MAX_VALUE + ), + new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()), + QueryBenchmarkUtil.NOOP_QUERYWATCHER + ); + } + + private IncrementalIndex makeIncIndex() + { + return new OnheapIncrementalIndex( + new IncrementalIndexSchema.Builder() + .withQueryGranularity(QueryGranularities.NONE) + .withMetrics(schemaInfo.getAggsArray()) + .withDimensionsSpec(new DimensionsSpec(null, null, null)) + .build(), + true, + false, + true, + rowsPerSegment + ); + } + + private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) + { + + QueryToolChest toolChest = factory.getToolchest(); + QueryRunner theRunner = new FinalizeResultsQueryRunner<>( + toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)), + toolChest + ); + + Sequence queryResult = theRunner.run(query, Maps.newHashMap()); + return Sequences.toList(queryResult, Lists.newArrayList()); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexStringOnly(Blackhole blackhole) throws Exception + { + QueryRunner> runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + List> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + for (Result result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexStringTwice(Blackhole blackhole) throws Exception + { + QueryRunner> runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + List> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + for (Result result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + for (Result result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexStringThenLong(Blackhole blackhole) throws Exception + { + QueryRunner> runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + List> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + for (Result result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + for (Result result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexStringThenFloat(Blackhole blackhole) throws Exception + { + QueryRunner> runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + List> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + for (Result result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + for (Result result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexLongOnly(Blackhole blackhole) throws Exception + { + QueryRunner> runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + List> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + for (Result result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexLongTwice(Blackhole blackhole) throws Exception + { + QueryRunner> runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + List> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + for (Result result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + for (Result result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexLongThenString(Blackhole blackhole) throws Exception + { + QueryRunner> runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + List> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + for (Result result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + for (Result result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexLongThenFloat(Blackhole blackhole) throws Exception + { + QueryRunner> runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + List> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + for (Result result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + for (Result result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexFloatOnly(Blackhole blackhole) throws Exception + { + QueryRunner> runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + List> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + for (Result result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexFloatTwice(Blackhole blackhole) throws Exception + { + QueryRunner> runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + List> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + for (Result result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + for (Result result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexFloatThenString(Blackhole blackhole) throws Exception + { + QueryRunner> runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + List> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + for (Result result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery); + for (Result result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndexFloatThenLong(Blackhole blackhole) throws Exception + { + QueryRunner> runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + List> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery); + for (Result result : results) { + blackhole.consume(result); + } + + runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + "qIndex", + new QueryableIndexSegment("qIndex", qIndexes.get(0)) + ); + + results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery); + for (Result result : results) { + blackhole.consume(result); + } + } +} diff --git a/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkSchemaInfo.java b/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkSchemaInfo.java index bc0ff220a865..b83a4c9e46c6 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkSchemaInfo.java +++ b/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkSchemaInfo.java @@ -29,16 +29,19 @@ public class BenchmarkSchemaInfo private List columnSchemas; private List aggs; private Interval dataInterval; + private boolean withRollup; public BenchmarkSchemaInfo ( List columnSchemas, List aggs, - Interval dataInterval + Interval dataInterval, + boolean withRollup ) { this.columnSchemas = columnSchemas; this.aggs = aggs; this.dataInterval = dataInterval; + this.withRollup = withRollup; } public List getColumnSchemas() @@ -61,4 +64,8 @@ public Interval getDataInterval() return dataInterval; } + public boolean isWithRollup() + { + return withRollup; + } } diff --git a/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkSchemas.java b/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkSchemas.java index e2c8bc176372..f8b5da8dcc95 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkSchemas.java +++ b/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkSchemas.java @@ -89,8 +89,71 @@ public class BenchmarkSchemas BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo( basicSchemaColumns, basicSchemaIngestAggs, - basicSchemaDataInterval + basicSchemaDataInterval, + true ); SCHEMA_MAP.put("basic", basicSchema); } + + static { // simple single string column and count agg schema, no rollup + List basicSchemaColumns = ImmutableList.of( + // dims + BenchmarkColumnSchema.makeSequential("dimSequential", ValueType.STRING, false, 1, null, 0, 1000000) + ); + + List basicSchemaIngestAggs = new ArrayList<>(); + basicSchemaIngestAggs.add(new CountAggregatorFactory("rows")); + + Interval basicSchemaDataInterval = new Interval(0, 1000000); + + BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo( + basicSchemaColumns, + basicSchemaIngestAggs, + basicSchemaDataInterval, + false + ); + SCHEMA_MAP.put("simple", basicSchema); + } + + static { // simple single long column and count agg schema, no rollup + List basicSchemaColumns = ImmutableList.of( + // dims, ingest as a metric for now with rollup off, until numeric dims at ingestion are supported + BenchmarkColumnSchema.makeSequential("dimSequential", ValueType.LONG, true, 1, null, 0, 1000000) + ); + + List basicSchemaIngestAggs = new ArrayList<>(); + basicSchemaIngestAggs.add(new LongSumAggregatorFactory("dimSequential", "dimSequential")); + basicSchemaIngestAggs.add(new CountAggregatorFactory("rows")); + + Interval basicSchemaDataInterval = new Interval(0, 1000000); + + BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo( + basicSchemaColumns, + basicSchemaIngestAggs, + basicSchemaDataInterval, + false + ); + SCHEMA_MAP.put("simpleLong", basicSchema); + } + + static { // simple single float column and count agg schema, no rollup + List basicSchemaColumns = ImmutableList.of( + // dims, ingest as a metric for now with rollup off, until numeric dims at ingestion are supported + BenchmarkColumnSchema.makeSequential("dimSequential", ValueType.FLOAT, true, 1, null, 0, 1000000) + ); + + List basicSchemaIngestAggs = new ArrayList<>(); + basicSchemaIngestAggs.add(new DoubleSumAggregatorFactory("dimSequential", "dimSequential")); + basicSchemaIngestAggs.add(new CountAggregatorFactory("rows")); + + Interval basicSchemaDataInterval = new Interval(0, 1000000); + + BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo( + basicSchemaColumns, + basicSchemaIngestAggs, + basicSchemaDataInterval, + false + ); + SCHEMA_MAP.put("simpleFloat", basicSchema); + } } diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java index f59922400c94..eb42773136cd 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java @@ -73,6 +73,7 @@ import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; import io.druid.segment.column.ColumnConfig; +import io.druid.segment.column.ValueType; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.OnheapIncrementalIndex; @@ -237,8 +238,90 @@ private void setupQueries() basicQueries.put("nested", queryA); } - SCHEMA_QUERY_MAP.put("basic", basicQueries); + + // simple one column schema, for testing performance difference between querying on numeric values as Strings and + // directly as longs + Map simpleQueries = new LinkedHashMap<>(); + BenchmarkSchemaInfo simpleSchema = BenchmarkSchemas.SCHEMA_MAP.get("simple"); + + { // simple.A + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(simpleSchema.getDataInterval())); + List queryAggs = new ArrayList<>(); + queryAggs.add(new LongSumAggregatorFactory( + "rows", + "rows" + )); + GroupByQuery queryA = GroupByQuery + .builder() + .setDataSource("blah") + .setQuerySegmentSpec(intervalSpec) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.STRING) + )) + .setAggregatorSpecs( + queryAggs + ) + .setGranularity(QueryGranularity.fromString(queryGranularity)) + .build(); + + simpleQueries.put("A", queryA); + } + SCHEMA_QUERY_MAP.put("simple", simpleQueries); + + + Map simpleLongQueries = new LinkedHashMap<>(); + BenchmarkSchemaInfo simpleLongSchema = BenchmarkSchemas.SCHEMA_MAP.get("simpleLong"); + { // simpleLong.A + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(simpleLongSchema.getDataInterval())); + List queryAggs = new ArrayList<>(); + queryAggs.add(new LongSumAggregatorFactory( + "rows", + "rows" + )); + GroupByQuery queryA = GroupByQuery + .builder() + .setDataSource("blah") + .setQuerySegmentSpec(intervalSpec) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.LONG) + )) + .setAggregatorSpecs( + queryAggs + ) + .setGranularity(QueryGranularity.fromString(queryGranularity)) + .build(); + + simpleLongQueries.put("A", queryA); + } + SCHEMA_QUERY_MAP.put("simpleLong", simpleLongQueries); + + + Map simpleFloatQueries = new LinkedHashMap<>(); + BenchmarkSchemaInfo simpleFloatSchema = BenchmarkSchemas.SCHEMA_MAP.get("simpleFloat"); + { // simpleFloat.A + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(simpleFloatSchema.getDataInterval())); + List queryAggs = new ArrayList<>(); + queryAggs.add(new LongSumAggregatorFactory( + "rows", + "rows" + )); + GroupByQuery queryA = GroupByQuery + .builder() + .setDataSource("blah") + .setQuerySegmentSpec(intervalSpec) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.FLOAT) + )) + .setAggregatorSpecs( + queryAggs + ) + .setGranularity(QueryGranularity.fromString(queryGranularity)) + .build(); + + simpleFloatQueries.put("A", queryA); + } + SCHEMA_QUERY_MAP.put("simpleFloat", simpleFloatQueries); } @Setup(Level.Trial) @@ -278,7 +361,7 @@ public void setup() throws IOException for (int i = 0; i < numSegments; i++) { log.info("Generating rows for segment %d/%d", i + 1, numSegments); - final IncrementalIndex index = makeIncIndex(); + final IncrementalIndex index = makeIncIndex(schemaInfo.isWithRollup()); for (int j = 0; j < rowsPerSegment; j++) { final InputRow row = dataGenerator.nextRow(); @@ -393,13 +476,14 @@ public String getFormatString() ); } - private IncrementalIndex makeIncIndex() + private IncrementalIndex makeIncIndex(boolean withRollup) { return new OnheapIncrementalIndex( new IncrementalIndexSchema.Builder() .withQueryGranularity(QueryGranularities.NONE) .withMetrics(schemaInfo.getAggsArray()) .withDimensionsSpec(new DimensionsSpec(null, null, null)) + .withRollup(withRollup) .build(), true, false, diff --git a/docs/content/querying/dimensionspecs.md b/docs/content/querying/dimensionspecs.md index 3af1e961921d..f29ca6e89053 100644 --- a/docs/content/querying/dimensionspecs.md +++ b/docs/content/querying/dimensionspecs.md @@ -15,9 +15,18 @@ The following JSON fields can be used in a query to operate on dimension values. Returns dimension values as is and optionally renames the dimension. ```json -{ "type" : "default", "dimension" : , "outputName": } +{ + "type" : "default", + "dimension" : , + "outputName": , + "outputType": <"STRING"|"LONG"|"FLOAT"> +} ``` +When specifying a DimensionSpec on a numeric column, the user should include the type of the column in the `outputType` field. If left unspecified, the `outputType` defaults to STRING. + +Please refer to the [Output Types](#output-types) section for more details. + ### Extraction DimensionSpec Returns dimension values transformed using the given [extraction function](#extraction-functions). @@ -27,10 +36,15 @@ Returns dimension values transformed using the given [extraction function](#extr "type" : "extraction", "dimension" : , "outputName" : , + "outputType": <"STRING"|"LONG"|"FLOAT">, "extractionFn" : } ``` +`outputType` may also be specified in an ExtractionDimensionSpec to apply type conversion to results before merging. If left unspecified, the `outputType` defaults to STRING. + +Please refer to the [Output Types](#output-types) section for more details. + ### Filtered DimensionSpecs These are only useful for multi-value dimensions. If you have a row in druid that has a multi-value dimension with values ["v1", "v2", "v3"] and you send a groupBy/topN query grouping by that dimension with [query filter](filters.html) for value "v1". In the response you will get 3 rows containing "v1", "v2" and "v3". This behavior might be unintuitive for some use cases. @@ -96,6 +110,21 @@ The second kind where it is not possible to pass at query time due to their size } ``` +## Output Types + +The dimension specs provide an option to specify the output type of a column's values. This is necessary as it is possible for a column with given name to have different value types in different segments; results will be converted to the type specified by `outputType` before merging. + +Note that not all use cases for DimensionSpec currently support `outputType`, the table below shows which use cases support this option: + +|Query Type|Supported?| +|--------|---------| +|GroupBy (v1)|no| +|GroupBy (v2)|yes| +|TopN|yes| +|Search|no| +|Select|no| +|Cardinality Aggregator|no| + ## Extraction Functions Extraction functions define the transformation applied to each dimension value. diff --git a/docs/content/querying/query-context.md b/docs/content/querying/query-context.md index 01059e1ea542..3e01b0a9083b 100644 --- a/docs/content/querying/query-context.md +++ b/docs/content/querying/query-context.md @@ -21,4 +21,3 @@ The query context is used for various query configuration parameters. |`maxResults`|500000|Maximum number of results groupBy query can process. Default value used can be changed by `druid.query.groupBy.maxResults` in druid configuration at broker and historical nodes. At query time you can only lower the value.| |`maxIntermediateRows`|50000|Maximum number of intermediate rows while processing single segment for groupBy query. Default value used can be changed by `druid.query.groupBy.maxIntermediateRows` in druid configuration at broker and historical nodes. At query time you can only lower the value.| |`groupByIsSingleThreaded`|false|Whether to run single threaded group By queries. Default value used can be changed by `druid.query.groupBy.singleThreaded` in druid configuration at historical nodes.| - diff --git a/docs/content/querying/sql.md b/docs/content/querying/sql.md index b0fb6eeafb82..1bec7880f7cd 100644 --- a/docs/content/querying/sql.md +++ b/docs/content/querying/sql.md @@ -160,8 +160,6 @@ Druid does not support all SQL features. Most of these are due to missing featur language. Some unsupported SQL features include: - Grouping on functions of multiple columns, like concatenation: `SELECT COUNT(*) FROM data_source GROUP BY dim1 || ' ' || dim2` -- Grouping on long and float columns. -- Filtering on float columns. - Filtering on non-boolean interactions between columns, like two columns equaling each other: `SELECT COUNT(*) FROM data_source WHERE dim1 = dim2`. - A number of miscellaneous functions, like `TRIM`. - Joins, other than semi-joins as described above. diff --git a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java index f2fbf0f5f41e..750a5f428358 100644 --- a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java +++ b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java @@ -70,37 +70,37 @@ public class MultiSegmentScanQueryTest new ScanQueryEngine() ); - // time modified version of druid.sample.tsv + // time modified version of druid.sample.numeric.tsv public static final String[] V_0112 = { - "2011-01-12T00:00:00.000Z spot automotive preferred apreferred 100.000000", - "2011-01-12T01:00:00.000Z spot business preferred bpreferred 100.000000", - "2011-01-12T02:00:00.000Z spot entertainment preferred epreferred 100.000000", - "2011-01-12T03:00:00.000Z spot health preferred hpreferred 100.000000", - "2011-01-12T04:00:00.000Z spot mezzanine preferred mpreferred 100.000000", - "2011-01-12T05:00:00.000Z spot news preferred npreferred 100.000000", - "2011-01-12T06:00:00.000Z spot premium preferred ppreferred 100.000000", - "2011-01-12T07:00:00.000Z spot technology preferred tpreferred 100.000000", - "2011-01-12T08:00:00.000Z spot travel preferred tpreferred 100.000000", - "2011-01-12T09:00:00.000Z total_market mezzanine preferred mpreferred 1000.000000", - "2011-01-12T10:00:00.000Z total_market premium preferred ppreferred 1000.000000", - "2011-01-12T11:00:00.000Z upfront mezzanine preferred mpreferred 800.000000 value", - "2011-01-12T12:00:00.000Z upfront premium preferred ppreferred 800.000000 value", - "2011-01-12T13:00:00.000Z upfront premium preferred ppreferred2 800.000000 value" + "2011-01-12T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 100.000000", + "2011-01-12T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 100.000000", + "2011-01-12T02:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 100.000000", + "2011-01-12T03:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 100.000000", + "2011-01-12T04:00:00.000Z spot mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 100.000000", + "2011-01-12T05:00:00.000Z spot news\t1500\t15000.0\t150000\tpreferred npreferred 100.000000", + "2011-01-12T06:00:00.000Z spot premium\t1600\t16000.0\t160000\tpreferred ppreferred 100.000000", + "2011-01-12T07:00:00.000Z spot technology\t1700\t17000.0\t170000\tpreferred tpreferred 100.000000", + "2011-01-12T08:00:00.000Z spot travel\t1800\t18000.0\t180000\tpreferred tpreferred 100.000000", + "2011-01-12T09:00:00.000Z total_market mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 1000.000000", + "2011-01-12T10:00:00.000Z total_market premium\t1600\t16000.0\t160000\tpreferred ppreferred 1000.000000", + "2011-01-12T11:00:00.000Z upfront mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 800.000000 value", + "2011-01-12T12:00:00.000Z upfront premium\t1600\t16000.0\t160000\tpreferred ppreferred 800.000000 value" }; + public static final String[] V_0113 = { - "2011-01-13T00:00:00.000Z spot automotive preferred apreferred 94.874713", - "2011-01-13T01:00:00.000Z spot business preferred bpreferred 103.629399", - "2011-01-13T02:00:00.000Z spot entertainment preferred epreferred 110.087299", - "2011-01-13T03:00:00.000Z spot health preferred hpreferred 114.947403", - "2011-01-13T04:00:00.000Z spot mezzanine preferred mpreferred 104.465767", - "2011-01-13T05:00:00.000Z spot news preferred npreferred 102.851683", - "2011-01-13T06:00:00.000Z spot premium preferred ppreferred 108.863011", - "2011-01-13T07:00:00.000Z spot technology preferred tpreferred 111.356672", - "2011-01-13T08:00:00.000Z spot travel preferred tpreferred 106.236928", - "2011-01-13T09:00:00.000Z total_market mezzanine preferred mpreferred 1040.945505", - "2011-01-13T10:00:00.000Z total_market premium preferred ppreferred 1689.012875", - "2011-01-13T11:00:00.000Z upfront mezzanine preferred mpreferred 826.060182 value", - "2011-01-13T12:00:00.000Z upfront premium preferred ppreferred 1564.617729 value" + "2011-01-13T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 94.874713", + "2011-01-13T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 103.629399", + "2011-01-13T02:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 110.087299", + "2011-01-13T03:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 114.947403", + "2011-01-13T04:00:00.000Z spot mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 104.465767", + "2011-01-13T05:00:00.000Z spot news\t1500\t15000.0\t150000\tpreferred npreferred 102.851683", + "2011-01-13T06:00:00.000Z spot premium\t1600\t16000.0\t160000\tpreferred ppreferred 108.863011", + "2011-01-13T07:00:00.000Z spot technology\t1700\t17000.0\t170000\tpreferred tpreferred 111.356672", + "2011-01-13T08:00:00.000Z spot travel\t1800\t18000.0\t180000\tpreferred tpreferred 106.236928", + "2011-01-13T09:00:00.000Z total_market mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 1040.945505", + "2011-01-13T10:00:00.000Z total_market premium\t1600\t16000.0\t160000\tpreferred ppreferred 1689.012875", + "2011-01-13T11:00:00.000Z upfront mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 826.060182 value", + "2011-01-13T12:00:00.000Z upfront premium\t1600\t16000.0\t160000\tpreferred ppreferred 1564.617729 value" }; private static Segment segment0; diff --git a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java index e5277256c5e2..94fc1bc5af9f 100644 --- a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java +++ b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java @@ -58,40 +58,40 @@ @RunWith(Parameterized.class) public class ScanQueryRunnerTest { - // copied from druid.sample.tsv + // copied from druid.sample.numeric.tsv public static final String[] V_0112 = { - "2011-01-12T00:00:00.000Z spot automotive preferred apreferred 100.000000", - "2011-01-12T00:00:00.000Z spot business preferred bpreferred 100.000000", - "2011-01-12T00:00:00.000Z spot entertainment preferred epreferred 100.000000", - "2011-01-12T00:00:00.000Z spot health preferred hpreferred 100.000000", - "2011-01-12T00:00:00.000Z spot mezzanine preferred mpreferred 100.000000", - "2011-01-12T00:00:00.000Z spot news preferred npreferred 100.000000", - "2011-01-12T00:00:00.000Z spot premium preferred ppreferred 100.000000", - "2011-01-12T00:00:00.000Z spot technology preferred tpreferred 100.000000", - "2011-01-12T00:00:00.000Z spot travel preferred tpreferred 100.000000", - "2011-01-12T00:00:00.000Z total_market mezzanine preferred mpreferred 1000.000000", - "2011-01-12T00:00:00.000Z total_market premium preferred ppreferred 1000.000000", - "2011-01-12T00:00:00.000Z upfront mezzanine preferred mpreferred 800.000000 value", - "2011-01-12T00:00:00.000Z upfront premium preferred ppreferred 800.000000 value" + "2011-01-12T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 100.000000", + "2011-01-12T00:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 100.000000", + "2011-01-12T00:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 100.000000", + "2011-01-12T00:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 100.000000", + "2011-01-12T00:00:00.000Z spot mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 100.000000", + "2011-01-12T00:00:00.000Z spot news\t1500\t15000.0\t150000\tpreferred npreferred 100.000000", + "2011-01-12T00:00:00.000Z spot premium\t1600\t16000.0\t160000\tpreferred ppreferred 100.000000", + "2011-01-12T00:00:00.000Z spot technology\t1700\t17000.0\t170000\tpreferred tpreferred 100.000000", + "2011-01-12T00:00:00.000Z spot travel\t1800\t18000.0\t180000\tpreferred tpreferred 100.000000", + "2011-01-12T00:00:00.000Z total_market mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 1000.000000", + "2011-01-12T00:00:00.000Z total_market premium\t1600\t16000.0\t160000\tpreferred ppreferred 1000.000000", + "2011-01-12T00:00:00.000Z upfront mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 800.000000 value", + "2011-01-12T00:00:00.000Z upfront premium\t1600\t16000.0\t160000\tpreferred ppreferred 800.000000 value" }; public static final String[] V_0113 = { - "2011-01-13T00:00:00.000Z spot automotive preferred apreferred 94.874713", - "2011-01-13T00:00:00.000Z spot business preferred bpreferred 103.629399", - "2011-01-13T00:00:00.000Z spot entertainment preferred epreferred 110.087299", - "2011-01-13T00:00:00.000Z spot health preferred hpreferred 114.947403", - "2011-01-13T00:00:00.000Z spot mezzanine preferred mpreferred 104.465767", - "2011-01-13T00:00:00.000Z spot news preferred npreferred 102.851683", - "2011-01-13T00:00:00.000Z spot premium preferred ppreferred 108.863011", - "2011-01-13T00:00:00.000Z spot technology preferred tpreferred 111.356672", - "2011-01-13T00:00:00.000Z spot travel preferred tpreferred 106.236928", - "2011-01-13T00:00:00.000Z total_market mezzanine preferred mpreferred 1040.945505", - "2011-01-13T00:00:00.000Z total_market premium preferred ppreferred 1689.012875", - "2011-01-13T00:00:00.000Z upfront mezzanine preferred mpreferred 826.060182 value", - "2011-01-13T00:00:00.000Z upfront premium preferred ppreferred 1564.617729 value" + "2011-01-13T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 94.874713", + "2011-01-13T00:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 103.629399", + "2011-01-13T00:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 110.087299", + "2011-01-13T00:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 114.947403", + "2011-01-13T00:00:00.000Z spot mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 104.465767", + "2011-01-13T00:00:00.000Z spot news\t1500\t15000.0\t150000\tpreferred npreferred 102.851683", + "2011-01-13T00:00:00.000Z spot premium\t1600\t16000.0\t160000\tpreferred ppreferred 108.863011", + "2011-01-13T00:00:00.000Z spot technology\t1700\t17000.0\t170000\tpreferred tpreferred 111.356672", + "2011-01-13T00:00:00.000Z spot travel\t1800\t18000.0\t180000\tpreferred tpreferred 106.236928", + "2011-01-13T00:00:00.000Z total_market mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 1040.945505", + "2011-01-13T00:00:00.000Z total_market premium\t1600\t16000.0\t160000\tpreferred ppreferred 1689.012875", + "2011-01-13T00:00:00.000Z upfront mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 826.060182 value", + "2011-01-13T00:00:00.000Z upfront premium\t1600\t16000.0\t160000\tpreferred ppreferred 1564.617729 value" }; public static final QuerySegmentSpec I_0112_0114 = new LegacySegmentSpec( - new Interval("2011-01-12/2011-01-14") + new Interval("2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z") ); public static final String[] V_0112_0114 = ObjectArrays.concat(V_0112, V_0113, String.class); @@ -133,6 +133,7 @@ public void testFullOnSelect() ScanResultValue.timestampKey, "market", "quality", + "qualityNumericString", "placement", "placementish", "partial_null_column", @@ -140,7 +141,9 @@ public void testFullOnSelect() "index", "indexMin", "indexMaxPlusTen", - "quality_uniques" + "quality_uniques", + "qualityLong", + "qualityFloat" ); ScanQuery query = newTestQuery() .intervals(I_0112_0114) @@ -168,6 +171,9 @@ public void testFullOnSelectAsCompactedList() ScanResultValue.timestampKey, "market", "quality", + "qualityLong", + "qualityFloat", + "qualityNumericString", "placement", "placementish", "partial_null_column", @@ -219,6 +225,9 @@ public void testSelectWithDimsAndMets() null, null, null, + null, + null, + null, QueryRunnerTestHelper.indexMetric + ":FLOAT" }, V_0112_0114 @@ -253,6 +262,9 @@ public void testSelectWithDimsAndMetsAsCompactedList() null, null, null, + null, + null, + null, QueryRunnerTestHelper.indexMetric + ":FLOAT" }, V_0112_0114 @@ -441,6 +453,9 @@ private List>> toFullEvents(final String[]... valueSet) ScanResultValue.timestampKey + ":TIME", QueryRunnerTestHelper.marketDimension + ":STRING", QueryRunnerTestHelper.qualityDimension + ":STRING", + "qualityLong" + ":LONG", + "qualityFloat" + ":FLOAT", + "qualityNumericString" + ":STRING", QueryRunnerTestHelper.placementDimension + ":STRING", QueryRunnerTestHelper.placementishDimension + ":STRINGS", QueryRunnerTestHelper.indexMetric + ":FLOAT", diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/types/CardinalityAggregatorColumnSelectorStrategyFactory.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/types/CardinalityAggregatorColumnSelectorStrategyFactory.java index 9a74997855c8..f840ce192f64 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/types/CardinalityAggregatorColumnSelectorStrategyFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/types/CardinalityAggregatorColumnSelectorStrategyFactory.java @@ -21,6 +21,7 @@ import io.druid.java.util.common.IAE; import io.druid.query.dimension.ColumnSelectorStrategyFactory; +import io.druid.segment.ColumnValueSelector; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ValueType; @@ -29,13 +30,17 @@ public class CardinalityAggregatorColumnSelectorStrategyFactory { @Override public CardinalityAggregatorColumnSelectorStrategy makeColumnSelectorStrategy( - ColumnCapabilities capabilities + ColumnCapabilities capabilities, ColumnValueSelector selector ) { ValueType type = capabilities.getType(); switch(type) { case STRING: return new StringCardinalityAggregatorColumnSelectorStrategy(); + case LONG: + return new LongCardinalityAggregatorColumnSelectorStrategy(); + case FLOAT: + return new FloatCardinalityAggregatorColumnSelectorStrategy(); default: throw new IAE("Cannot create query type helper from invalid type [%s]", type); } diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/types/FloatCardinalityAggregatorColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/types/FloatCardinalityAggregatorColumnSelectorStrategy.java new file mode 100644 index 000000000000..f80caf02b00b --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/types/FloatCardinalityAggregatorColumnSelectorStrategy.java @@ -0,0 +1,41 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.aggregation.cardinality.types; + +import com.google.common.hash.Hasher; +import io.druid.hll.HyperLogLogCollector; +import io.druid.query.aggregation.cardinality.CardinalityAggregator; +import io.druid.segment.FloatColumnSelector; + +public class FloatCardinalityAggregatorColumnSelectorStrategy + implements CardinalityAggregatorColumnSelectorStrategy +{ + @Override + public void hashRow(FloatColumnSelector dimSelector, Hasher hasher) + { + hasher.putFloat(dimSelector.get()); + } + + @Override + public void hashValues(FloatColumnSelector dimSelector, HyperLogLogCollector collector) + { + collector.add(CardinalityAggregator.hashFn.hashInt(Float.floatToIntBits(dimSelector.get())).asBytes()); + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/types/LongCardinalityAggregatorColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/types/LongCardinalityAggregatorColumnSelectorStrategy.java new file mode 100644 index 000000000000..174341f49330 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/types/LongCardinalityAggregatorColumnSelectorStrategy.java @@ -0,0 +1,41 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.aggregation.cardinality.types; + +import com.google.common.hash.Hasher; +import io.druid.hll.HyperLogLogCollector; +import io.druid.query.aggregation.cardinality.CardinalityAggregator; +import io.druid.segment.LongColumnSelector; + +public class LongCardinalityAggregatorColumnSelectorStrategy + implements CardinalityAggregatorColumnSelectorStrategy +{ + @Override + public void hashRow(LongColumnSelector dimSelector, Hasher hasher) + { + hasher.putLong(dimSelector.get()); + } + + @Override + public void hashValues(LongColumnSelector dimSelector, HyperLogLogCollector collector) + { + collector.add(CardinalityAggregator.hashFn.hashLong(dimSelector.get()).asBytes()); + } +} diff --git a/processing/src/main/java/io/druid/query/dimension/BaseFilteredDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/BaseFilteredDimensionSpec.java index 50ff76e88bb7..683e2966685a 100644 --- a/processing/src/main/java/io/druid/query/dimension/BaseFilteredDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/BaseFilteredDimensionSpec.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import io.druid.query.extraction.ExtractionFn; +import io.druid.segment.column.ValueType; /** */ @@ -54,12 +55,24 @@ public String getOutputName() return delegate.getOutputName(); } + @Override + public ValueType getOutputType() + { + return delegate.getOutputType(); + } + @Override public ExtractionFn getExtractionFn() { return delegate.getExtractionFn(); } + @Override + public boolean mustDecorate() + { + return true; + } + @Override public boolean preservesOrdering() { diff --git a/processing/src/main/java/io/druid/query/dimension/ColumnSelectorStrategyFactory.java b/processing/src/main/java/io/druid/query/dimension/ColumnSelectorStrategyFactory.java index 7729e05e48db..9cc2b49e56e7 100644 --- a/processing/src/main/java/io/druid/query/dimension/ColumnSelectorStrategyFactory.java +++ b/processing/src/main/java/io/druid/query/dimension/ColumnSelectorStrategyFactory.java @@ -19,9 +19,10 @@ package io.druid.query.dimension; +import io.druid.segment.ColumnValueSelector; import io.druid.segment.column.ColumnCapabilities; public interface ColumnSelectorStrategyFactory { - ColumnSelectorStrategyClass makeColumnSelectorStrategy(ColumnCapabilities capabilities); + ColumnSelectorStrategyClass makeColumnSelectorStrategy(ColumnCapabilities capabilities, ColumnValueSelector selector); } diff --git a/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java index 5e81f5d49578..eb5da594ef55 100644 --- a/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java @@ -27,6 +27,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.query.extraction.ExtractionFn; import io.druid.segment.DimensionSelector; +import io.druid.segment.column.ValueType; import java.nio.ByteBuffer; import java.util.Arrays; @@ -65,19 +66,30 @@ public DimensionSpec apply(String input) private static final byte CACHE_TYPE_ID = 0x0; private final String dimension; private final String outputName; + private final ValueType outputType; @JsonCreator public DefaultDimensionSpec( @JsonProperty("dimension") String dimension, - @JsonProperty("outputName") String outputName + @JsonProperty("outputName") String outputName, + @JsonProperty("outputType") ValueType outputType ) { this.dimension = dimension; + this.outputType = outputType == null ? ValueType.STRING : outputType; // Do null check for legacy backwards compatibility, callers should be setting the value. this.outputName = outputName == null ? dimension : outputName; } + public DefaultDimensionSpec( + String dimension, + String outputName + ) + { + this(dimension, outputName, ValueType.STRING); + } + @Override @JsonProperty public String getDimension() @@ -92,6 +104,13 @@ public String getOutputName() return outputName; } + @Override + @JsonProperty + public ValueType getOutputType() + { + return outputType; + } + @Override public ExtractionFn getExtractionFn() { @@ -104,6 +123,12 @@ public DimensionSelector decorate(DimensionSelector selector) return selector; } + @Override + public boolean mustDecorate() + { + return false; + } + @Override public byte[] getCacheKey() { @@ -127,6 +152,7 @@ public String toString() return "DefaultDimensionSpec{" + "dimension='" + dimension + '\'' + ", outputName='" + outputName + '\'' + + ", outputType='" + outputType + '\'' + '}'; } @@ -149,6 +175,9 @@ public boolean equals(Object o) if (outputName != null ? !outputName.equals(that.outputName) : that.outputName != null) { return false; } + if (outputType != null ? !outputType.equals(that.outputType) : that.outputType != null) { + return false; + } return true; } @@ -158,6 +187,7 @@ public int hashCode() { int result = dimension != null ? dimension.hashCode() : 0; result = 31 * result + (outputName != null ? outputName.hashCode() : 0); + result = 31 * result + (outputType != null ? outputType.hashCode() : 0); return result; } } diff --git a/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java index 21b9201d8c1c..c0afe367acdb 100644 --- a/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import io.druid.query.extraction.ExtractionFn; import io.druid.segment.DimensionSelector; +import io.druid.segment.column.ValueType; /** */ @@ -40,12 +41,19 @@ public interface DimensionSpec String getOutputName(); + ValueType getOutputType(); + //ExtractionFn can be implemented with decorate(..) fn @Deprecated ExtractionFn getExtractionFn(); DimensionSelector decorate(DimensionSelector selector); + /** + * Does this DimensionSpec require that decorate() be called to produce correct results? + */ + boolean mustDecorate(); + byte[] getCacheKey(); boolean preservesOrdering(); diff --git a/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java index 305a10d8258b..6f992018c816 100644 --- a/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java @@ -25,6 +25,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.query.extraction.ExtractionFn; import io.druid.segment.DimensionSelector; +import io.druid.segment.column.ValueType; import java.nio.ByteBuffer; @@ -37,11 +38,13 @@ public class ExtractionDimensionSpec implements DimensionSpec private final String dimension; private final ExtractionFn extractionFn; private final String outputName; + private final ValueType outputType; @JsonCreator public ExtractionDimensionSpec( @JsonProperty("dimension") String dimension, @JsonProperty("outputName") String outputName, + @JsonProperty("outputType") ValueType outputType, @JsonProperty("extractionFn") ExtractionFn extractionFn, // for backwards compatibility @Deprecated @JsonProperty("dimExtractionFn") ExtractionFn dimExtractionFn @@ -52,6 +55,7 @@ public ExtractionDimensionSpec( this.dimension = dimension; this.extractionFn = extractionFn != null ? extractionFn : dimExtractionFn; + this.outputType = outputType == null ? ValueType.STRING : outputType; // Do null check for backwards compatibility this.outputName = outputName == null ? dimension : outputName; @@ -59,7 +63,12 @@ public ExtractionDimensionSpec( public ExtractionDimensionSpec(String dimension, String outputName, ExtractionFn extractionFn) { - this(dimension, outputName, extractionFn, null); + this(dimension, outputName, null, extractionFn, null); + } + + public ExtractionDimensionSpec(String dimension, String outputName, ValueType outputType, ExtractionFn extractionFn) + { + this(dimension, outputName, outputType, extractionFn, null); } @Override @@ -76,6 +85,13 @@ public String getOutputName() return outputName; } + @Override + @JsonProperty + public ValueType getOutputType() + { + return outputType; + } + @Override @JsonProperty public ExtractionFn getExtractionFn() @@ -89,6 +105,12 @@ public DimensionSelector decorate(DimensionSelector selector) return selector; } + @Override + public boolean mustDecorate() + { + return false; + } + @Override public byte[] getCacheKey() { @@ -115,6 +137,7 @@ public String toString() "dimension='" + dimension + '\'' + ", extractionFn=" + extractionFn + ", outputName='" + outputName + '\'' + + ", outputType='" + outputType + '\'' + '}'; } @@ -139,6 +162,9 @@ public boolean equals(Object o) if (outputName != null ? !outputName.equals(that.outputName) : that.outputName != null) { return false; } + if (outputType != null ? !outputType.equals(that.outputType) : that.outputType != null) { + return false; + } return true; } @@ -149,6 +175,7 @@ public int hashCode() int result = dimension != null ? dimension.hashCode() : 0; result = 31 * result + (extractionFn != null ? extractionFn.hashCode() : 0); result = 31 * result + (outputName != null ? outputName.hashCode() : 0); + result = 31 * result + (outputType != null ? outputType.hashCode() : 0); return result; } } diff --git a/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java index fd41aeeba03c..8dbb0b45b63f 100644 --- a/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java @@ -31,6 +31,7 @@ import io.druid.query.lookup.LookupExtractor; import io.druid.query.lookup.LookupReferencesManager; import io.druid.segment.DimensionSelector; +import io.druid.segment.column.ValueType; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -109,6 +110,13 @@ public String getOutputName() return outputName; } + @Override + public ValueType getOutputType() + { + // Extraction functions always output String + return ValueType.STRING; + } + @JsonProperty @Nullable public LookupExtractor getLookup() @@ -149,6 +157,12 @@ public DimensionSelector decorate(DimensionSelector selector) return selector; } + @Override + public boolean mustDecorate() + { + return false; + } + @Override public byte[] getCacheKey() { diff --git a/processing/src/main/java/io/druid/query/extraction/DimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/DimExtractionFn.java index afb22cd567b7..40c9d77b77a8 100644 --- a/processing/src/main/java/io/druid/query/extraction/DimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/DimExtractionFn.java @@ -19,12 +19,14 @@ package io.druid.query.extraction; +import java.util.Objects; + public abstract class DimExtractionFn implements ExtractionFn { @Override public String apply(Object value) { - return apply(value == null ? null : value.toString()); + return apply(Objects.toString(value, null)); } @Override diff --git a/processing/src/main/java/io/druid/query/filter/BoundDimFilter.java b/processing/src/main/java/io/druid/query/filter/BoundDimFilter.java index f93830bfea69..b9b20dab7466 100644 --- a/processing/src/main/java/io/druid/query/filter/BoundDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/BoundDimFilter.java @@ -27,6 +27,7 @@ import com.google.common.collect.Range; import com.google.common.collect.RangeSet; import com.google.common.collect.TreeRangeSet; +import com.google.common.primitives.Floats; import io.druid.common.guava.GuavaUtils; import io.druid.java.util.common.StringUtils; import io.druid.query.extraction.ExtractionFn; @@ -47,6 +48,7 @@ public class BoundDimFilter implements DimFilter private final ExtractionFn extractionFn; private final StringComparator ordering; private final Supplier longPredicateSupplier; + private final Supplier floatPredicateSupplier; @JsonCreator public BoundDimFilter( @@ -87,6 +89,7 @@ public BoundDimFilter( } this.extractionFn = extractionFn; this.longPredicateSupplier = makeLongPredicateSupplier(); + this.floatPredicateSupplier = makeFloatPredicateSupplier(); } @JsonProperty @@ -146,6 +149,10 @@ public Supplier getLongPredicateSupplier() return longPredicateSupplier; } + public Supplier getFloatPredicateSupplier() { + return floatPredicateSupplier; + } + @Override public byte[] getCacheKey() { @@ -260,7 +267,6 @@ public boolean equals(Object o) return false; } return getOrdering().equals(that.getOrdering()); - } @Override @@ -317,89 +323,45 @@ private Supplier makeLongPredicateSupplier() class BoundLongPredicateSupplier implements Supplier { private final Object initLock = new Object(); - - // longsInitialized is volatile since it establishes the happens-before relationship on - // writes/reads to the rest of the fields (it's written last and read first). - private volatile boolean longsInitialized = false; - - // Other fields are not volatile. - private boolean matchesNothing; - private boolean hasLowerLongBound; - private boolean hasUpperLongBound; - private long lowerLongBound; - private long upperLongBound; + private DruidLongPredicate predicate; @Override public DruidLongPredicate get() { - initLongData(); - - if (matchesNothing) { - return new DruidLongPredicate() - { - @Override - public boolean applyLong(long input) - { - return false; - } - }; - } - - return new DruidLongPredicate() - { - private final boolean hasLowerLongBound = BoundLongPredicateSupplier.this.hasLowerLongBound; - private final boolean hasUpperLongBound = BoundLongPredicateSupplier.this.hasUpperLongBound; - private final long lowerLongBound = hasLowerLongBound ? BoundLongPredicateSupplier.this.lowerLongBound : 0L; - private final long upperLongBound = hasUpperLongBound ? BoundLongPredicateSupplier.this.upperLongBound : 0L; - - @Override - public boolean applyLong(long input) - { - int lowerComparing = 1; - int upperComparing = 1; - if (hasLowerLongBound) { - lowerComparing = Long.compare(input, lowerLongBound); - } - if (hasUpperLongBound) { - upperComparing = Long.compare(upperLongBound, input); - } - if (lowerStrict && upperStrict) { - return ((lowerComparing > 0)) && (upperComparing > 0); - } else if (lowerStrict) { - return (lowerComparing > 0) && (upperComparing >= 0); - } else if (upperStrict) { - return (lowerComparing >= 0) && (upperComparing > 0); - } - return (lowerComparing >= 0) && (upperComparing >= 0); - - } - }; + initPredicate(); + return predicate; } - private void initLongData() + private void initPredicate() { - if (longsInitialized) { + if (predicate != null) { return; } synchronized (initLock) { - if (longsInitialized) { + if (predicate != null) { return; } - matchesNothing = false; + final boolean hasLowerLongBound; + final boolean hasUpperLongBound; + final long lowerLongBound; + final long upperLongBound; + boolean matchesNothing = false; if (hasLowerBound()) { final Long lowerLong = GuavaUtils.tryParseLong(lower); if (lowerLong == null) { // Unparseable values fall before all actual numbers, so all numbers will match the lower bound. hasLowerLongBound = false; + lowerLongBound = 0L; } else { hasLowerLongBound = true; lowerLongBound = lowerLong; } } else { hasLowerLongBound = false; + lowerLongBound = 0L; } if (hasUpperBound()) { @@ -407,19 +369,320 @@ private void initLongData() if (upperLong == null) { // Unparseable values fall before all actual numbers, so no numbers can match the upper bound. matchesNothing = true; - return; + hasUpperLongBound = false; + upperLongBound = 0L; + } else { + hasUpperLongBound = true; + upperLongBound = upperLong; } - - hasUpperLongBound = true; - upperLongBound = upperLong; } else { hasUpperLongBound = false; + upperLongBound = 0L; } - longsInitialized = true; + if (matchesNothing) { + predicate = DruidLongPredicate.ALWAYS_FALSE; + } else { + predicate = makeLongPredicateFromBounds( + hasLowerLongBound, + hasUpperLongBound, + lowerStrict, + upperStrict, + lowerLongBound, + upperLongBound + ); + } } } } return new BoundLongPredicateSupplier(); } + + private Supplier makeFloatPredicateSupplier() + { + class BoundFloatPredicateSupplier implements Supplier + { + private final Object initLock = new Object(); + private DruidFloatPredicate predicate; + + @Override + public DruidFloatPredicate get() + { + initPredicate(); + return predicate; + } + + private void initPredicate() + { + if (predicate != null) { + return; + } + + synchronized (initLock) { + if (predicate != null) { + return; + } + + final boolean hasLowerFloatBound; + final boolean hasUpperFloatBound; + final float lowerFloatBound; + final float upperFloatBound; + boolean matchesNothing = false; + + if (hasLowerBound()) { + final Float lowerFloat = Floats.tryParse(lower); + if (lowerFloat == null) { + // Unparseable values fall before all actual numbers, so all numbers will match the lower bound. + hasLowerFloatBound = false; + lowerFloatBound = 0L; + } else { + hasLowerFloatBound = true; + lowerFloatBound = lowerFloat; + } + } else { + hasLowerFloatBound = false; + lowerFloatBound = 0L; + } + + if (hasUpperBound()) { + Float upperFloat = Floats.tryParse(upper); + if (upperFloat == null) { + // Unparseable values fall before all actual numbers, so no numbers can match the upper bound. + matchesNothing = true; + hasUpperFloatBound = false; + upperFloatBound = 0L; + } else { + hasUpperFloatBound = true; + upperFloatBound = upperFloat; + } + } else { + hasUpperFloatBound = false; + upperFloatBound = 0L; + } + + if (matchesNothing) { + predicate = DruidFloatPredicate.ALWAYS_FALSE; + } else { + predicate = makeFloatPredicateFromBounds( + hasLowerFloatBound, + hasUpperFloatBound, + lowerStrict, + upperStrict, + lowerFloatBound, + upperFloatBound + ); + } + } + } + } + return new BoundFloatPredicateSupplier(); + } + + private static DruidLongPredicate makeLongPredicateFromBounds( + final boolean hasLowerLongBound, + final boolean hasUpperLongBound, + final boolean lowerStrict, + final boolean upperStrict, + final long lowerLongBound, + final long upperLongBound + ) + { + if (hasLowerLongBound && hasUpperLongBound) { + if (upperStrict && lowerStrict) { + return new DruidLongPredicate() + { + @Override + public boolean applyLong(long input) + { + final int lowerComparing = Long.compare(input, lowerLongBound); + final int upperComparing = Long.compare(upperLongBound, input); + return ((lowerComparing > 0)) && (upperComparing > 0); + } + }; + } else if (lowerStrict) { + return new DruidLongPredicate() + { + @Override + public boolean applyLong(long input) + { + final int lowerComparing = Long.compare(input, lowerLongBound); + final int upperComparing = Long.compare(upperLongBound, input); + return (lowerComparing > 0) && (upperComparing >= 0); + } + }; + } else if (upperStrict) { + return new DruidLongPredicate() + { + @Override + public boolean applyLong(long input) + { + final int lowerComparing = Long.compare(input, lowerLongBound); + final int upperComparing = Long.compare(upperLongBound, input); + return (lowerComparing >= 0) && (upperComparing > 0); + } + }; + } else { + return new DruidLongPredicate() + { + @Override + public boolean applyLong(long input) + { + final int lowerComparing = Long.compare(input, lowerLongBound); + final int upperComparing = Long.compare(upperLongBound, input); + return (lowerComparing >= 0) && (upperComparing >= 0); + } + }; + } + } else if (hasUpperLongBound) { + if (upperStrict) { + return new DruidLongPredicate() + { + @Override + public boolean applyLong(long input) + { + final int upperComparing = Long.compare(upperLongBound, input); + return upperComparing > 0; + } + }; + } else { + return new DruidLongPredicate() + { + @Override + public boolean applyLong(long input) + { + final int upperComparing = Long.compare(upperLongBound, input); + return upperComparing >= 0; + } + }; + } + } else if (hasLowerLongBound) { + if (lowerStrict) { + return new DruidLongPredicate() + { + @Override + public boolean applyLong(long input) + { + final int lowerComparing = Long.compare(input, lowerLongBound); + return lowerComparing > 0; + } + }; + } else { + return new DruidLongPredicate() + { + @Override + public boolean applyLong(long input) + { + final int lowerComparing = Long.compare(input, lowerLongBound); + return lowerComparing >= 0; + } + }; + } + } else { + return DruidLongPredicate.ALWAYS_TRUE; + } + } + + private static DruidFloatPredicate makeFloatPredicateFromBounds( + final boolean hasLowerFloatBound, + final boolean hasUpperFloatBound, + final boolean lowerStrict, + final boolean upperStrict, + final float lowerFloatBound, + final float upperFloatBound + ) + { + if (hasLowerFloatBound && hasUpperFloatBound) { + if (upperStrict && lowerStrict) { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + final int lowerComparing = Float.compare(input, lowerFloatBound); + final int upperComparing = Float.compare(upperFloatBound, input); + return ((lowerComparing > 0)) && (upperComparing > 0); + } + }; + } else if (lowerStrict) { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + final int lowerComparing = Float.compare(input, lowerFloatBound); + final int upperComparing = Float.compare(upperFloatBound, input); + return (lowerComparing > 0) && (upperComparing >= 0); + } + }; + } else if (upperStrict) { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + final int lowerComparing = Float.compare(input, lowerFloatBound); + final int upperComparing = Float.compare(upperFloatBound, input); + return (lowerComparing >= 0) && (upperComparing > 0); + } + }; + } else { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + final int lowerComparing = Float.compare(input, lowerFloatBound); + final int upperComparing = Float.compare(upperFloatBound, input); + return (lowerComparing >= 0) && (upperComparing >= 0); + } + }; + } + } else if (hasUpperFloatBound) { + if (upperStrict) { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + final int upperComparing = Float.compare(upperFloatBound, input); + return upperComparing > 0; + } + }; + } else { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + final int upperComparing = Float.compare(upperFloatBound, input); + return upperComparing >= 0; + } + }; + } + } else if (hasLowerFloatBound) { + if (lowerStrict) { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + final int lowerComparing = Float.compare(input, lowerFloatBound); + return lowerComparing > 0; + } + }; + } else { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + final int lowerComparing = Float.compare(input, lowerFloatBound); + return lowerComparing >= 0; + } + }; + } + } else { + return DruidFloatPredicate.ALWAYS_TRUE; + } + } } diff --git a/processing/src/main/java/io/druid/query/filter/DruidFloatPredicate.java b/processing/src/main/java/io/druid/query/filter/DruidFloatPredicate.java new file mode 100644 index 000000000000..737c4e6651a4 --- /dev/null +++ b/processing/src/main/java/io/druid/query/filter/DruidFloatPredicate.java @@ -0,0 +1,47 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.filter; + +/** + * FloatPredicate is only supported in Java 8+, so use this to avoid boxing when a float predicate is needed. + */ +public interface DruidFloatPredicate +{ + DruidFloatPredicate ALWAYS_FALSE = new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + return false; + } + }; + + DruidFloatPredicate ALWAYS_TRUE = new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + return true; + } + }; + + + boolean applyFloat(float input); +} diff --git a/processing/src/main/java/io/druid/query/filter/DruidLongPredicate.java b/processing/src/main/java/io/druid/query/filter/DruidLongPredicate.java index 1651776777f8..20debafcd82d 100644 --- a/processing/src/main/java/io/druid/query/filter/DruidLongPredicate.java +++ b/processing/src/main/java/io/druid/query/filter/DruidLongPredicate.java @@ -24,5 +24,23 @@ */ public interface DruidLongPredicate { + DruidLongPredicate ALWAYS_FALSE = new DruidLongPredicate() + { + @Override + public boolean applyLong(long input) + { + return false; + } + }; + + DruidLongPredicate ALWAYS_TRUE = new DruidLongPredicate() + { + @Override + public boolean applyLong(long input) + { + return true; + } + }; + boolean applyLong(long input); } diff --git a/processing/src/main/java/io/druid/query/filter/DruidPredicateFactory.java b/processing/src/main/java/io/druid/query/filter/DruidPredicateFactory.java index bc94931167ca..f8f1b03b8b80 100644 --- a/processing/src/main/java/io/druid/query/filter/DruidPredicateFactory.java +++ b/processing/src/main/java/io/druid/query/filter/DruidPredicateFactory.java @@ -26,4 +26,6 @@ public interface DruidPredicateFactory public Predicate makeStringPredicate(); public DruidLongPredicate makeLongPredicate(); + + public DruidFloatPredicate makeFloatPredicate(); } diff --git a/processing/src/main/java/io/druid/query/filter/FloatValueMatcherColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/filter/FloatValueMatcherColumnSelectorStrategy.java new file mode 100644 index 000000000000..ebc91c4f0cd8 --- /dev/null +++ b/processing/src/main/java/io/druid/query/filter/FloatValueMatcherColumnSelectorStrategy.java @@ -0,0 +1,62 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.filter; + +import io.druid.segment.DimensionHandlerUtils; +import io.druid.segment.FloatColumnSelector; +import io.druid.segment.filter.BooleanValueMatcher; + +public class FloatValueMatcherColumnSelectorStrategy implements ValueMatcherColumnSelectorStrategy +{ + @Override + public ValueMatcher makeValueMatcher(final FloatColumnSelector selector, final String value) + { + final Float matchVal = DimensionHandlerUtils.convertObjectToFloat(value); + if (matchVal == null) { + return BooleanValueMatcher.of(false); + } + + final int matchValIntBits = Float.floatToIntBits(matchVal); + return new ValueMatcher() + { + @Override + public boolean matches() + { + return Float.floatToIntBits(selector.get()) == matchValIntBits; + } + }; + } + + @Override + public ValueMatcher makeValueMatcher( + final FloatColumnSelector selector, DruidPredicateFactory predicateFactory + ) + { + final DruidFloatPredicate predicate = predicateFactory.makeFloatPredicate(); + return new ValueMatcher() + { + @Override + public boolean matches() + { + return predicate.applyFloat(selector.get()); + } + }; + } +} diff --git a/processing/src/main/java/io/druid/query/filter/InDimFilter.java b/processing/src/main/java/io/druid/query/filter/InDimFilter.java index 8d6b25053f57..a4b5823038f4 100644 --- a/processing/src/main/java/io/druid/query/filter/InDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/InDimFilter.java @@ -31,18 +31,22 @@ import com.google.common.collect.Range; import com.google.common.collect.RangeSet; import com.google.common.collect.TreeRangeSet; +import com.google.common.primitives.Floats; import io.druid.common.guava.GuavaUtils; import io.druid.java.util.common.StringUtils; import io.druid.query.extraction.ExtractionFn; import io.druid.query.lookup.LookupExtractionFn; import io.druid.query.lookup.LookupExtractor; import io.druid.segment.filter.InFilter; +import it.unimi.dsi.fastutil.ints.IntArrayList; +import it.unimi.dsi.fastutil.ints.IntOpenHashSet; +import it.unimi.dsi.fastutil.longs.LongArrayList; +import it.unimi.dsi.fastutil.longs.LongOpenHashSet; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.HashSet; import java.util.List; import java.util.Objects; import java.util.Set; @@ -51,12 +55,13 @@ public class InDimFilter implements DimFilter { // determined through benchmark that binary search on long[] is faster than HashSet until ~16 elements // Hashing threshold is not applied to String for now, String still uses ImmutableSortedSet - public static final int LONG_HASHING_THRESHOLD = 16; + public static final int NUMERIC_HASHING_THRESHOLD = 16; private final ImmutableSortedSet values; private final String dimension; private final ExtractionFn extractionFn; private final Supplier longPredicateSupplier; + private final Supplier floatPredicateSupplier; @JsonCreator public InDimFilter( @@ -83,6 +88,7 @@ public String apply(String input) this.dimension = dimension; this.extractionFn = extractionFn; this.longPredicateSupplier = getLongPredicateSupplier(); + this.floatPredicateSupplier = getFloatPredicateSupplier(); } @JsonProperty @@ -182,7 +188,7 @@ private InDimFilter optimizeLookup() { @Override public Filter toFilter() { - return new InFilter(dimension, values, longPredicateSupplier, extractionFn); + return new InFilter(dimension, values, longPredicateSupplier, floatPredicateSupplier, extractionFn); } @Override @@ -258,23 +264,21 @@ private Supplier getLongPredicateSupplier() return new Supplier() { private final Object initLock = new Object(); - private volatile boolean longsInitialized = false; - private volatile boolean useLongHash; - private volatile long[] longArray; - private volatile HashSet longHashSet; + private DruidLongPredicate predicate; + private void initLongValues() { - if (longsInitialized) { + if (predicate != null) { return; } synchronized (initLock) { - if (longsInitialized) { + if (predicate != null) { return; } - List longs = new ArrayList<>(); + LongArrayList longs = new LongArrayList(values.size()); for (String value : values) { Long longValue = GuavaUtils.tryParseLong(value); if (longValue != null) { @@ -282,18 +286,30 @@ private void initLongValues() } } - useLongHash = longs.size() > LONG_HASHING_THRESHOLD; - if (useLongHash) { - longHashSet = new HashSet(longs); + if (longs.size() > NUMERIC_HASHING_THRESHOLD) { + final LongOpenHashSet longHashSet = new LongOpenHashSet(longs); + + predicate = new DruidLongPredicate() + { + @Override + public boolean applyLong(long input) + { + return longHashSet.contains(input); + } + }; } else { - longArray = new long[longs.size()]; - for (int i = 0; i < longs.size(); i++) { - longArray[i] = longs.get(i).longValue(); - } + final long[] longArray = longs.toLongArray(); Arrays.sort(longArray); - } - longsInitialized = true; + predicate = new DruidLongPredicate() + { + @Override + public boolean applyLong(long input) + { + return Arrays.binarySearch(longArray, input) >= 0; + } + }; + } } } @@ -301,27 +317,70 @@ private void initLongValues() public DruidLongPredicate get() { initLongValues(); + return predicate; + } + }; + } - if (useLongHash) { - return new DruidLongPredicate() - { - @Override - public boolean applyLong(long input) - { - return longHashSet.contains(input); + private Supplier getFloatPredicateSupplier() + { + return new Supplier() + { + private final Object initLock = new Object(); + private DruidFloatPredicate predicate; + + private void initFloatValues() + { + if (predicate != null) { + return; + } + + synchronized (initLock) { + if (predicate != null) { + return; + } + + IntArrayList floatBits = new IntArrayList(values.size()); + for (String value : values) { + Float floatValue = Floats.tryParse(value); + if (floatValue != null) { + floatBits.add(Float.floatToIntBits(floatValue)); } - }; - } else { - return new DruidLongPredicate() - { - @Override - public boolean applyLong(long input) + } + + if (floatBits.size() > NUMERIC_HASHING_THRESHOLD) { + final IntOpenHashSet floatBitsHashSet = new IntOpenHashSet(floatBits); + + predicate = new DruidFloatPredicate() { - return Arrays.binarySearch(longArray, input) >= 0; - } - }; + @Override + public boolean applyFloat(float input) + { + return floatBitsHashSet.contains(Float.floatToIntBits(input)); + } + }; + } else { + final int[] floatBitsArray = floatBits.toIntArray(); + Arrays.sort(floatBitsArray); + + predicate = new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + return Arrays.binarySearch(floatBitsArray, Float.floatToIntBits(input)) >= 0; + } + }; + } } } + + @Override + public DruidFloatPredicate get() + { + initFloatValues(); + return predicate; + } }; } } diff --git a/processing/src/main/java/io/druid/query/filter/JavaScriptDimFilter.java b/processing/src/main/java/io/druid/query/filter/JavaScriptDimFilter.java index 0373cfe750b6..9d140128f34a 100644 --- a/processing/src/main/java/io/druid/query/filter/JavaScriptDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/JavaScriptDimFilter.java @@ -217,6 +217,20 @@ public boolean applyLong(long input) }; } + @Override + public DruidFloatPredicate makeFloatPredicate() + { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + // Can't avoid boxing here because the Mozilla JS Function.call() only accepts Object[] + return applyObject(input); + } + }; + } + public boolean applyObject(final Object input) { // one and only one context per thread diff --git a/processing/src/main/java/io/druid/query/filter/LikeDimFilter.java b/processing/src/main/java/io/druid/query/filter/LikeDimFilter.java index 236b52c171ea..c1f640e02a26 100644 --- a/processing/src/main/java/io/druid/query/filter/LikeDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/LikeDimFilter.java @@ -224,6 +224,30 @@ public boolean applyLong(long input) }; } } + + @Override + public DruidFloatPredicate makeFloatPredicate() + { + if (extractionFn != null) { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + return matches(extractionFn.apply(input)); + } + }; + } else { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + return matches(String.valueOf(input)); + } + }; + } + } }; } diff --git a/processing/src/main/java/io/druid/query/filter/LongValueMatcherColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/filter/LongValueMatcherColumnSelectorStrategy.java new file mode 100644 index 000000000000..8660cfba1001 --- /dev/null +++ b/processing/src/main/java/io/druid/query/filter/LongValueMatcherColumnSelectorStrategy.java @@ -0,0 +1,61 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.filter; + +import io.druid.segment.DimensionHandlerUtils; +import io.druid.segment.LongColumnSelector; +import io.druid.segment.filter.BooleanValueMatcher; + +public class LongValueMatcherColumnSelectorStrategy implements ValueMatcherColumnSelectorStrategy +{ + @Override + public ValueMatcher makeValueMatcher(final LongColumnSelector selector, final String value) + { + final Long matchVal = DimensionHandlerUtils.convertObjectToLong(value); + if (matchVal == null) { + return BooleanValueMatcher.of(false); + } + final long matchValLong = matchVal; + return new ValueMatcher() + { + @Override + public boolean matches() + { + return selector.get() == matchValLong; + } + }; + } + + @Override + public ValueMatcher makeValueMatcher( + final LongColumnSelector selector, DruidPredicateFactory predicateFactory + ) + { + final DruidLongPredicate predicate = predicateFactory.makeLongPredicate(); + return new ValueMatcher() + { + @Override + public boolean matches() + { + return predicate.applyLong(selector.get()); + } + }; + } +} diff --git a/processing/src/main/java/io/druid/query/filter/SelectorDimFilter.java b/processing/src/main/java/io/druid/query/filter/SelectorDimFilter.java index 320b51dd737a..ffbf983d13d4 100644 --- a/processing/src/main/java/io/druid/query/filter/SelectorDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/SelectorDimFilter.java @@ -23,11 +23,13 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; +import com.google.common.base.Predicates; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.Range; import com.google.common.collect.RangeSet; import com.google.common.collect.TreeRangeSet; +import com.google.common.primitives.Floats; import io.druid.common.guava.GuavaUtils; import io.druid.java.util.common.StringUtils; import io.druid.query.extraction.ExtractionFn; @@ -46,9 +48,9 @@ public class SelectorDimFilter implements DimFilter private final ExtractionFn extractionFn; private final Object initLock = new Object(); - private volatile boolean longsInitialized = false; - private volatile Long valueAsLong; + private DruidLongPredicate longPredicate; + private DruidFloatPredicate floatPredicate; @JsonCreator public SelectorDimFilter( @@ -100,42 +102,21 @@ public Filter toFilter() @Override public Predicate makeStringPredicate() { - return new Predicate() - { - @Override - public boolean apply(String input) - { - return Objects.equals(valueOrNull, input); - } - }; + return Predicates.equalTo(valueOrNull); } @Override public DruidLongPredicate makeLongPredicate() { - initLongValue(); - - if (valueAsLong == null) { - return new DruidLongPredicate() - { - @Override - public boolean applyLong(long input) - { - return false; - } - }; - } else { - // store the primitive, so we don't unbox for every comparison - final long unboxedLong = valueAsLong.longValue(); - return new DruidLongPredicate() - { - @Override - public boolean applyLong(long input) - { - return input == unboxedLong; - } - }; - } + initLongPredicate(); + return longPredicate; + } + + @Override + public DruidFloatPredicate makeFloatPredicate() + { + initFloatPredicate(); + return floatPredicate; } }; return new DimensionPredicateFilter(dimension, predicateFactory, extractionFn); @@ -212,17 +193,57 @@ public int hashCode() } - private void initLongValue() + private void initLongPredicate() + { + if (longPredicate != null) { + return; + } + synchronized (initLock) { + if (longPredicate != null) { + return; + } + final Long valueAsLong = GuavaUtils.tryParseLong(value); + if (valueAsLong == null) { + longPredicate = DruidLongPredicate.ALWAYS_FALSE; + } else { + // store the primitive, so we don't unbox for every comparison + final long unboxedLong = valueAsLong.longValue(); + longPredicate = new DruidLongPredicate() + { + @Override + public boolean applyLong(long input) + { + return input == unboxedLong; + } + }; + } + } + } + + private void initFloatPredicate() { - if (longsInitialized) { + if (floatPredicate != null) { return; } synchronized (initLock) { - if (longsInitialized) { + if (floatPredicate != null) { return; } - valueAsLong = GuavaUtils.tryParseLong(value); - longsInitialized = true; + final Float valueAsFloat = Floats.tryParse(value); + + if (valueAsFloat == null) { + floatPredicate = DruidFloatPredicate.ALWAYS_FALSE; + } else { + final int floatBits = Float.floatToIntBits(valueAsFloat); + floatPredicate = new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + return Float.floatToIntBits(input) == floatBits; + } + }; + } } } } diff --git a/processing/src/main/java/io/druid/query/filter/ValueMatcherColumnSelectorStrategyFactory.java b/processing/src/main/java/io/druid/query/filter/ValueMatcherColumnSelectorStrategyFactory.java index 2343b33ca23d..55b7f608a5a6 100644 --- a/processing/src/main/java/io/druid/query/filter/ValueMatcherColumnSelectorStrategyFactory.java +++ b/processing/src/main/java/io/druid/query/filter/ValueMatcherColumnSelectorStrategyFactory.java @@ -21,6 +21,7 @@ import io.druid.java.util.common.IAE; import io.druid.query.dimension.ColumnSelectorStrategyFactory; +import io.druid.segment.ColumnValueSelector; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ValueType; @@ -41,15 +42,19 @@ public static ValueMatcherColumnSelectorStrategyFactory instance() @Override public ValueMatcherColumnSelectorStrategy makeColumnSelectorStrategy( - ColumnCapabilities capabilities + ColumnCapabilities capabilities, ColumnValueSelector selector ) { ValueType type = capabilities.getType(); switch (type) { case STRING: return new StringValueMatcherColumnSelectorStrategy(); + case LONG: + return new LongValueMatcherColumnSelectorStrategy(); + case FLOAT: + return new FloatValueMatcherColumnSelectorStrategy(); default: - throw new IAE("Cannot create query type helper from invalid type [%s]", type); + throw new IAE("Cannot create column selector strategy from invalid type [%s]", type); } } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java index d78f64bb61f1..6db482c1ab63 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java @@ -49,6 +49,7 @@ import io.druid.segment.DimensionSelector; import io.druid.segment.StorageAdapter; import io.druid.segment.VirtualColumns; +import io.druid.segment.column.ValueType; import io.druid.segment.data.IndexedInts; import io.druid.segment.filter.Filters; import org.joda.time.DateTime; @@ -326,8 +327,18 @@ public RowIterator(GroupByQuery query, final Cursor cursor, ByteBuffer metricsBu for (int i = 0; i < dimensionSpecs.size(); ++i) { final DimensionSpec dimSpec = dimensionSpecs.get(i); + if (dimSpec.getOutputType() != ValueType.STRING) { + throw new UnsupportedOperationException( + "GroupBy v1 only supports dimensions with an outputType of STRING." + ); + } + final DimensionSelector selector = cursor.makeDimensionSelector(dimSpec); if (selector != null) { + if (selector.getValueCardinality() == DimensionSelector.CARDINALITY_UNKNOWN) { + throw new UnsupportedOperationException( + "GroupBy v1 does not support dimension selectors with unknown cardinality."); + } dimensions.add(selector); dimNames.add(dimSpec.getOutputName()); } 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 1d3684e20ddc..a504731ddd95 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java @@ -237,7 +237,7 @@ public static Map rowSignatureFor(final GroupByQuery query) final ImmutableMap.Builder types = ImmutableMap.builder(); for (DimensionSpec dimensionSpec : query.getDimensions()) { - types.put(dimensionSpec.getOutputName(), ValueType.STRING); + types.put(dimensionSpec.getOutputName(), dimensionSpec.getOutputType()); } for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) { diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java index a6deacf14952..a4b68cc7c680 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java @@ -179,6 +179,7 @@ public CloseableGrouperIterator make() Pair, Accumulator, Row>> pair = RowBasedGrouperHelper.createGrouperAccumulatorPair( query, false, + null, config, mergeBufferHolder.get(), concurrencyHint, diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java index a38822fa2bae..5aa1d7bd790c 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java @@ -22,7 +22,6 @@ import com.google.common.base.Function; import com.google.common.base.Strings; import com.google.common.collect.Maps; -import com.google.common.primitives.Ints; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidPool; import io.druid.data.input.MapBasedRow; @@ -35,10 +34,15 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.query.ColumnSelectorPlus; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.dimension.ColumnSelectorStrategy; import io.druid.query.dimension.ColumnSelectorStrategyFactory; import io.druid.query.groupby.GroupByQuery; import io.druid.query.groupby.GroupByQueryConfig; +import io.druid.query.groupby.epinephelinae.column.DictionaryBuildingStringGroupByColumnSelectorStrategy; +import io.druid.query.groupby.epinephelinae.column.FloatGroupByColumnSelectorStrategy; +import io.druid.query.groupby.epinephelinae.column.GroupByColumnSelectorPlus; +import io.druid.query.groupby.epinephelinae.column.GroupByColumnSelectorStrategy; +import io.druid.query.groupby.epinephelinae.column.LongGroupByColumnSelectorStrategy; +import io.druid.query.groupby.epinephelinae.column.StringGroupByColumnSelectorStrategy; import io.druid.query.groupby.strategy.GroupByStrategyV2; import io.druid.segment.ColumnValueSelector; import io.druid.segment.Cursor; @@ -48,7 +52,6 @@ import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ValueType; import io.druid.segment.VirtualColumns; -import io.druid.segment.data.IndexedInts; import io.druid.segment.filter.Filters; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -174,173 +177,28 @@ private static class GroupByStrategyFactory implements ColumnSelectorStrategyFac { @Override public GroupByColumnSelectorStrategy makeColumnSelectorStrategy( - ColumnCapabilities capabilities + ColumnCapabilities capabilities, ColumnValueSelector selector ) { ValueType type = capabilities.getType(); switch(type) { case STRING: - return new StringGroupByColumnSelectorStrategy(); + DimensionSelector dimSelector = (DimensionSelector) selector; + if (dimSelector.getValueCardinality() >= 0) { + return new StringGroupByColumnSelectorStrategy(); + } else { + return new DictionaryBuildingStringGroupByColumnSelectorStrategy(); + } + case LONG: + return new LongGroupByColumnSelectorStrategy(); + case FLOAT: + return new FloatGroupByColumnSelectorStrategy(); default: throw new IAE("Cannot create query type helper from invalid type [%s]", type); } } } - /** - * Contains a collection of query processing methods for type-specific operations used exclusively by - * GroupByQueryEngineV2. - * - * Each GroupByColumnSelectorStrategy is associated with a single dimension. - */ - private interface GroupByColumnSelectorStrategy extends ColumnSelectorStrategy - { - /** - * Return the size, in bytes, of this dimension's values in the grouping key. - * - * For example, a String implementation would return 4, the size of an int. - * - * @return size, in bytes, of this dimension's values in the grouping key. - */ - int getGroupingKeySize(); - - /** - * Read a value from a grouping key and add it to the group by query result map, using the output name specified - * in a DimensionSpec. - * - * An implementation may choose to not add anything to the result map - * (e.g., as the String implementation does for empty rows) - * - * selectorPlus provides access to: - * - the keyBufferPosition offset from which to read the value - * - the dimension value selector - * - the DimensionSpec for this dimension from the query - * - * @param selectorPlus dimension info containing the key offset, value selector, and dimension spec - * @param resultMap result map for the group by query being served - * @param key grouping key - */ - void processValueFromGroupingKey( - GroupByColumnSelectorPlus selectorPlus, - ByteBuffer key, - Map resultMap - ); - - /** - * Retrieve a row object from the ColumnSelectorPlus and put it in valuess at columnIndex. - * - * @param selector Value selector for a column. - * @param columnIndex Index of the column within the row values array - * @param valuess Row values array, one index per column - */ - void initColumnValues(ColumnValueSelector selector, int columnIndex, Object[] valuess); - - /** - * Read the first value within a row values object (IndexedInts, IndexedLongs, etc.) and write that value - * to the keyBuffer at keyBufferPosition. If rowSize is 0, write GROUP_BY_MISSING_VALUE instead. - * - * If the size of the row is > 0, write 1 to stack[] at columnIndex, otherwise write 0. - * - * @param keyBufferPosition Starting offset for this column's value within the grouping key. - * @param columnIndex Index of the column within the row values array - * @param rowObj Row value object for this column (e.g., IndexedInts) - * @param keyBuffer grouping key - * @param stack array containing the current within-row value index for each column - */ - void initGroupingKeyColumnValue(int keyBufferPosition, int columnIndex, Object rowObj, ByteBuffer keyBuffer, int[] stack); - - /** - * If rowValIdx is less than the size of rowObj (haven't handled all of the row values): - * First, read the value at rowValIdx from a rowObj and write that value to the keyBuffer at keyBufferPosition. - * Then return true - * - * Otherwise, return false. - * - * @param keyBufferPosition Starting offset for this column's value within the grouping key. - * @param rowObj Row value object for this column (e.g., IndexedInts) - * @param rowValIdx Index of the current value being grouped on within the row - * @param keyBuffer grouping key - * @return true if rowValIdx < size of rowObj, false otherwise - */ - boolean checkRowIndexAndAddValueToGroupingKey(int keyBufferPosition, Object rowObj, int rowValIdx, ByteBuffer keyBuffer); - } - - private static class StringGroupByColumnSelectorStrategy implements GroupByColumnSelectorStrategy - { - private static final int GROUP_BY_MISSING_VALUE = -1; - - @Override - public int getGroupingKeySize() - { - return Ints.BYTES; - } - - @Override - public void processValueFromGroupingKey(GroupByColumnSelectorPlus selectorPlus, ByteBuffer key, Map resultMap) - { - final int id = key.getInt(selectorPlus.getKeyBufferPosition()); - - // GROUP_BY_MISSING_VALUE is used to indicate empty rows, which are omitted from the result map. - if (id != GROUP_BY_MISSING_VALUE) { - resultMap.put( - selectorPlus.getOutputName(), - ((DimensionSelector) selectorPlus.getSelector()).lookupName(id) - ); - } else { - resultMap.put(selectorPlus.getOutputName(), ""); - } - } - - @Override - public void initColumnValues(ColumnValueSelector selector, int columnIndex, Object[] valuess) - { - DimensionSelector dimSelector = (DimensionSelector) selector; - IndexedInts row = dimSelector.getRow(); - valuess[columnIndex] = row; - } - - @Override - public void initGroupingKeyColumnValue(int keyBufferPosition, int columnIndex, Object rowObj, ByteBuffer keyBuffer, int[] stack) - { - IndexedInts row = (IndexedInts) rowObj; - int rowSize = row.size(); - - initializeGroupingKeyV2Dimension(row, rowSize, keyBuffer, keyBufferPosition); - stack[columnIndex] = rowSize == 0 ? 0 : 1; - } - - @Override - public boolean checkRowIndexAndAddValueToGroupingKey(int keyBufferPosition, Object rowObj, int rowValIdx, ByteBuffer keyBuffer) - { - IndexedInts row = (IndexedInts) rowObj; - int rowSize = row.size(); - - if (rowValIdx < rowSize) { - keyBuffer.putInt( - keyBufferPosition, - row.get(rowValIdx) - ); - return true; - } else { - return false; - } - } - - private void initializeGroupingKeyV2Dimension( - final IndexedInts values, - final int rowSize, - final ByteBuffer keyBuffer, - final int keyBufferPosition - ) - { - if (rowSize == 0) { - keyBuffer.putInt(keyBufferPosition, GROUP_BY_MISSING_VALUE); - } else { - keyBuffer.putInt(keyBufferPosition, values.get(0)); - } - } - } - private static class GroupByEngineIterator implements Iterator, Closeable { private final GroupByQuery query; @@ -594,28 +452,4 @@ public void reset() // No state, nothing to reset } } - - private static class GroupByColumnSelectorPlus extends ColumnSelectorPlus - { - /** - * Indicates the offset of this dimension's value within the grouping key. - */ - private int keyBufferPosition; - - public GroupByColumnSelectorPlus(ColumnSelectorPlus baseInfo, int keyBufferPosition) - { - super( - baseInfo.getName(), - baseInfo.getOutputName(), - baseInfo.getColumnSelectorStrategy(), - baseInfo.getSelector() - ); - this.keyBufferPosition = keyBufferPosition; - } - - public int getKeyBufferPosition() - { - return keyBufferPosition; - } - } } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByRowProcessor.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByRowProcessor.java index b3821767228a..daf6576dc72a 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByRowProcessor.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByRowProcessor.java @@ -156,6 +156,7 @@ public CloseableGrouperIterator make() Pair, Accumulator, Row>> pair = RowBasedGrouperHelper.createGrouperAccumulatorPair( query, true, + rowSignature, querySpecificConfig, mergeBufferHolder.get(), -1, diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/Groupers.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/Groupers.java index e916af27bbe6..2990a07548e5 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/Groupers.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/Groupers.java @@ -31,12 +31,28 @@ private Groupers() // No instantiation } + private static final int C1 = 0xcc9e2d51; + private static final int C2 = 0x1b873593; + + /* + * This method was rewritten in Java from an intermediate step of the Murmur hash function in + * https://github.com/aappleby/smhasher/blob/master/src/MurmurHash3.cpp, which contained the + * following header: + * + * MurmurHash3 was written by Austin Appleby, and is placed in the public domain. The author + * hereby disclaims copyright to this source code. + */ + static int smear(int hashCode) { + return C2 * Integer.rotateLeft(hashCode * C1, 15); + } + public static int hash(final Object obj) { // Mask off the high bit so we can use that to determine if a bucket is used or not. - // Also apply the same XOR transformation that j.u.HashMap applies, to improve distribution. + // Also apply the smear function, to improve distribution. final int code = obj.hashCode(); - return (code ^ (code >>> 16)) & 0x7fffffff; + return smear(code) & 0x7fffffff; + } public static Iterator> mergeIterators( diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index dbb3228b2cac..350163f56f40 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -25,30 +25,39 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import com.google.common.base.Supplier; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.primitives.Chars; +import com.google.common.primitives.Floats; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.granularity.AllGranularity; +import io.druid.java.util.common.IAE; import io.druid.java.util.common.Pair; import io.druid.java.util.common.guava.Accumulator; import io.druid.query.QueryInterruptedException; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.dimension.DimensionSpec; import io.druid.query.groupby.GroupByQuery; import io.druid.query.groupby.GroupByQueryConfig; -import io.druid.query.groupby.GroupByQueryHelper; import io.druid.query.groupby.RowBasedColumnSelectorFactory; import io.druid.query.groupby.strategy.GroupByStrategyV2; import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.ColumnValueSelector; +import io.druid.segment.DimensionHandlerUtils; import io.druid.segment.DimensionSelector; +import io.druid.segment.FloatColumnSelector; +import io.druid.segment.LongColumnSelector; +import io.druid.segment.column.ValueType; import io.druid.segment.data.IndexedInts; import org.joda.time.DateTime; import java.io.Closeable; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; import java.util.List; @@ -57,10 +66,15 @@ // this class contains shared code between GroupByMergingQueryRunnerV2 and GroupByRowProcessor public class RowBasedGrouperHelper { - + /** + * If isInputRaw is true, transformations such as timestamp truncation and extraction functions have not + * been applied to the input rows yet, for example, in a nested query, if an extraction function is being + * applied in the outer query to a field of the inner query. This method must apply those transformations. + */ public static Pair, Accumulator, Row>> createGrouperAccumulatorPair( final GroupByQuery query, final boolean isInputRaw, + final Map rawInputRowSignature, final GroupByQueryConfig config, final ByteBuffer buffer, final int concurrencyHint, @@ -72,18 +86,21 @@ public static Pair, Accumulator, Row>> // concurrencyHint >= 1 for concurrent groupers, -1 for single-threaded Preconditions.checkArgument(concurrencyHint >= 1 || concurrencyHint == -1, "invalid concurrencyHint"); + final List valueTypes = DimensionHandlerUtils.getValueTypesFromDimensionSpecs(query.getDimensions()); + final GroupByQueryConfig querySpecificConfig = config.withOverrides(query); final boolean includeTimestamp = GroupByStrategyV2.getUniversalTimestamp(query) == null; final Grouper.KeySerdeFactory keySerdeFactory = new RowBasedKeySerdeFactory( includeTimestamp, query.getContextSortByDimsFirst(), query.getDimensions().size(), - querySpecificConfig.getMaxMergingDictionarySize() / (concurrencyHint == -1 ? 1 : concurrencyHint) + querySpecificConfig.getMaxMergingDictionarySize() / (concurrencyHint == -1 ? 1 : concurrencyHint), + valueTypes ); final ThreadLocal columnSelectorRow = new ThreadLocal<>(); final ColumnSelectorFactory columnSelectorFactory = RowBasedColumnSelectorFactory.create( columnSelectorRow, - GroupByQueryHelper.rowSignatureFor(query) + rawInputRowSignature ); final Grouper grouper; if (concurrencyHint == -1) { @@ -114,14 +131,15 @@ public static Pair, Accumulator, Row>> ); } - final DimensionSelector[] dimensionSelectors; + final Supplier[] inputRawSuppliers; if (isInputRaw) { - dimensionSelectors = new DimensionSelector[query.getDimensions().size()]; - for (int i = 0; i < dimensionSelectors.length; i++) { - dimensionSelectors[i] = columnSelectorFactory.makeDimensionSelector(query.getDimensions().get(i)); - } + inputRawSuppliers = getValueSuppliersForDimensions( + columnSelectorFactory, + query.getDimensions(), + rawInputRowSignature + ); } else { - dimensionSelectors = null; + inputRawSuppliers = null; } final Accumulator, Row> accumulator = new Accumulator, Row>() @@ -168,14 +186,34 @@ public Grouper accumulate( } for (int i = dimStart; i < key.length; i++) { - final String value; + final ValueType type = valueTypes.get(i - dimStart); + Object valObj; if (isInputRaw) { - IndexedInts index = dimensionSelectors[i - dimStart].getRow(); - value = index.size() == 0 ? "" : dimensionSelectors[i - dimStart].lookupName(index.get(0)); + valObj = inputRawSuppliers[i - dimStart].get(); } else { - value = (String) row.getRaw(query.getDimensions().get(i - dimStart).getOutputName()); + valObj = row.getRaw(query.getDimensions().get(i - dimStart).getOutputName()); } - key[i] = Strings.nullToEmpty(value); + // convert values to the output type specified by the DimensionSpec, for merging purposes + switch (type) { + case STRING: + valObj = valObj == null ? "" : valObj.toString(); + break; + case LONG: + valObj = DimensionHandlerUtils.convertObjectToLong(valObj); + if (valObj == null) { + valObj = 0L; + } + break; + case FLOAT: + valObj = DimensionHandlerUtils.convertObjectToFloat(valObj); + if (valObj == null) { + valObj = 0.0f; + } + break; + default: + throw new IAE("invalid type: [%s]", type); + } + key[i] = (Comparable) valObj; } final boolean didAggregate = theGrouper.aggregate(new RowBasedKey(key)); @@ -224,9 +262,10 @@ public Row apply(Grouper.Entry entry) // Add dimensions. for (int i = dimStart; i < entry.getKey().getKey().length; i++) { + Object dimVal = entry.getKey().getKey()[i]; theMap.put( query.getDimensions().get(i - dimStart).getOutputName(), - Strings.emptyToNull((String) entry.getKey().getKey()[i]) + dimVal instanceof String ? Strings.emptyToNull((String)dimVal) : dimVal ); } @@ -254,9 +293,14 @@ static class RowBasedKey @JsonCreator public static RowBasedKey fromJsonArray(final Object[] key) { - // Type info is lost during serde. We know we don't want ints as timestamps, so adjust. - if (key.length > 0 && key[0] instanceof Integer) { - key[0] = ((Integer) key[0]).longValue(); + // Type info is lost during serde: + // Floats may be deserialized as doubles, Longs may be deserialized as integers, convert them back + for (int i = 0; i < key.length; i++) { + if (key[i] instanceof Integer) { + key[i] = ((Integer) key[i]).longValue(); + } else if (key[i] instanceof Double) { + key[i] = ((Double) key[i]).floatValue(); + } } return new RowBasedKey(key); @@ -296,25 +340,94 @@ public String toString() } } + private static Supplier[] getValueSuppliersForDimensions( + final ColumnSelectorFactory columnSelectorFactory, + final List dimensions, + final Map rawInputRowSignature + ) + { + final Supplier[] inputRawSuppliers = new Supplier[dimensions.size()]; + for (int i = 0; i < dimensions.size(); i++) { + final ColumnValueSelector selector = DimensionHandlerUtils.getColumnValueSelectorFromDimensionSpec( + dimensions.get(i), + columnSelectorFactory + ); + ValueType type = rawInputRowSignature.get(dimensions.get(i).getDimension()); + if (type == null) { + // Subquery post-aggs aren't added to the rowSignature (see rowSignatureFor() in GroupByQueryHelper) because + // their types aren't known, so default to String handling. + type = ValueType.STRING; + } + switch (type) { + case STRING: + inputRawSuppliers[i] = new Supplier() + { + @Override + public Object get() + { + final String value; + IndexedInts index = ((DimensionSelector) selector).getRow(); + value = index.size() == 0 + ? "" + : ((DimensionSelector) selector).lookupName(index.get(0)); + return Strings.nullToEmpty(value); + } + }; + break; + case LONG: + inputRawSuppliers[i] = new Supplier() + { + @Override + public Object get() + { + return ((LongColumnSelector) selector).get(); + } + }; + break; + case FLOAT: + inputRawSuppliers[i] = new Supplier() + { + @Override + public Object get() + { + return ((FloatColumnSelector) selector).get(); + } + }; + break; + default: + throw new IAE("invalid type: [%s]", type); + } + } + return inputRawSuppliers; + } + private static class RowBasedKeySerdeFactory implements Grouper.KeySerdeFactory { private final boolean includeTimestamp; private final boolean sortByDimsFirst; private final int dimCount; private final long maxDictionarySize; + private final List valueTypes; - RowBasedKeySerdeFactory(boolean includeTimestamp, boolean sortByDimsFirst, int dimCount, long maxDictionarySize) + RowBasedKeySerdeFactory( + boolean includeTimestamp, + boolean sortByDimsFirst, + int dimCount, + long maxDictionarySize, + List valueTypes + ) { this.includeTimestamp = includeTimestamp; this.sortByDimsFirst = sortByDimsFirst; this.dimCount = dimCount; this.maxDictionarySize = maxDictionarySize; + this.valueTypes = valueTypes; } @Override public Grouper.KeySerde factorize() { - return new RowBasedKeySerde(includeTimestamp, sortByDimsFirst, dimCount, maxDictionarySize); + return new RowBasedKeySerde(includeTimestamp, sortByDimsFirst, dimCount, maxDictionarySize, valueTypes); } @Override @@ -366,7 +479,7 @@ public int compare(RowBasedKey key1, RowBasedKey key2) private static int compareDimsInRows(RowBasedKey key1, RowBasedKey key2, int dimStart) { for (int i = dimStart; i < key1.getKey().length; i++) { - final int cmp = ((String) key1.getKey()[i]).compareTo((String) key2.getKey()[i]); + final int cmp = ((Comparable) key1.getKey()[i]).compareTo(key2.getKey()[i]); if (cmp != 0) { return cmp; } @@ -388,6 +501,8 @@ private static class RowBasedKeySerde implements Grouper.KeySerde private final ByteBuffer keyBuffer; private final List dictionary = Lists.newArrayList(); private final Map reverseDictionary = Maps.newHashMap(); + private final List valueTypes; + private final List serdeHelpers; // Size limiting for the dictionary, in (roughly estimated) bytes. private final long maxDictionarySize; @@ -400,14 +515,17 @@ private static class RowBasedKeySerde implements Grouper.KeySerde final boolean includeTimestamp, final boolean sortByDimsFirst, final int dimCount, - final long maxDictionarySize + final long maxDictionarySize, + final List valueTypes ) { this.includeTimestamp = includeTimestamp; this.sortByDimsFirst = sortByDimsFirst; this.dimCount = dimCount; this.maxDictionarySize = maxDictionarySize; - this.keySize = (includeTimestamp ? Longs.BYTES : 0) + dimCount * Ints.BYTES; + this.valueTypes = valueTypes; + this.serdeHelpers = makeSerdeHelpers(); + this.keySize = (includeTimestamp ? Longs.BYTES : 0) + getTotalKeySize(); this.keyBuffer = ByteBuffer.allocate(keySize); } @@ -435,13 +553,10 @@ public ByteBuffer toByteBuffer(RowBasedKey key) } else { dimStart = 0; } - for (int i = dimStart; i < key.getKey().length; i++) { - final int id = addToDictionary((String) key.getKey()[i]); - if (id < 0) { + if (!serdeHelpers.get(i - dimStart).putToKeyBuffer(key, i)) { return null; } - keyBuffer.putInt(id); } keyBuffer.flip(); @@ -467,7 +582,8 @@ public RowBasedKey fromByteBuffer(ByteBuffer buffer, int position) } for (int i = dimStart; i < key.length; i++) { - key[i] = dictionary.get(buffer.getInt(dimsPosition + (Ints.BYTES * (i - dimStart)))); + // Writes value from buffer to key[i] + serdeHelpers.get(i - dimStart).getFromByteBuffer(buffer, dimsPosition, i, key); } return new RowBasedKey(key); @@ -496,6 +612,7 @@ public Grouper.KeyComparator bufferComparator() public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) { final int cmp = compareDimsInBuffersForNullFudgeTimestamp( + serdeHelpers, sortableIds, dimCount, lhsBuffer, @@ -523,6 +640,7 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, } return compareDimsInBuffersForNullFudgeTimestamp( + serdeHelpers, sortableIds, dimCount, lhsBuffer, @@ -540,9 +658,11 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) { for (int i = 0; i < dimCount; i++) { - final int cmp = Ints.compare( - sortableIds[lhsBuffer.getInt(lhsPosition + (Ints.BYTES * i))], - sortableIds[rhsBuffer.getInt(rhsPosition + (Ints.BYTES * i))] + final int cmp = serdeHelpers.get(i).compare( + lhsBuffer, + rhsBuffer, + lhsPosition, + rhsPosition ); if (cmp != 0) { @@ -557,6 +677,7 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, } private static int compareDimsInBuffersForNullFudgeTimestamp( + List serdeHelpers, int[] sortableIds, int dimCount, ByteBuffer lhsBuffer, @@ -566,11 +687,12 @@ private static int compareDimsInBuffersForNullFudgeTimestamp( ) { for (int i = 0; i < dimCount; i++) { - final int cmp = Ints.compare( - sortableIds[lhsBuffer.getInt(lhsPosition + Longs.BYTES + (Ints.BYTES * i))], - sortableIds[rhsBuffer.getInt(rhsPosition + Longs.BYTES + (Ints.BYTES * i))] + final int cmp = serdeHelpers.get(i).compare( + lhsBuffer, + rhsBuffer, + lhsPosition + Longs.BYTES, + rhsPosition + Longs.BYTES ); - if (cmp != 0) { return cmp; } @@ -612,5 +734,204 @@ private int addToDictionary(final String s) } return idx; } + + private int getTotalKeySize() + { + int size = 0; + for (RowBasedKeySerdeHelper helper : serdeHelpers) { + size += helper.getKeyBufferValueSize(); + } + return size; + } + + private List makeSerdeHelpers() + { + List helpers = new ArrayList<>(); + int keyBufferPosition = 0; + for (ValueType valType : valueTypes) { + RowBasedKeySerdeHelper helper; + switch (valType) { + case STRING: + helper = new StringRowBasedKeySerdeHelper(keyBufferPosition); + break; + case LONG: + helper = new LongRowBasedKeySerdeHelper(keyBufferPosition); + break; + case FLOAT: + helper = new FloatRowBasedKeySerdeHelper(keyBufferPosition); + break; + default: + throw new IAE("invalid type: %s", valType); + } + keyBufferPosition += helper.getKeyBufferValueSize(); + helpers.add(helper); + } + return helpers; + } + + private interface RowBasedKeySerdeHelper + { + /** + * @return The size in bytes for a value of the column handled by this SerdeHelper. + */ + int getKeyBufferValueSize(); + + /** + * Read a value from RowBasedKey at `idx` and put the value at the current position of RowBasedKeySerde's keyBuffer. + * advancing the position by the size returned by getKeyBufferValueSize(). + * + * If an internal resource limit has been reached and the value could not be added to the keyBuffer, + * (e.g., maximum dictionary size exceeded for Strings), this method returns false. + * + * @param key RowBasedKey containing the grouping key values for a row. + * @param idx Index of the grouping key column within that this SerdeHelper handles + * @return true if the value was added to the key, false otherwise + */ + boolean putToKeyBuffer(RowBasedKey key, int idx); + + /** + * Read a value from a ByteBuffer containing a grouping key in the same format as RowBasedKeySerde's keyBuffer and + * put the value in `dimValues` at `dimValIdx`. + * + * The value to be read resides in the buffer at position (`initialOffset` + the SerdeHelper's keyBufferPosition). + * + * @param buffer ByteBuffer containing an array of grouping keys for a row + * @param initialOffset Offset where non-timestamp grouping key columns start, needed because timestamp is not + * always included in the buffer. + * @param dimValIdx Index within dimValues to store the value read from the buffer + * @param dimValues Output array containing grouping key values for a row + */ + void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValIdx, Comparable[] dimValues); + + /** + * Compare the values at lhsBuffer[lhsPosition] and rhsBuffer[rhsPosition] using the natural ordering + * for this SerdeHelper's value type. + * + * @param lhsBuffer ByteBuffer containing an array of grouping keys for a row + * @param rhsBuffer ByteBuffer containing an array of grouping keys for a row + * @param lhsPosition Position of value within lhsBuffer + * @param rhsPosition Position of value within rhsBuffer + * @return Negative number if lhs < rhs, positive if lhs > rhs, 0 if lhs == rhs + */ + int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition); + } + + private class StringRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper + { + final int keyBufferPosition; + + public StringRowBasedKeySerdeHelper(int keyBufferPosition) + { + this.keyBufferPosition = keyBufferPosition; + } + + @Override + public int getKeyBufferValueSize() + { + return Ints.BYTES; + } + + @Override + public boolean putToKeyBuffer(RowBasedKey key, int idx) + { + final int id = addToDictionary((String) key.getKey()[idx]); + if (id < 0) { + return false; + } + keyBuffer.putInt(id); + return true; + } + + @Override + public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValIdx, Comparable[] dimValues) + { + dimValues[dimValIdx] = dictionary.get(buffer.getInt(initialOffset + keyBufferPosition)); + } + + @Override + public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + { + return Ints.compare( + sortableIds[lhsBuffer.getInt(lhsPosition + keyBufferPosition)], + sortableIds[rhsBuffer.getInt(rhsPosition + keyBufferPosition)] + ); + } + } + + private class LongRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper + { + final int keyBufferPosition; + + public LongRowBasedKeySerdeHelper(int keyBufferPosition) + { + this.keyBufferPosition = keyBufferPosition; + } + + @Override + public int getKeyBufferValueSize() + { + return Longs.BYTES; + } + + @Override + public boolean putToKeyBuffer(RowBasedKey key, int idx) + { + keyBuffer.putLong((Long) key.getKey()[idx]); + return true; + } + + @Override + public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValIdx, Comparable[] dimValues) + { + dimValues[dimValIdx] = buffer.getLong(initialOffset + keyBufferPosition); + } + + @Override + public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + { + return Longs.compare( + lhsBuffer.getLong(lhsPosition + keyBufferPosition), + rhsBuffer.getLong(rhsPosition + keyBufferPosition) + ); + } + } + + private class FloatRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper + { + final int keyBufferPosition; + + public FloatRowBasedKeySerdeHelper(int keyBufferPosition) + { + this.keyBufferPosition = keyBufferPosition; + } + + @Override + public int getKeyBufferValueSize() + { + return Floats.BYTES; + } + + @Override + public boolean putToKeyBuffer(RowBasedKey key, int idx) + { + keyBuffer.putFloat((Float) key.getKey()[idx]); + return true; + } + + @Override + public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValIdx, Comparable[] dimValues) + { + dimValues[dimValIdx] = buffer.getFloat(initialOffset + keyBufferPosition); + } + + @Override + public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + { + return Float.compare( + lhsBuffer.getFloat(lhsPosition + keyBufferPosition), + rhsBuffer.getFloat(rhsPosition + keyBufferPosition) + ); + } + } } } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/DictionaryBuildingStringGroupByColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/DictionaryBuildingStringGroupByColumnSelectorStrategy.java new file mode 100644 index 000000000000..528f432baecf --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/DictionaryBuildingStringGroupByColumnSelectorStrategy.java @@ -0,0 +1,86 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.groupby.epinephelinae.column; + +import com.google.common.collect.Lists; +import io.druid.segment.ColumnValueSelector; +import io.druid.segment.DimensionSelector; +import io.druid.segment.data.ArrayBasedIndexedInts; +import io.druid.segment.data.IndexedInts; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; + +/** + * A String strategy that builds an internal String<->Integer dictionary for + * DimensionSelectors that return false for nameLookupPossibleInAdvance() + */ +public class DictionaryBuildingStringGroupByColumnSelectorStrategy extends StringGroupByColumnSelectorStrategy +{ + private static final int GROUP_BY_MISSING_VALUE = -1; + + private int nextId = 0; + private final List dictionary = Lists.newArrayList(); + private final Object2IntOpenHashMap reverseDictionary = new Object2IntOpenHashMap<>(); + { + reverseDictionary.defaultReturnValue(-1); + } + + @Override + public void processValueFromGroupingKey(GroupByColumnSelectorPlus selectorPlus, ByteBuffer key, Map resultMap) + { + final int id = key.getInt(selectorPlus.getKeyBufferPosition()); + + // GROUP_BY_MISSING_VALUE is used to indicate empty rows, which are omitted from the result map. + if (id != GROUP_BY_MISSING_VALUE) { + final String value = dictionary.get(id); + resultMap.put( + selectorPlus.getOutputName(), + value + ); + } else { + resultMap.put(selectorPlus.getOutputName(), ""); + } + } + + @Override + public void initColumnValues(ColumnValueSelector selector, int columnIndex, Object[] valuess) + { + final DimensionSelector dimSelector = (DimensionSelector) selector; + final IndexedInts row = dimSelector.getRow(); + final int[] newIds = new int[row.size()]; + + for (int i = 0; i < row.size(); i++) { + final String value = dimSelector.lookupName(row.get(i)); + final int dictId = reverseDictionary.getInt(value); + if (dictId < 0) { + dictionary.add(value); + reverseDictionary.put(value, nextId); + newIds[i] = nextId; + nextId++; + } else { + newIds[i] = dictId; + } + } + valuess[columnIndex] = ArrayBasedIndexedInts.of(newIds); + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/FloatGroupByColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/FloatGroupByColumnSelectorStrategy.java new file mode 100644 index 000000000000..3e63be4d59d6 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/FloatGroupByColumnSelectorStrategy.java @@ -0,0 +1,71 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.groupby.epinephelinae.column; + +import com.google.common.primitives.Floats; +import io.druid.segment.ColumnValueSelector; +import io.druid.segment.FloatColumnSelector; + +import java.nio.ByteBuffer; +import java.util.Map; + +public class FloatGroupByColumnSelectorStrategy implements GroupByColumnSelectorStrategy +{ + + @Override + public int getGroupingKeySize() + { + return Floats.BYTES; + } + + @Override + public void processValueFromGroupingKey( + GroupByColumnSelectorPlus selectorPlus, ByteBuffer key, Map resultMap + ) + { + final float val = key.getFloat(selectorPlus.getKeyBufferPosition()); + resultMap.put(selectorPlus.getOutputName(), val); + } + + @Override + public void initColumnValues(ColumnValueSelector selector, int columnIndex, Object[] valuess) + { + valuess[columnIndex] = ((FloatColumnSelector) selector).get(); + } + + @Override + public void initGroupingKeyColumnValue( + int keyBufferPosition, int columnIndex, Object rowObj, ByteBuffer keyBuffer, int[] stack + ) + { + keyBuffer.putFloat(keyBufferPosition, (Float) rowObj); + stack[columnIndex] = 1; + } + + @Override + public boolean checkRowIndexAndAddValueToGroupingKey( + int keyBufferPosition, Object rowObj, int rowValIdx, ByteBuffer keyBuffer + ) + { + // rows from a float column always have a single value, multi-value is not currently supported + // this method handles row values after the first in a multivalued row, so just return false + return false; + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorPlus.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorPlus.java new file mode 100644 index 000000000000..5276e69e5296 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorPlus.java @@ -0,0 +1,46 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.groupby.epinephelinae.column; + +import io.druid.query.ColumnSelectorPlus; + +public class GroupByColumnSelectorPlus extends ColumnSelectorPlus +{ + /** + * Indicates the offset of this dimension's value within the grouping key. + */ + private int keyBufferPosition; + + public GroupByColumnSelectorPlus(ColumnSelectorPlus baseInfo, int keyBufferPosition) + { + super( + baseInfo.getName(), + baseInfo.getOutputName(), + baseInfo.getColumnSelectorStrategy(), + baseInfo.getSelector() + ); + this.keyBufferPosition = keyBufferPosition; + } + + public int getKeyBufferPosition() + { + return keyBufferPosition; + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorStrategy.java new file mode 100644 index 000000000000..7a11258af7e8 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorStrategy.java @@ -0,0 +1,104 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.groupby.epinephelinae.column; + +import io.druid.query.dimension.ColumnSelectorStrategy; +import io.druid.segment.ColumnValueSelector; + +import java.nio.ByteBuffer; +import java.util.Map; + +/** + * Contains a collection of query processing methods for type-specific operations used exclusively by + * GroupByQueryEngineV2. + * + * Each GroupByColumnSelectorStrategy is associated with a single dimension. + */ +public interface GroupByColumnSelectorStrategy extends ColumnSelectorStrategy +{ + /** + * Return the size, in bytes, of this dimension's values in the grouping key. + * + * For example, a String implementation would return 4, the size of an int. + * + * @return size, in bytes, of this dimension's values in the grouping key. + */ + int getGroupingKeySize(); + + /** + * Read a value from a grouping key and add it to the group by query result map, using the output name specified + * in a DimensionSpec. + * + * An implementation may choose to not add anything to the result map + * (e.g., as the String implementation does for empty rows) + * + * selectorPlus provides access to: + * - the keyBufferPosition offset from which to read the value + * - the dimension value selector + * - the DimensionSpec for this dimension from the query + * + * @param selectorPlus dimension info containing the key offset, value selector, and dimension spec + * @param resultMap result map for the group by query being served + * @param key grouping key + */ + void processValueFromGroupingKey( + GroupByColumnSelectorPlus selectorPlus, + ByteBuffer key, + Map resultMap + ); + + /** + * Retrieve a row object from the ColumnSelectorPlus and put it in valuess at columnIndex. + * + * @param selector Value selector for a column. + * @param columnIndex Index of the column within the row values array + * @param valuess Row values array, one index per column + */ + void initColumnValues(ColumnValueSelector selector, int columnIndex, Object[] valuess); + + /** + * Read the first value within a row values object (IndexedInts, IndexedLongs, etc.) and write that value + * to the keyBuffer at keyBufferPosition. If rowSize is 0, write GROUP_BY_MISSING_VALUE instead. + * + * If the size of the row is > 0, write 1 to stack[] at columnIndex, otherwise write 0. + * + * @param keyBufferPosition Starting offset for this column's value within the grouping key. + * @param columnIndex Index of the column within the row values array + * @param rowObj Row value object for this column (e.g., IndexedInts) + * @param keyBuffer grouping key + * @param stack array containing the current within-row value index for each column + */ + void initGroupingKeyColumnValue(int keyBufferPosition, int columnIndex, Object rowObj, ByteBuffer keyBuffer, int[] stack); + + /** + * If rowValIdx is less than the size of rowObj (haven't handled all of the row values): + * First, read the value at rowValIdx from a rowObj and write that value to the keyBuffer at keyBufferPosition. + * Then return true + * + * Otherwise, return false. + * + * @param keyBufferPosition Starting offset for this column's value within the grouping key. + * @param rowObj Row value object for this column (e.g., IndexedInts) + * @param rowValIdx Index of the current value being grouped on within the row + * @param keyBuffer grouping key + * @return true if rowValIdx < size of rowObj, false otherwise + */ + boolean checkRowIndexAndAddValueToGroupingKey(int keyBufferPosition, Object rowObj, int rowValIdx, ByteBuffer keyBuffer); +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/LongGroupByColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/LongGroupByColumnSelectorStrategy.java new file mode 100644 index 000000000000..2e5fed321941 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/LongGroupByColumnSelectorStrategy.java @@ -0,0 +1,71 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.groupby.epinephelinae.column; + +import com.google.common.primitives.Longs; +import io.druid.segment.ColumnValueSelector; +import io.druid.segment.LongColumnSelector; + +import java.nio.ByteBuffer; +import java.util.Map; + +public class LongGroupByColumnSelectorStrategy implements GroupByColumnSelectorStrategy +{ + + @Override + public int getGroupingKeySize() + { + return Longs.BYTES; + } + + @Override + public void processValueFromGroupingKey( + GroupByColumnSelectorPlus selectorPlus, ByteBuffer key, Map resultMap + ) + { + final long val = key.getLong(selectorPlus.getKeyBufferPosition()); + resultMap.put(selectorPlus.getOutputName(), val); + } + + @Override + public void initColumnValues(ColumnValueSelector selector, int columnIndex, Object[] valuess) + { + valuess[columnIndex] = ((LongColumnSelector) selector).get(); + } + + @Override + public void initGroupingKeyColumnValue( + int keyBufferPosition, int columnIndex, Object rowObj, ByteBuffer keyBuffer, int[] stack + ) + { + keyBuffer.putLong(keyBufferPosition, (Long) rowObj); + stack[columnIndex] = 1; + } + + @Override + public boolean checkRowIndexAndAddValueToGroupingKey( + int keyBufferPosition, Object rowObj, int rowValIdx, ByteBuffer keyBuffer + ) + { + // rows from a long column always have a single value, multi-value is not currently supported + // this method handles row values after the first in a multivalued row, so just return false + return false; + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategy.java new file mode 100644 index 000000000000..c531437b2b16 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategy.java @@ -0,0 +1,104 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.groupby.epinephelinae.column; + +import com.google.common.primitives.Ints; +import io.druid.segment.ColumnValueSelector; +import io.druid.segment.DimensionSelector; +import io.druid.segment.data.IndexedInts; + +import java.nio.ByteBuffer; +import java.util.Map; + +public class StringGroupByColumnSelectorStrategy implements GroupByColumnSelectorStrategy +{ + private static final int GROUP_BY_MISSING_VALUE = -1; + + @Override + public int getGroupingKeySize() + { + return Ints.BYTES; + } + + @Override + public void processValueFromGroupingKey(GroupByColumnSelectorPlus selectorPlus, ByteBuffer key, Map resultMap) + { + final int id = key.getInt(selectorPlus.getKeyBufferPosition()); + + // GROUP_BY_MISSING_VALUE is used to indicate empty rows, which are omitted from the result map. + if (id != GROUP_BY_MISSING_VALUE) { + resultMap.put( + selectorPlus.getOutputName(), + ((DimensionSelector) selectorPlus.getSelector()).lookupName(id) + ); + } else { + resultMap.put(selectorPlus.getOutputName(), ""); + } + } + + @Override + public void initColumnValues(ColumnValueSelector selector, int columnIndex, Object[] valuess) + { + DimensionSelector dimSelector = (DimensionSelector) selector; + IndexedInts row = dimSelector.getRow(); + valuess[columnIndex] = row; + } + + @Override + public void initGroupingKeyColumnValue(int keyBufferPosition, int columnIndex, Object rowObj, ByteBuffer keyBuffer, int[] stack) + { + IndexedInts row = (IndexedInts) rowObj; + int rowSize = row.size(); + + initializeGroupingKeyV2Dimension(row, rowSize, keyBuffer, keyBufferPosition); + stack[columnIndex] = rowSize == 0 ? 0 : 1; + } + + @Override + public boolean checkRowIndexAndAddValueToGroupingKey(int keyBufferPosition, Object rowObj, int rowValIdx, ByteBuffer keyBuffer) + { + IndexedInts row = (IndexedInts) rowObj; + int rowSize = row.size(); + + if (rowValIdx < rowSize) { + keyBuffer.putInt( + keyBufferPosition, + row.get(rowValIdx) + ); + return true; + } else { + return false; + } + } + + private void initializeGroupingKeyV2Dimension( + final IndexedInts values, + final int rowSize, + final ByteBuffer keyBuffer, + final int keyBufferPosition + ) + { + if (rowSize == 0) { + keyBuffer.putInt(keyBufferPosition, GROUP_BY_MISSING_VALUE); + } else { + keyBuffer.putInt(keyBufferPosition, values.get(0)); + } + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java index 0b89b120b7ff..da1223f6594d 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java @@ -39,6 +39,7 @@ import io.druid.query.dimension.DimensionSpec; import io.druid.query.ordering.StringComparator; import io.druid.query.ordering.StringComparators; +import io.druid.segment.column.ValueType; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -107,10 +108,25 @@ public Function, Sequence> build( for (int i = 0; i < columns.size(); i++) { final OrderByColumnSpec columnSpec = columns.get(i); + if (aggAndPostAggNames.contains(columnSpec.getDimension())) { + sortingNeeded = true; + break; + } + + final ValueType columnType = getOrderByType(columnSpec, dimensions); + final StringComparator naturalComparator; + if (columnType == ValueType.STRING) { + naturalComparator = StringComparators.LEXICOGRAPHIC; + } else if (columnType == ValueType.LONG || columnType == ValueType.FLOAT) { + naturalComparator = StringComparators.NUMERIC; + } else { + sortingNeeded = true; + break; + } + if (columnSpec.getDirection() != OrderByColumnSpec.Direction.ASCENDING - || !columnSpec.getDimensionComparator().equals(StringComparators.LEXICOGRAPHIC) - || !columnSpec.getDimension().equals(dimensions.get(i).getOutputName()) - || aggAndPostAggNames.contains(columnSpec.getDimension())) { + || !columnSpec.getDimensionComparator().equals(naturalComparator) + || !columnSpec.getDimension().equals(dimensions.get(i).getOutputName())) { sortingNeeded = true; break; } @@ -137,6 +153,17 @@ public LimitSpec merge(LimitSpec other) return this; } + private ValueType getOrderByType(final OrderByColumnSpec columnSpec, final List dimensions) + { + for (DimensionSpec dimSpec : dimensions) { + if (columnSpec.getDimension().equals(dimSpec.getOutputName())) { + return dimSpec.getOutputType(); + } + } + + throw new ISE("Unknown column in order clause[%s]", columnSpec); + } + private Ordering makeComparator( List dimensions, List aggs, List postAggs ) 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 1c64a37bd966..4a6903e0ba5b 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java @@ -39,6 +39,8 @@ import io.druid.query.search.search.SearchQuerySpec; import io.druid.segment.ColumnValueSelector; import io.druid.segment.DimensionSelector; +import io.druid.segment.FloatColumnSelector; +import io.druid.segment.LongColumnSelector; import io.druid.segment.Segment; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ValueType; @@ -70,13 +72,17 @@ private static class SearchColumnSelectorStrategyFactory { @Override public SearchColumnSelectorStrategy makeColumnSelectorStrategy( - ColumnCapabilities capabilities + ColumnCapabilities capabilities, ColumnValueSelector selector ) { ValueType type = capabilities.getType(); switch (type) { case STRING: return new StringSearchColumnSelectorStrategy(); + case LONG: + return new LongSearchColumnSelectorStrategy(); + case FLOAT: + return new FloatSearchColumnSelectorStrategy(); default: throw new IAE("Cannot create query type helper from invalid type [%s]", type); } @@ -135,6 +141,47 @@ public void updateSearchResultSet( } } + public static class LongSearchColumnSelectorStrategy implements SearchColumnSelectorStrategy + { + @Override + public void updateSearchResultSet( + String outputName, + LongColumnSelector selector, + SearchQuerySpec searchQuerySpec, + int limit, + Object2IntRBTreeMap set + ) + { + if (selector != null) { + final String dimVal = String.valueOf(selector.get()); + if (searchQuerySpec.accept(dimVal)) { + set.addTo(new SearchHit(outputName, dimVal), 1); + } + } + } + } + + public static class FloatSearchColumnSelectorStrategy implements SearchColumnSelectorStrategy + { + @Override + public void updateSearchResultSet( + String outputName, + FloatColumnSelector selector, + SearchQuerySpec searchQuerySpec, + int limit, + Object2IntRBTreeMap set + ) + { + if (selector != null) { + final String dimVal = String.valueOf(selector.get()); + if (searchQuerySpec.accept(dimVal)) { + set.addTo(new SearchHit(outputName, dimVal), 1); + } + } + } + } + + @Override public Sequence> run( final Query> input, 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 31a160f08036..4bb3b4fe996e 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java @@ -39,6 +39,7 @@ import io.druid.segment.Cursor; import io.druid.segment.DimensionHandlerUtils; import io.druid.segment.DimensionSelector; +import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.Segment; @@ -67,13 +68,17 @@ public static class SelectStrategyFactory implements ColumnSelectorStrategyFacto { @Override public SelectColumnSelectorStrategy makeColumnSelectorStrategy( - ColumnCapabilities capabilities + ColumnCapabilities capabilities, ColumnValueSelector selector ) { ValueType type = capabilities.getType(); switch(type) { case STRING: return new StringSelectColumnSelectorStrategy(); + case LONG: + return new LongSelectColumnSelectorStrategy(); + case FLOAT: + return new FloatSelectColumnSelectorStrategy(); default: throw new IAE("Cannot create query type helper from invalid type [%s]", type); } @@ -122,6 +127,37 @@ public void addRowValuesToSelectResult(String outputName, DimensionSelector sele } } + public static class LongSelectColumnSelectorStrategy implements SelectColumnSelectorStrategy + { + + @Override + public void addRowValuesToSelectResult( + String outputName, LongColumnSelector dimSelector, Map resultMap + ) + { + if (dimSelector == null) { + resultMap.put(outputName, null); + } else { + resultMap.put(outputName, dimSelector.get()); + } + } + } + + public static class FloatSelectColumnSelectorStrategy implements SelectColumnSelectorStrategy + { + @Override + public void addRowValuesToSelectResult( + String outputName, FloatColumnSelector dimSelector, Map resultMap + ) + { + if (dimSelector == null) { + resultMap.put(outputName, null); + } else { + resultMap.put(outputName, dimSelector.get()); + } + } + } + public Sequence> process(final SelectQuery query, final Segment segment) { final StorageAdapter adapter = segment.asStorageAdapter(); diff --git a/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java b/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java index 2a669990982d..c8733a20b4b7 100644 --- a/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java @@ -128,6 +128,10 @@ public void cleanup(TopNParams params) private int[] getDimValSelectorForTopNMetric(TopNParams params, TopNResultBuilder resultBuilder) { + if (params.getCardinality() < 0) { + throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality"); + } + int[] dimValSelector = new int[params.getCardinality()]; Arrays.fill(dimValSelector, SKIP_POSITION_VALUE); diff --git a/processing/src/main/java/io/druid/query/topn/BaseTopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/BaseTopNAlgorithm.java index 678cfe688370..0122fefdc440 100644 --- a/processing/src/main/java/io/druid/query/topn/BaseTopNAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/BaseTopNAlgorithm.java @@ -24,6 +24,7 @@ import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.topn.types.TopNColumnSelectorStrategy; import io.druid.segment.Capabilities; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; @@ -73,10 +74,23 @@ public void run( TopNResultBuilder resultBuilder, DimValSelector dimValSelector ) + { + if (params.getCardinality() != TopNColumnSelectorStrategy.CARDINALITY_UNKNOWN) { + runWithCardinalityKnown(params, resultBuilder, dimValSelector); + } else { + runWithCardinalityUnknown(params, resultBuilder); + } + } + + private void runWithCardinalityKnown( + Parameters params, + TopNResultBuilder resultBuilder, + DimValSelector dimValSelector + ) { boolean hasDimValSelector = (dimValSelector != null); - final int cardinality = params.getCardinality(); + int cardinality = params.getCardinality(); int numProcessed = 0; while (numProcessed < cardinality) { final int numToProcess; @@ -105,6 +119,25 @@ public void run( } } + /** + * This function currently handles TopNs on long and float columns, which do not provide cardinality or an ID lookup. + * When cardinality is unknown, process everything in one pass. + * Existing implementations of makeDimValSelector() require cardinality as well, so the DimValSelector is not used. + * @param params TopN parameters from run() + * @param resultBuilder Result builder from run() + */ + private void runWithCardinalityUnknown( + Parameters params, + TopNResultBuilder resultBuilder + ) + { + DimValAggregateStore aggregatesStore = makeDimValAggregateStore(params); + scanAndAggregate(params, null, aggregatesStore, 0); + updateResults(params, null, aggregatesStore, resultBuilder); + closeAggregators(aggregatesStore); + params.getCursor().reset(); + } + protected abstract DimValSelector makeDimValSelector(Parameters params, int numProcessed, int numToProcess); /** diff --git a/processing/src/main/java/io/druid/query/topn/DimExtractionTopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/DimExtractionTopNAlgorithm.java index dc9749758f14..807e4c54cc5a 100644 --- a/processing/src/main/java/io/druid/query/topn/DimExtractionTopNAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/DimExtractionTopNAlgorithm.java @@ -19,7 +19,7 @@ package io.druid.query.topn; -import com.google.common.collect.Maps; +import com.google.common.base.Function; import io.druid.query.ColumnSelectorPlus; import io.druid.query.aggregation.Aggregator; import io.druid.query.topn.types.TopNColumnSelectorStrategy; @@ -61,6 +61,9 @@ public TopNParams makeInitParams( @Override protected Aggregator[][] makeDimValSelector(TopNParams params, int numProcessed, int numToProcess) { + if (params.getCardinality() < 0) { + throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality"); + } ColumnSelectorPlus selectorPlus = params.getSelectorPlus(); return selectorPlus.getColumnSelectorStrategy().getDimExtractionRowSelector(query, params, capabilities); } @@ -74,7 +77,8 @@ protected Aggregator[][] updateDimValSelector(Aggregator[][] aggregators, int nu @Override protected Map makeDimValAggregateStore(TopNParams params) { - return Maps.newHashMap(); + final ColumnSelectorPlus selectorPlus = params.getSelectorPlus(); + return selectorPlus.getColumnSelectorStrategy().makeDimExtractionAggregateStore(); } @Override @@ -88,16 +92,13 @@ public void scanAndAggregate( final Cursor cursor = params.getCursor(); final ColumnSelectorPlus selectorPlus = params.getSelectorPlus(); - while (!cursor.isDone()) { - selectorPlus.getColumnSelectorStrategy().dimExtractionScanAndAggregate( - query, - selectorPlus.getSelector(), - cursor, - rowSelector, - aggregatesStore - ); - cursor.advance(); - } + selectorPlus.getColumnSelectorStrategy().dimExtractionScanAndAggregate( + query, + selectorPlus.getSelector(), + cursor, + rowSelector, + aggregatesStore + ); } @Override @@ -108,21 +109,17 @@ protected void updateResults( TopNResultBuilder resultBuilder ) { - for (Map.Entry entry : aggregatesStore.entrySet()) { - Aggregator[] aggs = entry.getValue(); - if (aggs != null && aggs.length > 0) { - Object[] vals = new Object[aggs.length]; - for (int i = 0; i < aggs.length; i++) { - vals[i] = aggs[i].get(); - } - - resultBuilder.addEntry( - entry.getKey() == null ? null : entry.getKey().toString(), - entry.getKey(), - vals - ); - } - } + final ColumnSelectorPlus selectorPlus = params.getSelectorPlus(); + final boolean needsResultTypeConversion = needsResultTypeConversion(params); + final Function valueTransformer = TopNMapFn.getValueTransformer( + query.getDimensionSpec().getOutputType() + ); + + selectorPlus.getColumnSelectorStrategy().updateDimExtractionResults( + aggregatesStore, + needsResultTypeConversion ? valueTransformer : null, + resultBuilder + ); } @Override @@ -139,4 +136,11 @@ protected void closeAggregators(Map valueMap) public void cleanup(TopNParams params) { } + + private boolean needsResultTypeConversion(TopNParams params) + { + ColumnSelectorPlus selectorPlus = params.getSelectorPlus(); + TopNColumnSelectorStrategy strategy = selectorPlus.getColumnSelectorStrategy(); + return query.getDimensionSpec().getOutputType() != strategy.getValueType(); + } } diff --git a/processing/src/main/java/io/druid/query/topn/DimValHolder.java b/processing/src/main/java/io/druid/query/topn/DimValHolder.java index 2299622cbff1..f983d8960d84 100644 --- a/processing/src/main/java/io/druid/query/topn/DimValHolder.java +++ b/processing/src/main/java/io/druid/query/topn/DimValHolder.java @@ -26,13 +26,13 @@ public class DimValHolder { private final Object topNMetricVal; - private final String dimName; + private final Comparable dimName; private final Object dimValIndex; private final Map metricValues; public DimValHolder( Object topNMetricVal, - String dimName, + Comparable dimName, Object dimValIndex, Map metricValues ) @@ -48,7 +48,7 @@ public Object getTopNMetricVal() return topNMetricVal; } - public String getDimName() + public Comparable getDimName() { return dimName; } @@ -66,7 +66,7 @@ public Map getMetricValues() public static class Builder { private Object topNMetricVal; - private String dimName; + private Comparable dimName; private Object dimValIndex; private Map metricValues; @@ -84,7 +84,7 @@ public Builder withTopNMetricVal(Object topNMetricVal) return this; } - public Builder withDimName(String dimName) + public Builder withDimName(Comparable dimName) { this.dimName = dimName; return this; diff --git a/processing/src/main/java/io/druid/query/topn/DimensionAndMetricValueExtractor.java b/processing/src/main/java/io/druid/query/topn/DimensionAndMetricValueExtractor.java index b53033d70f4c..6ddb09c9d86d 100644 --- a/processing/src/main/java/io/druid/query/topn/DimensionAndMetricValueExtractor.java +++ b/processing/src/main/java/io/druid/query/topn/DimensionAndMetricValueExtractor.java @@ -38,11 +38,6 @@ public DimensionAndMetricValueExtractor(Map value) this.value = value; } - public String getStringDimensionValue(String dimension) - { - return (String) value.get(dimension); - } - public Object getDimensionValue(String dimension) { return value.get(dimension); diff --git a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java index 709d7508a7bf..62737a295fb5 100644 --- a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java @@ -19,6 +19,7 @@ package io.druid.query.topn; +import com.google.common.base.Function; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidPool; import io.druid.java.util.common.Pair; @@ -28,6 +29,7 @@ import io.druid.segment.Capabilities; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; +import io.druid.segment.column.ValueType; import io.druid.segment.data.IndexedInts; import java.nio.ByteBuffer; @@ -190,6 +192,10 @@ protected void scanAndAggregate( final int numProcessed ) { + if (params.getCardinality() < 0) { + throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality"); + } + final ByteBuffer resultsBuf = params.getResultsBuf(); final int numBytesPerRecord = params.getNumBytesPerRecord(); final int[] aggregatorSizes = params.getAggregatorSizes(); @@ -457,10 +463,18 @@ protected void updateResults( TopNResultBuilder resultBuilder ) { + if (params.getCardinality() < 0) { + throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality"); + } + final ByteBuffer resultsBuf = params.getResultsBuf(); final int[] aggregatorSizes = params.getAggregatorSizes(); final DimensionSelector dimSelector = params.getDimSelector(); + final ValueType outType = query.getDimensionSpec().getOutputType(); + final boolean needsResultConversion = outType != ValueType.STRING; + final Function valueTransformer = TopNMapFn.getValueTransformer(outType); + for (int i = 0; i < positions.length; i++) { int position = positions[i]; if (position >= 0) { @@ -470,8 +484,14 @@ protected void updateResults( position += aggregatorSizes[j]; } + Object retVal = dimSelector.lookupName(i); + if (needsResultConversion) { + retVal = valueTransformer.apply(retVal); + } + + resultBuilder.addEntry( - dimSelector.lookupName(i), + (Comparable) retVal, i, vals ); diff --git a/processing/src/main/java/io/druid/query/topn/TimeExtractionTopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/TimeExtractionTopNAlgorithm.java index f4fa5a226777..ba44c593a495 100644 --- a/processing/src/main/java/io/druid/query/topn/TimeExtractionTopNAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/TimeExtractionTopNAlgorithm.java @@ -73,6 +73,10 @@ protected void scanAndAggregate( TopNParams params, int[] dimValSelector, Map aggregatesStore, int numProcessed ) { + if (params.getCardinality() < 0) { + throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality"); + } + final Cursor cursor = params.getCursor(); final DimensionSelector dimSelector = params.getDimSelector(); diff --git a/processing/src/main/java/io/druid/query/topn/TopNBinaryFn.java b/processing/src/main/java/io/druid/query/topn/TopNBinaryFn.java index 545ab0c062d2..d3aaaf2f7275 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNBinaryFn.java +++ b/processing/src/main/java/io/druid/query/topn/TopNBinaryFn.java @@ -84,16 +84,16 @@ public Result apply(Result arg1, Result retVals = new LinkedHashMap<>(); + Map retVals = new LinkedHashMap<>(); TopNResultValue arg1Vals = arg1.getValue(); TopNResultValue arg2Vals = arg2.getValue(); for (DimensionAndMetricValueExtractor arg1Val : arg1Vals) { - retVals.put(arg1Val.getStringDimensionValue(dimension), arg1Val); + retVals.put(arg1Val.getDimensionValue(dimension), arg1Val); } for (DimensionAndMetricValueExtractor arg2Val : arg2Vals) { - final String dimensionValue = arg2Val.getStringDimensionValue(dimension); + final Object dimensionValue = arg2Val.getDimensionValue(dimension); DimensionAndMetricValueExtractor arg1Val = retVals.get(dimensionValue); if (arg1Val != null) { diff --git a/processing/src/main/java/io/druid/query/topn/TopNLexicographicResultBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNLexicographicResultBuilder.java index 5b591bd1ea7b..5a97c486c383 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNLexicographicResultBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNLexicographicResultBuilder.java @@ -32,6 +32,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.PriorityQueue; /** @@ -81,11 +82,12 @@ public int compare( @Override public TopNResultBuilder addEntry( - String dimName, + Comparable dimNameObj, Object dimValIndex, Object[] metricVals ) { + final String dimName = Objects.toString(dimNameObj, null); final Map metricValues = Maps.newHashMapWithExpectedSize(metricVals.length + 1); if (shouldAdd(dimName)) { @@ -131,7 +133,7 @@ public TopNResultBuilder addEntry( public TopNResultBuilder addEntry(DimensionAndMetricValueExtractor dimensionAndMetricValueExtractor) { Object dimensionValueObj = dimensionAndMetricValueExtractor.getDimensionValue(dimSpec.getOutputName()); - String dimensionValue = dimensionValueObj == null ? null : dimensionValueObj.toString(); + String dimensionValue = Objects.toString(dimensionValueObj, null); if (shouldAdd(dimensionValue)) { pQueue.add( diff --git a/processing/src/main/java/io/druid/query/topn/TopNMapFn.java b/processing/src/main/java/io/druid/query/topn/TopNMapFn.java index 0da65fe47cf4..e49dec0a045b 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNMapFn.java +++ b/processing/src/main/java/io/druid/query/topn/TopNMapFn.java @@ -20,15 +20,62 @@ package io.druid.query.topn; import com.google.common.base.Function; +import io.druid.java.util.common.IAE; import io.druid.query.ColumnSelectorPlus; import io.druid.query.Result; -import io.druid.query.topn.types.TopNStrategyFactory; +import io.druid.query.topn.types.TopNColumnSelectorStrategyFactory; import io.druid.segment.Cursor; import io.druid.segment.DimensionHandlerUtils; +import io.druid.segment.column.ValueType; + +import java.util.Objects; public class TopNMapFn implements Function> { - private static final TopNStrategyFactory STRATEGY_FACTORY = new TopNStrategyFactory(); + public static Function getValueTransformer(ValueType outputType) + { + switch (outputType) { + case STRING: + return STRING_TRANSFORMER; + case LONG: + return LONG_TRANSFORMER; + case FLOAT: + return FLOAT_TRANSFORMER; + default: + throw new IAE("invalid type: %s", outputType); + } + } + + private static Function STRING_TRANSFORMER = new Function() + { + @Override + public Object apply(Object input) + { + return Objects.toString(input, null); + } + }; + + private static Function LONG_TRANSFORMER = new Function() + { + @Override + public Object apply(Object input) + { + final Long longVal = DimensionHandlerUtils.convertObjectToLong(input); + return longVal == null ? 0L : longVal; + } + }; + + private static Function FLOAT_TRANSFORMER = new Function() + { + @Override + public Object apply(Object input) + { + final Float floatVal = DimensionHandlerUtils.convertObjectToFloat(input); + return floatVal == null ? 0.0f : floatVal; + } + }; + + private static final TopNColumnSelectorStrategyFactory STRATEGY_FACTORY = new TopNColumnSelectorStrategyFactory(); private final TopNQuery query; private final TopNAlgorithm topNAlgorithm; diff --git a/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java index e6ebaa9b4da0..0fe43daa2577 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java @@ -50,10 +50,10 @@ public class TopNNumericResultBuilder implements TopNResultBuilder private final PriorityQueue pQueue; private final Comparator dimValComparator; private final String[] aggFactoryNames; - private static final Comparator dimNameComparator = new Comparator() + private static final Comparator dimNameComparator = new Comparator() { @Override - public int compare(String o1, String o2) + public int compare(Comparable o1, Comparable o2) { int retval; if (null == o1) { @@ -114,7 +114,7 @@ public int compare(DimValHolder d1, DimValHolder d2) @Override public TopNNumericResultBuilder addEntry( - String dimName, + Comparable dimName, Object dimValIndex, Object[] metricVals ) @@ -196,7 +196,7 @@ public TopNResultBuilder addEntry(DimensionAndMetricValueExtractor dimensionAndM if (shouldAdd(dimValue)) { final DimValHolder valHolder = new DimValHolder.Builder() .withTopNMetricVal(dimValue) - .withDimName(dimensionAndMetricValueExtractor.getStringDimensionValue(dimSpec.getOutputName())) + .withDimName((Comparable) dimensionAndMetricValueExtractor.getDimensionValue(dimSpec.getOutputName())) .withMetricValues(dimensionAndMetricValueExtractor.getBaseObject()) .build(); pQueue.add(valHolder); diff --git a/processing/src/main/java/io/druid/query/topn/TopNParams.java b/processing/src/main/java/io/druid/query/topn/TopNParams.java index c973267fe34c..7024b17130b6 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNParams.java +++ b/processing/src/main/java/io/druid/query/topn/TopNParams.java @@ -43,10 +43,6 @@ protected TopNParams( this.cursor = cursor; this.cardinality = selectorPlus.getColumnSelectorStrategy().getCardinality(selectorPlus.getSelector()); this.numValuesPerPass = numValuesPerPass; - - if (cardinality < 0) { - throw new UnsupportedOperationException("Cannot operate on a dimension without a dictionary"); - } } // Only used by TopN algorithms that support String exclusively 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 d8aef7b54658..81bc3fcad63e 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java @@ -37,6 +37,8 @@ import io.druid.segment.StorageAdapter; import io.druid.segment.VirtualColumns; import io.druid.segment.column.Column; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ValueType; import io.druid.segment.filter.Filters; import org.joda.time.Interval; @@ -104,6 +106,8 @@ private Function> getMapFn(TopNQuery query, fina final TopNAlgorithmSelector selector = new TopNAlgorithmSelector(cardinality, numBytesPerRecord); query.initTopNAlgorithmSelector(selector); + final ColumnCapabilities columnCapabilities = adapter.getColumnCapabilities(dimension); + final TopNAlgorithm topNAlgorithm; if ( selector.isHasExtractionFn() && @@ -117,6 +121,9 @@ private Function> getMapFn(TopNQuery query, fina topNAlgorithm = new TimeExtractionTopNAlgorithm(capabilities, query); } else if (selector.isHasExtractionFn()) { topNAlgorithm = new DimExtractionTopNAlgorithm(capabilities, query); + } else if (columnCapabilities != null && columnCapabilities.getType() != ValueType.STRING) { + // force non-Strings to use DimExtraction for now, do a typed PooledTopN later + topNAlgorithm = new DimExtractionTopNAlgorithm(capabilities, query); } else if (selector.isAggregateAllMetrics()) { topNAlgorithm = new PooledTopNAlgorithm(capabilities, query, bufferPool); } else if (selector.isAggregateTopNMetricFirst() || query.getContextBoolean("doAggregateTopNMetricFirst", false)) { diff --git a/processing/src/main/java/io/druid/query/topn/TopNResultBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNResultBuilder.java index 7c2b64c37f05..b52df6a960a4 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNResultBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNResultBuilder.java @@ -28,7 +28,7 @@ public interface TopNResultBuilder { public TopNResultBuilder addEntry( - String dimName, + Comparable dimNameObj, Object dimValIndex, Object[] metricVals ); diff --git a/processing/src/main/java/io/druid/query/topn/types/FloatTopNColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/topn/types/FloatTopNColumnSelectorStrategy.java new file mode 100644 index 000000000000..1bebff0c2966 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/types/FloatTopNColumnSelectorStrategy.java @@ -0,0 +1,115 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.topn.types; + +import com.google.common.base.Function; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.topn.BaseTopNAlgorithm; +import io.druid.query.topn.TopNParams; +import io.druid.query.topn.TopNQuery; +import io.druid.query.topn.TopNResultBuilder; +import io.druid.segment.Capabilities; +import io.druid.segment.Cursor; +import io.druid.segment.FloatColumnSelector; +import io.druid.segment.column.ValueType; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; + +public class FloatTopNColumnSelectorStrategy + implements TopNColumnSelectorStrategy> +{ + @Override + public int getCardinality(FloatColumnSelector selector) + { + return TopNColumnSelectorStrategy.CARDINALITY_UNKNOWN; + } + + @Override + public ValueType getValueType() + { + return ValueType.FLOAT; + } + + @Override + public Aggregator[][] getDimExtractionRowSelector( + TopNQuery query, TopNParams params, Capabilities capabilities + ) + { + return null; + } + + @Override + public Int2ObjectMap makeDimExtractionAggregateStore() + { + return new Int2ObjectOpenHashMap<>(); + } + + @Override + public void dimExtractionScanAndAggregate( + TopNQuery query, + FloatColumnSelector selector, + Cursor cursor, + Aggregator[][] rowSelector, + Int2ObjectMap aggregatesStore + ) + { + while (!cursor.isDone()) { + int key = Float.floatToIntBits(selector.get()); + Aggregator[] theAggregators = aggregatesStore.get(key); + if (theAggregators == null) { + theAggregators = BaseTopNAlgorithm.makeAggregators(cursor, query.getAggregatorSpecs()); + aggregatesStore.put(key, theAggregators); + } + for (Aggregator aggregator : theAggregators) { + aggregator.aggregate(); + } + cursor.advance(); + } + } + + @Override + public void updateDimExtractionResults( + final Int2ObjectMap aggregatesStore, + final Function valueTransformer, + final TopNResultBuilder resultBuilder + ) + { + for (Int2ObjectMap.Entry entry : aggregatesStore.int2ObjectEntrySet()) { + Aggregator[] aggs = entry.getValue(); + if (aggs != null && aggs.length > 0) { + Object[] vals = new Object[aggs.length]; + for (int i = 0; i < aggs.length; i++) { + vals[i] = aggs[i].get(); + } + + Comparable key = Float.intBitsToFloat(entry.getIntKey()); + if (valueTransformer != null) { + key = (Comparable) valueTransformer.apply(key); + } + + resultBuilder.addEntry( + key, + key, + vals + ); + } + } + } +} diff --git a/processing/src/main/java/io/druid/query/topn/types/LongTopNColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/topn/types/LongTopNColumnSelectorStrategy.java new file mode 100644 index 000000000000..a9dbec9d6cb4 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/types/LongTopNColumnSelectorStrategy.java @@ -0,0 +1,115 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.topn.types; + +import com.google.common.base.Function; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.topn.BaseTopNAlgorithm; +import io.druid.query.topn.TopNParams; +import io.druid.query.topn.TopNQuery; +import io.druid.query.topn.TopNResultBuilder; +import io.druid.segment.Capabilities; +import io.druid.segment.Cursor; +import io.druid.segment.LongColumnSelector; +import io.druid.segment.column.ValueType; +import it.unimi.dsi.fastutil.longs.Long2ObjectMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap; + +public class LongTopNColumnSelectorStrategy + implements TopNColumnSelectorStrategy> +{ + @Override + public int getCardinality(LongColumnSelector selector) + { + return TopNColumnSelectorStrategy.CARDINALITY_UNKNOWN; + } + + @Override + public ValueType getValueType() + { + return ValueType.LONG; + } + + @Override + public Aggregator[][] getDimExtractionRowSelector( + TopNQuery query, TopNParams params, Capabilities capabilities + ) + { + return null; + } + + @Override + public Long2ObjectMap makeDimExtractionAggregateStore() + { + return new Long2ObjectOpenHashMap<>(); + } + + @Override + public void dimExtractionScanAndAggregate( + TopNQuery query, + LongColumnSelector selector, + Cursor cursor, + Aggregator[][] rowSelector, + Long2ObjectMap aggregatesStore + ) + { + while (!cursor.isDone()) { + long key = selector.get(); + Aggregator[] theAggregators = aggregatesStore.get(key); + if (theAggregators == null) { + theAggregators = BaseTopNAlgorithm.makeAggregators(cursor, query.getAggregatorSpecs()); + aggregatesStore.put(key, theAggregators); + } + for (Aggregator aggregator : theAggregators) { + aggregator.aggregate(); + } + cursor.advance(); + } + } + + @Override + public void updateDimExtractionResults( + final Long2ObjectMap aggregatesStore, + final Function valueTransformer, + final TopNResultBuilder resultBuilder + ) + { + for (Long2ObjectMap.Entry entry : aggregatesStore.long2ObjectEntrySet()) { + Aggregator[] aggs = entry.getValue(); + if (aggs != null && aggs.length > 0) { + Object[] vals = new Object[aggs.length]; + for (int i = 0; i < aggs.length; i++) { + vals[i] = aggs[i].get(); + } + + Comparable key = entry.getLongKey(); + if (valueTransformer != null) { + key = (Comparable) valueTransformer.apply(key); + } + + resultBuilder.addEntry( + key, + key, + vals + ); + } + } + } +} diff --git a/processing/src/main/java/io/druid/query/topn/types/StringTopNColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/topn/types/StringTopNColumnSelectorStrategy.java index 47b83b637738..5489ebd417aa 100644 --- a/processing/src/main/java/io/druid/query/topn/types/StringTopNColumnSelectorStrategy.java +++ b/processing/src/main/java/io/druid/query/topn/types/StringTopNColumnSelectorStrategy.java @@ -19,18 +19,23 @@ package io.druid.query.topn.types; +import com.google.common.base.Function; +import com.google.common.collect.Maps; import io.druid.query.aggregation.Aggregator; import io.druid.query.topn.BaseTopNAlgorithm; import io.druid.query.topn.TopNParams; import io.druid.query.topn.TopNQuery; +import io.druid.query.topn.TopNResultBuilder; import io.druid.segment.Capabilities; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; +import io.druid.segment.column.ValueType; import io.druid.segment.data.IndexedInts; import java.util.Map; -public class StringTopNColumnSelectorStrategy implements TopNColumnSelectorStrategy +public class StringTopNColumnSelectorStrategy + implements TopNColumnSelectorStrategy> { @Override public int getCardinality(DimensionSelector selector) @@ -38,14 +43,23 @@ public int getCardinality(DimensionSelector selector) return selector.getValueCardinality(); } + @Override + public ValueType getValueType() + { + return ValueType.STRING; + } + @Override public Aggregator[][] getDimExtractionRowSelector(TopNQuery query, TopNParams params, Capabilities capabilities) { + if (params.getCardinality() < 0) { + throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality"); + } + // This method is used for the DimExtractionTopNAlgorithm only. // Unlike regular topN we cannot rely on ordering to optimize. // Optimization possibly requires a reverse lookup from value to ID, which is // not possible when applying an extraction function - final BaseTopNAlgorithm.AggregatorArrayProvider provider = new BaseTopNAlgorithm.AggregatorArrayProvider( (DimensionSelector) params.getSelectorPlus().getSelector(), query, @@ -56,33 +70,111 @@ public Aggregator[][] getDimExtractionRowSelector(TopNQuery query, TopNParams pa return provider.build(); } + @Override + public Map makeDimExtractionAggregateStore() + { + return Maps.newHashMap(); + } + @Override public void dimExtractionScanAndAggregate( - final TopNQuery query, + TopNQuery query, DimensionSelector selector, Cursor cursor, Aggregator[][] rowSelector, - Map aggregatesStore + Map aggregatesStore ) { - final IndexedInts dimValues = selector.getRow(); + if (selector.getValueCardinality() != DimensionSelector.CARDINALITY_UNKNOWN) { + dimExtractionScanAndAggregateWithCardinalityKnown(query, cursor, selector, rowSelector, aggregatesStore); + } else { + dimExtractionScanAndAggregateWithCardinalityUnknown(query, cursor, selector, aggregatesStore); + } + } + + @Override + public void updateDimExtractionResults( + final Map aggregatesStore, + final Function valueTransformer, + final TopNResultBuilder resultBuilder + ) + { + for (Map.Entry entry : aggregatesStore.entrySet()) { + Aggregator[] aggs = entry.getValue(); + if (aggs != null && aggs.length > 0) { + Object[] vals = new Object[aggs.length]; + for (int i = 0; i < aggs.length; i++) { + vals[i] = aggs[i].get(); + } - for (int i = 0; i < dimValues.size(); ++i) { - final int dimIndex = dimValues.get(i); - Aggregator[] theAggregators = rowSelector[dimIndex]; - if (theAggregators == null) { + Comparable key = entry.getKey(); + if (valueTransformer != null) { + key = (Comparable) valueTransformer.apply(key); + } + + resultBuilder.addEntry( + key, + key, + vals + ); + } + } + } + + private void dimExtractionScanAndAggregateWithCardinalityKnown( + TopNQuery query, + Cursor cursor, + DimensionSelector selector, + Aggregator[][] rowSelector, + Map aggregatesStore + ) + { + while (!cursor.isDone()) { + final IndexedInts dimValues = selector.getRow(); + for (int i = 0; i < dimValues.size(); ++i) { + final int dimIndex = dimValues.get(i); + Aggregator[] theAggregators = rowSelector[dimIndex]; + if (theAggregators == null) { + final String key = selector.lookupName(dimIndex); + theAggregators = aggregatesStore.get(key); + if (theAggregators == null) { + theAggregators = BaseTopNAlgorithm.makeAggregators(cursor, query.getAggregatorSpecs()); + aggregatesStore.put(key, theAggregators); + } + rowSelector[dimIndex] = theAggregators; + } + + for (Aggregator aggregator : theAggregators) { + aggregator.aggregate(); + } + } + cursor.advance(); + } + } + + private void dimExtractionScanAndAggregateWithCardinalityUnknown( + TopNQuery query, + Cursor cursor, + DimensionSelector selector, + Map aggregatesStore + ) + { + while (!cursor.isDone()) { + final IndexedInts dimValues = selector.getRow(); + for (int i = 0; i < dimValues.size(); ++i) { + final int dimIndex = dimValues.get(i); final String key = selector.lookupName(dimIndex); - theAggregators = aggregatesStore.get(key); + + Aggregator[] theAggregators = aggregatesStore.get(key); if (theAggregators == null) { theAggregators = BaseTopNAlgorithm.makeAggregators(cursor, query.getAggregatorSpecs()); aggregatesStore.put(key, theAggregators); } - rowSelector[dimIndex] = theAggregators; - } - - for (Aggregator aggregator : theAggregators) { - aggregator.aggregate(); + for (Aggregator aggregator : theAggregators) { + aggregator.aggregate(); + } } + cursor.advance(); } } } diff --git a/processing/src/main/java/io/druid/query/topn/types/TopNColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/topn/types/TopNColumnSelectorStrategy.java index 01e293bc4ee9..1c022665becc 100644 --- a/processing/src/main/java/io/druid/query/topn/types/TopNColumnSelectorStrategy.java +++ b/processing/src/main/java/io/druid/query/topn/types/TopNColumnSelectorStrategy.java @@ -19,20 +19,30 @@ package io.druid.query.topn.types; +import com.google.common.base.Function; import io.druid.query.aggregation.Aggregator; import io.druid.query.dimension.ColumnSelectorStrategy; import io.druid.query.topn.TopNParams; import io.druid.query.topn.TopNQuery; +import io.druid.query.topn.TopNResultBuilder; import io.druid.segment.Capabilities; import io.druid.segment.ColumnValueSelector; import io.druid.segment.Cursor; +import io.druid.segment.column.ValueType; +import javax.annotation.Nullable; import java.util.Map; -public interface TopNColumnSelectorStrategy extends ColumnSelectorStrategy +public interface TopNColumnSelectorStrategy + + extends ColumnSelectorStrategy { + int CARDINALITY_UNKNOWN = -1; + int getCardinality(ValueSelectorType selector); + ValueType getValueType(); + /** * Used by DimExtractionTopNAlgorithm. * @@ -50,11 +60,20 @@ public interface TopNColumnSelectorStrategy aggregatesStore + DimExtractionAggregateStoreType aggregatesStore + ); + + /** + * Used by DimExtractionTopNAlgorithm. + * + * Read entries from the aggregates store, adding the keys and associated values to the resultBuilder, applying the + * valueTransformer to the keys if present + * + * @param aggregatesStore Map created by makeDimExtractionAggregateStore() + * @param valueTransformer Converts keys to different types, if null no conversion is needed + * @param resultBuilder TopN result builder + */ + void updateDimExtractionResults( + DimExtractionAggregateStoreType aggregatesStore, + @Nullable Function valueTransformer, + TopNResultBuilder resultBuilder ); } diff --git a/processing/src/main/java/io/druid/query/topn/types/TopNStrategyFactory.java b/processing/src/main/java/io/druid/query/topn/types/TopNColumnSelectorStrategyFactory.java similarity index 78% rename from processing/src/main/java/io/druid/query/topn/types/TopNStrategyFactory.java rename to processing/src/main/java/io/druid/query/topn/types/TopNColumnSelectorStrategyFactory.java index 0a141b0aa4fe..5a2948aa564d 100644 --- a/processing/src/main/java/io/druid/query/topn/types/TopNStrategyFactory.java +++ b/processing/src/main/java/io/druid/query/topn/types/TopNColumnSelectorStrategyFactory.java @@ -21,20 +21,25 @@ import io.druid.java.util.common.IAE; import io.druid.query.dimension.ColumnSelectorStrategyFactory; +import io.druid.segment.ColumnValueSelector; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ValueType; -public class TopNStrategyFactory implements ColumnSelectorStrategyFactory +public class TopNColumnSelectorStrategyFactory implements ColumnSelectorStrategyFactory { @Override public TopNColumnSelectorStrategy makeColumnSelectorStrategy( - ColumnCapabilities capabilities + ColumnCapabilities capabilities, ColumnValueSelector selector ) { ValueType type = capabilities.getType(); switch(type) { case STRING: return new StringTopNColumnSelectorStrategy(); + case LONG: + return new LongTopNColumnSelectorStrategy(); + case FLOAT: + return new FloatTopNColumnSelectorStrategy(); default: throw new IAE("Cannot create query type helper from invalid type [%s]", type); } diff --git a/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java b/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java index d9a3b44478fb..4e2087efe2a0 100644 --- a/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java +++ b/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java @@ -20,8 +20,11 @@ package io.druid.segment; import com.google.common.collect.ImmutableList; +import com.google.common.primitives.Floats; +import io.druid.common.guava.GuavaUtils; import io.druid.data.input.impl.DimensionSchema.MultiValueHandling; import io.druid.java.util.common.IAE; +import io.druid.java.util.common.parsers.ParseException; import io.druid.query.ColumnSelectorPlus; import io.druid.query.dimension.ColumnSelectorStrategy; import io.druid.query.dimension.ColumnSelectorStrategyFactory; @@ -30,6 +33,7 @@ import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ValueType; +import java.util.ArrayList; import java.util.List; public final class DimensionHandlerUtils @@ -64,6 +68,15 @@ public static DimensionHandler getHandlerFromCapabilities( return new StringDimensionHandler(dimensionName, multiValueHandling); } + public static List getValueTypesFromDimensionSpecs(List dimSpecs) + { + List types = new ArrayList<>(dimSpecs.size()); + for (DimensionSpec dimSpec : dimSpecs) { + types.add(dimSpec.getOutputType()); + } + return types; + } + /** * Convenience function equivalent to calling * {@link #createColumnSelectorPluses(ColumnSelectorStrategyFactory, List, ColumnSelectorFactory)} with a singleton @@ -113,15 +126,16 @@ public static Colum for (int i = 0; i < dimCount; i++) { final DimensionSpec dimSpec = dimensionSpecs.get(i); final String dimName = dimSpec.getDimension(); - ColumnSelectorStrategyClass strategy = makeStrategy( - strategyFactory, - dimName, - cursor.getColumnCapabilities(dimSpec.getDimension()) - ); final ColumnValueSelector selector = getColumnValueSelectorFromDimensionSpec( dimSpec, cursor ); + ColumnSelectorStrategyClass strategy = makeStrategy( + strategyFactory, + dimSpec, + cursor.getColumnCapabilities(dimSpec.getDimension()), + selector + ); final ColumnSelectorPlus selectorPlus = new ColumnSelectorPlus<>( dimName, dimSpec.getOutputName(), @@ -133,11 +147,31 @@ public static Colum return dims; } + public static ColumnValueSelector getColumnValueSelectorFromDimensionSpec( + DimensionSpec dimSpec, + ColumnSelectorFactory columnSelectorFactory + ) + { + String dimName = dimSpec.getDimension(); + ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(dimName); + capabilities = getEffectiveCapabilities(dimSpec, capabilities); + switch (capabilities.getType()) { + case STRING: + return columnSelectorFactory.makeDimensionSelector(dimSpec); + case LONG: + return columnSelectorFactory.makeLongColumnSelector(dimSpec.getDimension()); + case FLOAT: + return columnSelectorFactory.makeFloatColumnSelector(dimSpec.getDimension()); + default: + return null; + } + } + // When determining the capabilites of a column during query processing, this function // adjusts the capabilities for columns that cannot be handled as-is to manageable defaults // (e.g., treating missing columns as empty String columns) private static ColumnCapabilities getEffectiveCapabilities( - String dimName, + DimensionSpec dimSpec, ColumnCapabilities capabilities ) { @@ -145,37 +179,80 @@ private static ColumnCapabilities getEffectiveCapabilities( capabilities = DEFAULT_STRING_CAPABILITIES; } - // non-Strings aren't actually supported yet - if (capabilities.getType() != ValueType.STRING) { + // Complex dimension type is not supported + if (capabilities.getType() == ValueType.COMPLEX) { + capabilities = DEFAULT_STRING_CAPABILITIES; + } + + // Currently, all extractionFns output Strings, so the column will return String values via a + // DimensionSelector if an extractionFn is present. + if (dimSpec.getExtractionFn() != null) { capabilities = DEFAULT_STRING_CAPABILITIES; } + // DimensionSpec's decorate only operates on DimensionSelectors, so if a spec mustDecorate(), + // we need to wrap selectors on numeric columns with a string casting DimensionSelector. + if (capabilities.getType() == ValueType.LONG || capabilities.getType() == ValueType.FLOAT) { + if (dimSpec.mustDecorate()) { + capabilities = DEFAULT_STRING_CAPABILITIES; + } + } + return capabilities; } - private static ColumnValueSelector getColumnValueSelectorFromDimensionSpec( + private static ColumnSelectorStrategyClass makeStrategy( + ColumnSelectorStrategyFactory strategyFactory, DimensionSpec dimSpec, - ColumnSelectorFactory columnSelectorFactory + ColumnCapabilities capabilities, + ColumnValueSelector selector ) { - String dimName = dimSpec.getDimension(); - ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(dimName); - capabilities = getEffectiveCapabilities(dimName, capabilities); - switch (capabilities.getType()) { - case STRING: - return columnSelectorFactory.makeDimensionSelector(dimSpec); - default: - return null; + capabilities = getEffectiveCapabilities(dimSpec, capabilities); + return strategyFactory.makeColumnSelectorStrategy(capabilities, selector); + } + + public static Long convertObjectToLong(Object valObj) + { + if (valObj == null) { + return 0L; + } + + if (valObj instanceof Long) { + return (Long) valObj; + } else if (valObj instanceof Number) { + return ((Number) valObj).longValue(); + } else if (valObj instanceof String) { + try { + return GuavaUtils.tryParseLong((String) valObj); + } + catch (Exception e) { + throw new ParseException(e, "Unable to parse value[%s] as long", valObj); + } + } else { + throw new ParseException("Unknown type[%s]", valObj.getClass()); } } - private static ColumnSelectorStrategyClass makeStrategy( - ColumnSelectorStrategyFactory strategyFactory, - String dimName, - ColumnCapabilities capabilities - ) + public static Float convertObjectToFloat(Object valObj) { - capabilities = getEffectiveCapabilities(dimName, capabilities); - return strategyFactory.makeColumnSelectorStrategy(capabilities); + if (valObj == null) { + return 0.0f; + } + + if (valObj instanceof Float) { + return (Float) valObj; + } else if (valObj instanceof Number) { + return ((Number) valObj).floatValue(); + } else if (valObj instanceof String) { + try { + return Floats.tryParse((String) valObj); + } + catch (Exception e) { + throw new ParseException(e, "Unable to parse value[%s] as float", valObj); + } + } else { + throw new ParseException("Unknown type[%s]", valObj.getClass()); + } } } diff --git a/processing/src/main/java/io/druid/segment/FloatWrappingDimensionSelector.java b/processing/src/main/java/io/druid/segment/FloatWrappingDimensionSelector.java new file mode 100644 index 000000000000..259744bb9acc --- /dev/null +++ b/processing/src/main/java/io/druid/segment/FloatWrappingDimensionSelector.java @@ -0,0 +1,45 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment; + +import io.druid.query.extraction.ExtractionFn; +import io.druid.segment.virtual.BaseSingleValueDimensionSelector; + +public class FloatWrappingDimensionSelector extends BaseSingleValueDimensionSelector +{ + private final FloatColumnSelector selector; + private final ExtractionFn extractionFn; + + public FloatWrappingDimensionSelector(FloatColumnSelector selector, ExtractionFn extractionFn) + { + this.selector = selector; + this.extractionFn = extractionFn; + } + + @Override + protected String getValue() + { + if (extractionFn == null) { + return String.valueOf(selector.get()); + } else { + return extractionFn.apply(selector.get()); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/LongWrappingDimensionSelector.java b/processing/src/main/java/io/druid/segment/LongWrappingDimensionSelector.java new file mode 100644 index 000000000000..4783a8eb3ca5 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/LongWrappingDimensionSelector.java @@ -0,0 +1,45 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment; + +import io.druid.query.extraction.ExtractionFn; +import io.druid.segment.virtual.BaseSingleValueDimensionSelector; + +public class LongWrappingDimensionSelector extends BaseSingleValueDimensionSelector +{ + private final LongColumnSelector selector; + private final ExtractionFn extractionFn; + + public LongWrappingDimensionSelector(LongColumnSelector selector, ExtractionFn extractionFn) + { + this.selector = selector; + this.extractionFn = extractionFn; + } + + @Override + protected String getValue() + { + if (extractionFn == null) { + return String.valueOf(selector.get()); + } else { + return extractionFn.apply(selector.get()); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index 6e778b7aa399..dadf2539b5c4 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -454,6 +454,14 @@ private DimensionSelector makeDimensionSelectorUndecorated( ); } + if (columnDesc.getCapabilities().getType() == ValueType.LONG) { + return new LongWrappingDimensionSelector(makeLongColumnSelector(dimension), extractionFn); + } + + if (columnDesc.getCapabilities().getType() == ValueType.FLOAT) { + return new FloatWrappingDimensionSelector(makeFloatColumnSelector(dimension), extractionFn); + } + DictionaryEncodedColumn cachedColumn = dictionaryColumnCache.get(dimension); if (cachedColumn == null) { cachedColumn = columnDesc.getDictionaryEncoding(); diff --git a/processing/src/main/java/io/druid/segment/column/ValueType.java b/processing/src/main/java/io/druid/segment/column/ValueType.java index aa9919a129b9..171517c2f902 100644 --- a/processing/src/main/java/io/druid/segment/column/ValueType.java +++ b/processing/src/main/java/io/druid/segment/column/ValueType.java @@ -19,6 +19,8 @@ package io.druid.segment.column; +import com.fasterxml.jackson.annotation.JsonCreator; + /** */ public enum ValueType @@ -26,5 +28,14 @@ public enum ValueType FLOAT, LONG, STRING, - COMPLEX + COMPLEX; + + @JsonCreator + public static ValueType fromString(String name) + { + if (name == null) { + return null; + } + return valueOf(name.toUpperCase()); + } } diff --git a/processing/src/main/java/io/druid/segment/filter/BoundFilter.java b/processing/src/main/java/io/druid/segment/filter/BoundFilter.java index aeb735de7239..3baeb9d15b65 100644 --- a/processing/src/main/java/io/druid/segment/filter/BoundFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/BoundFilter.java @@ -26,6 +26,7 @@ import io.druid.query.extraction.ExtractionFn; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.BoundDimFilter; +import io.druid.query.filter.DruidFloatPredicate; import io.druid.query.filter.DruidLongPredicate; import io.druid.query.filter.DruidPredicateFactory; import io.druid.query.filter.Filter; @@ -46,6 +47,7 @@ public class BoundFilter implements Filter private final ExtractionFn extractionFn; private final Supplier longPredicateSupplier; + private final Supplier floatPredicateSupplier; public BoundFilter(final BoundDimFilter boundDimFilter) { @@ -53,6 +55,7 @@ public BoundFilter(final BoundDimFilter boundDimFilter) this.comparator = boundDimFilter.getOrdering(); this.extractionFn = boundDimFilter.getExtractionFn(); this.longPredicateSupplier = boundDimFilter.getLongPredicateSupplier(); + this.floatPredicateSupplier = boundDimFilter.getFloatPredicateSupplier(); } @Override @@ -234,6 +237,32 @@ public boolean applyLong(long input) }; } } + + @Override + public DruidFloatPredicate makeFloatPredicate() + { + if (extractionFn != null) { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + return doesMatch(extractionFn.apply(input)); + } + }; + } else if (boundDimFilter.getOrdering().equals(StringComparators.NUMERIC)) { + return floatPredicateSupplier.get(); + } else { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + return doesMatch(String.valueOf(input)); + } + }; + } + } }; } diff --git a/processing/src/main/java/io/druid/segment/filter/DimensionPredicateFilter.java b/processing/src/main/java/io/druid/segment/filter/DimensionPredicateFilter.java index 0155f9f4cb9e..c4a6f7f4ac49 100644 --- a/processing/src/main/java/io/druid/segment/filter/DimensionPredicateFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/DimensionPredicateFilter.java @@ -24,6 +24,7 @@ import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.query.extraction.ExtractionFn; import io.druid.query.filter.BitmapIndexSelector; +import io.druid.query.filter.DruidFloatPredicate; import io.druid.query.filter.DruidLongPredicate; import io.druid.query.filter.DruidPredicateFactory; import io.druid.query.filter.Filter; @@ -83,6 +84,19 @@ public boolean applyLong(long input) } }; } + + @Override + public DruidFloatPredicate makeFloatPredicate() + { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + return baseStringPredicate.apply(extractionFn.apply(input)); + } + }; + } }; } } diff --git a/processing/src/main/java/io/druid/segment/filter/Filters.java b/processing/src/main/java/io/druid/segment/filter/Filters.java index f4b1ccdabb63..c64c8e593fc4 100644 --- a/processing/src/main/java/io/druid/segment/filter/Filters.java +++ b/processing/src/main/java/io/druid/segment/filter/Filters.java @@ -65,7 +65,11 @@ */ public class Filters { - public static final List FILTERABLE_TYPES = ImmutableList.of(ValueType.STRING, ValueType.LONG); + public static final List FILTERABLE_TYPES = ImmutableList.of( + ValueType.STRING, + ValueType.LONG, + ValueType.FLOAT + ); private static final String CTX_KEY_USE_FILTER_CNF = "useFilterCNF"; /** @@ -122,16 +126,6 @@ public static ValueMatcher makeValueMatcher( final String value ) { - final ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(columnName); - - // This should be folded into the ValueMatcherColumnSelectorStrategy once that can handle LONG typed columns. - if (capabilities != null && capabilities.getType() == ValueType.LONG) { - return getLongValueMatcher( - columnSelectorFactory.makeLongColumnSelector(columnName), - value - ); - } - final ColumnSelectorPlus selector = DimensionHandlerUtils.createColumnSelectorPlus( ValueMatcherColumnSelectorStrategyFactory.instance(), diff --git a/processing/src/main/java/io/druid/segment/filter/InFilter.java b/processing/src/main/java/io/druid/segment/filter/InFilter.java index c9a7ce902d1b..d03171f3f906 100644 --- a/processing/src/main/java/io/druid/segment/filter/InFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/InFilter.java @@ -25,6 +25,7 @@ import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.query.extraction.ExtractionFn; import io.druid.query.filter.BitmapIndexSelector; +import io.druid.query.filter.DruidFloatPredicate; import io.druid.query.filter.DruidLongPredicate; import io.druid.query.filter.DruidPredicateFactory; import io.druid.query.filter.Filter; @@ -48,11 +49,13 @@ public class InFilter implements Filter private final Set values; private final ExtractionFn extractionFn; private final Supplier longPredicateSupplier; + private final Supplier floatPredicateSupplier; public InFilter( String dimension, Set values, Supplier longPredicateSupplier, + Supplier floatPredicateSupplier, ExtractionFn extractionFn ) { @@ -60,6 +63,7 @@ public InFilter( this.values = values; this.extractionFn = extractionFn; this.longPredicateSupplier = longPredicateSupplier; + this.floatPredicateSupplier = floatPredicateSupplier; } @Override @@ -192,6 +196,23 @@ public boolean applyLong(long input) return longPredicateSupplier.get(); } } + + @Override + public DruidFloatPredicate makeFloatPredicate() + { + if (extractionFn != null) { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + return values.contains(extractionFn.apply(input)); + } + }; + } else { + return floatPredicateSupplier.get(); + } + } }; } } diff --git a/processing/src/main/java/io/druid/segment/filter/RegexFilter.java b/processing/src/main/java/io/druid/segment/filter/RegexFilter.java index 0c7eb7e53846..8cda35aa1f47 100644 --- a/processing/src/main/java/io/druid/segment/filter/RegexFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/RegexFilter.java @@ -21,6 +21,7 @@ import com.google.common.base.Predicate; import io.druid.query.extraction.ExtractionFn; +import io.druid.query.filter.DruidFloatPredicate; import io.druid.query.filter.DruidLongPredicate; import io.druid.query.filter.DruidPredicateFactory; @@ -66,6 +67,19 @@ public boolean applyLong(long input) }; } + @Override + public DruidFloatPredicate makeFloatPredicate() + { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + return pattern.matcher(String.valueOf(input)).find(); + } + }; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/segment/filter/SearchQueryFilter.java b/processing/src/main/java/io/druid/segment/filter/SearchQueryFilter.java index 215e5cd59624..650e5d26fa15 100644 --- a/processing/src/main/java/io/druid/segment/filter/SearchQueryFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/SearchQueryFilter.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Predicate; import io.druid.query.extraction.ExtractionFn; +import io.druid.query.filter.DruidFloatPredicate; import io.druid.query.filter.DruidLongPredicate; import io.druid.query.filter.DruidPredicateFactory; import io.druid.query.search.search.SearchQuerySpec; @@ -69,6 +70,19 @@ public boolean applyLong(long input) } }; } + + @Override + public DruidFloatPredicate makeFloatPredicate() + { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + return query.accept(String.valueOf(input)); + } + }; + } }, extractionFn ); diff --git a/processing/src/main/java/io/druid/segment/filter/SpatialFilter.java b/processing/src/main/java/io/druid/segment/filter/SpatialFilter.java index a17831fa297f..ba11a264be54 100644 --- a/processing/src/main/java/io/druid/segment/filter/SpatialFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/SpatialFilter.java @@ -23,6 +23,7 @@ import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.spatial.search.Bound; import io.druid.query.filter.BitmapIndexSelector; +import io.druid.query.filter.DruidFloatPredicate; import io.druid.query.filter.DruidLongPredicate; import io.druid.query.filter.DruidPredicateFactory; import io.druid.query.filter.Filter; @@ -82,15 +83,15 @@ public boolean apply(String input) @Override public DruidLongPredicate makeLongPredicate() { - return new DruidLongPredicate() - { - @Override - public boolean applyLong(long input) - { - // SpatialFilter does not currently support longs - return false; - } - }; + // SpatialFilter does not currently support longs + return DruidLongPredicate.ALWAYS_FALSE; + } + + @Override + public DruidFloatPredicate makeFloatPredicate() + { + // SpatialFilter does not currently support floats + return DruidFloatPredicate.ALWAYS_FALSE; } } ); 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 b553774aa62b..503ba9f12fdd 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -38,7 +38,9 @@ import io.druid.segment.DimensionIndexer; import io.druid.segment.DimensionSelector; import io.druid.segment.FloatColumnSelector; +import io.druid.segment.FloatWrappingDimensionSelector; import io.druid.segment.LongColumnSelector; +import io.druid.segment.LongWrappingDimensionSelector; import io.druid.segment.Metadata; import io.druid.segment.NullDimensionSelector; import io.druid.segment.ObjectColumnSelector; @@ -49,6 +51,7 @@ import io.druid.segment.ZeroLongColumnSelector; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ValueType; import io.druid.segment.data.Indexed; import io.druid.segment.data.ListIndexed; import io.druid.segment.filter.BooleanValueMatcher; @@ -341,6 +344,13 @@ public DimensionSelector makeDimensionSelector( return virtualColumns.makeDimensionSelector(dimensionSpec, this); } + return dimensionSpec.decorate(makeDimensionSelectorUndecorated(dimensionSpec)); + } + + private DimensionSelector makeDimensionSelectorUndecorated( + DimensionSpec dimensionSpec + ) + { final String dimension = dimensionSpec.getDimension(); final ExtractionFn extractionFn = dimensionSpec.getExtractionFn(); @@ -350,16 +360,26 @@ public DimensionSelector makeDimensionSelector( extractionFn, descending ); - return dimensionSpec.decorate(selector); + return selector; + } + + ColumnCapabilities capabilities = getColumnCapabilities(dimension); + if (capabilities != null) { + if (capabilities.getType() == ValueType.LONG) { + return new LongWrappingDimensionSelector(makeLongColumnSelector(dimension), extractionFn); + } + if (capabilities.getType() == ValueType.FLOAT) { + return new FloatWrappingDimensionSelector(makeFloatColumnSelector(dimension), extractionFn); + } } final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(dimensionSpec.getDimension()); if (dimensionDesc == null) { - return dimensionSpec.decorate(NullDimensionSelector.instance()); + return NullDimensionSelector.instance(); } final DimensionIndexer indexer = dimensionDesc.getIndexer(); - return dimensionSpec.decorate((DimensionSelector) indexer.makeColumnValueSelector(dimensionSpec, currEntry, dimensionDesc)); + return (DimensionSelector) indexer.makeColumnValueSelector(dimensionSpec, currEntry, dimensionDesc); } @Override diff --git a/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java b/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java index 8aa5997c109a..69a06f6ce37f 100644 --- a/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java +++ b/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java @@ -349,7 +349,7 @@ public void testNumericEvolutionFiltering() // Only float(3) -- which we can't filter, but can aggregate Assert.assertEquals( - timeseriesResult(ImmutableMap.of("a", 0L, "b", 0.0, "c", 0L)), + timeseriesResult(ImmutableMap.of("a", 19L, "b", 19.100000381469727, "c", 2L)), runQuery(query, factory, ImmutableList.of(index3)) ); @@ -362,9 +362,9 @@ public void testNumericEvolutionFiltering() // string(1) + long(2) + float(3) + nonexistent(4) Assert.assertEquals( timeseriesResult(ImmutableMap.of( - "a", 19L, - "b", 19.0, - "c", 4L + "a", 38L, + "b", 38.10000038146973, + "c", 6L )), runQuery(query, factory, ImmutableList.of(index1, index2, index3, index4)) ); diff --git a/processing/src/test/java/io/druid/query/dimension/DefaultDimensionSpecTest.java b/processing/src/test/java/io/druid/query/dimension/DefaultDimensionSpecTest.java index a1cad5f6aff1..7d612be2d720 100644 --- a/processing/src/test/java/io/druid/query/dimension/DefaultDimensionSpecTest.java +++ b/processing/src/test/java/io/druid/query/dimension/DefaultDimensionSpecTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import io.druid.jackson.DefaultObjectMapper; +import io.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -42,4 +43,16 @@ public void testEqualsSerde() throws IOException Assert.assertEquals(spec, other); Assert.assertEquals(spec.hashCode(), other.hashCode()); } + + @Test + public void testEqualsSerdeWithType() throws IOException + { + final String name = "foo"; + final DimensionSpec spec = new DefaultDimensionSpec(name, name, ValueType.FLOAT); + final String json = mapper.writeValueAsString(spec); + final DimensionSpec other = mapper.readValue(json, DimensionSpec.class); + Assert.assertEquals(spec.toString(), other.toString()); + Assert.assertEquals(spec, other); + Assert.assertEquals(spec.hashCode(), other.hashCode()); + } } diff --git a/processing/src/test/java/io/druid/query/dimension/ExtractionDimensionSpecTest.java b/processing/src/test/java/io/druid/query/dimension/ExtractionDimensionSpecTest.java index 88845a71907b..9e703df4c754 100644 --- a/processing/src/test/java/io/druid/query/dimension/ExtractionDimensionSpecTest.java +++ b/processing/src/test/java/io/druid/query/dimension/ExtractionDimensionSpecTest.java @@ -23,6 +23,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.query.extraction.MatchingDimExtractionFn; import io.druid.query.extraction.RegexDimExtractionFn; +import io.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -48,6 +49,40 @@ public void testSerde() throws Exception Assert.assertEquals("first3Letters", extractionDimensionSpec.getOutputName()); Assert.assertEquals("myDim", extractionDimensionSpec.getDimension()); Assert.assertNotNull(extractionDimensionSpec.getExtractionFn()); + Assert.assertEquals(ValueType.STRING, extractionDimensionSpec.getOutputType()); + Assert.assertTrue(extractionDimensionSpec.getExtractionFn() instanceof RegexDimExtractionFn); + + Assert.assertEquals( + extractionDimensionSpec, + objectMapper.readValue( + objectMapper.writeValueAsBytes(extractionDimensionSpec), + DimensionSpec.class + ) + ); + } + + @Test + public void testSerdeWithType() throws Exception + { + final ObjectMapper objectMapper = new DefaultObjectMapper(); + + final String oldJson = "{\n" + + " \"type\": \"extraction\",\n" + + " \"outputName\": \"first3Letters\",\n" + + " \"outputType\": \"LONG\",\n" + + " \"dimension\": \"myDim\"," + + " \"extractionFn\": {\n" + + " \"type\": \"regex\",\n" + + " \"expr\": \"(...).*\"\n" + + " }\n" + + "}"; + + final ExtractionDimensionSpec extractionDimensionSpec = (ExtractionDimensionSpec) objectMapper.readValue(oldJson, DimensionSpec.class); + + Assert.assertEquals("first3Letters", extractionDimensionSpec.getOutputName()); + Assert.assertEquals("myDim", extractionDimensionSpec.getDimension()); + Assert.assertNotNull(extractionDimensionSpec.getExtractionFn()); + Assert.assertEquals(ValueType.LONG, extractionDimensionSpec.getOutputType()); Assert.assertTrue(extractionDimensionSpec.getExtractionFn() instanceof RegexDimExtractionFn); Assert.assertEquals( 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 12c4c0917465..c8a26d75bb98 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.smile.SmileFactory; +import com.google.common.base.Functions; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; @@ -29,6 +30,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; import com.google.common.util.concurrent.MoreExecutors; import io.druid.collections.BlockingPool; import io.druid.collections.StupidPool; @@ -60,6 +62,7 @@ import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.FilteredAggregatorFactory; import io.druid.query.aggregation.JavaScriptAggregatorFactory; +import io.druid.query.aggregation.LongMaxAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory; @@ -74,12 +77,15 @@ import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; import io.druid.query.dimension.ExtractionDimensionSpec; +import io.druid.query.dimension.ListFilteredDimensionSpec; +import io.druid.query.dimension.RegexFilteredDimensionSpec; import io.druid.query.extraction.CascadeExtractionFn; import io.druid.query.extraction.DimExtractionFn; import io.druid.query.extraction.ExtractionFn; import io.druid.query.extraction.JavaScriptExtractionFn; import io.druid.query.extraction.MapLookupExtractor; import io.druid.query.extraction.RegexDimExtractionFn; +import io.druid.query.extraction.StrlenExtractionFn; import io.druid.query.extraction.TimeFormatExtractionFn; import io.druid.query.filter.AndDimFilter; import io.druid.query.filter.BoundDimFilter; @@ -110,6 +116,7 @@ import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.segment.TestHelper; import io.druid.segment.column.Column; +import io.druid.segment.column.ValueType; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Interval; @@ -1264,8 +1271,7 @@ public void testGroupByWithRebucketRename() new ExtractionDimensionSpec( "quality", "alias", - new LookupExtractionFn(new MapLookupExtractor(map, false), false, null, false, false), - null + new LookupExtractionFn(new MapLookupExtractor(map, false), false, null, false, false) ) ) ) @@ -1343,8 +1349,7 @@ public void testGroupByWithSimpleRenameRetainMissingNonInjective() new ExtractionDimensionSpec( "quality", "alias", - new LookupExtractionFn(new MapLookupExtractor(map, false), true, null, false, false), - null + new LookupExtractionFn(new MapLookupExtractor(map, false), true, null, false, false) ) ) ) @@ -1422,8 +1427,7 @@ public void testGroupByWithSimpleRenameRetainMissing() new ExtractionDimensionSpec( "quality", "alias", - new LookupExtractionFn(new MapLookupExtractor(map, false), true, null, true, false), - null + new LookupExtractionFn(new MapLookupExtractor(map, false), true, null, true, false) ) ) ) @@ -1501,8 +1505,7 @@ public void testGroupByWithSimpleRenameAndMissingString() new ExtractionDimensionSpec( "quality", "alias", - new LookupExtractionFn(new MapLookupExtractor(map, false), false, "MISSING", true, false), - null + new LookupExtractionFn(new MapLookupExtractor(map, false), false, "MISSING", true, false) ) ) ) @@ -1579,8 +1582,7 @@ public void testGroupByWithSimpleRename() new ExtractionDimensionSpec( "quality", "alias", - new LookupExtractionFn(new MapLookupExtractor(map, false), false, null, true, false), - null + new LookupExtractionFn(new MapLookupExtractor(map, false), false, null, true, false) ) ) ) @@ -1824,7 +1826,7 @@ public String apply(String dimValue) .setGranularity(QueryRunnerTestHelper.dayGran) .setDimensions( Lists.newArrayList( - new ExtractionDimensionSpec("quality", "alias", nullExtractionFn, null) + new ExtractionDimensionSpec("quality", "alias", nullExtractionFn) ) ) .build(); @@ -1893,7 +1895,7 @@ public String apply(String dimValue) .setGranularity(QueryRunnerTestHelper.dayGran) .setDimensions( Lists.newArrayList( - new ExtractionDimensionSpec("quality", "alias", emptyStringExtractionFn, null) + new ExtractionDimensionSpec("quality", "alias", emptyStringExtractionFn) ) ) .build(); @@ -3050,8 +3052,7 @@ public void testGroupByWithAlphaNumericDimensionOrder() new ExtractionDimensionSpec( "quality", "alias", - new LookupExtractionFn(new MapLookupExtractor(map, false), false, null, false, false), - null + new LookupExtractionFn(new MapLookupExtractor(map, false), false, null, false, false) ) ) ) @@ -3720,65 +3721,6 @@ public void testGroupByWithRegEx() throws Exception TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); } - @Test - public void testGroupByWithMetricColumnDisappears() throws Exception - { - GroupByQuery.Builder builder = GroupByQuery - .builder() - .setDataSource(QueryRunnerTestHelper.dataSource) - .setInterval("2011-04-02/2011-04-04") - .addDimension("quality") - .addDimension("index") - .setAggregatorSpecs( - Arrays.asList( - QueryRunnerTestHelper.rowsCount - ) - ) - .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)); - - final GroupByQuery query = builder.build(); - - List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "index", - null, - "quality", - "automotive", - "rows", - 2L - ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "index", null, "quality", "business", "rows", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "index", - null, - "quality", - "entertainment", - "rows", - 2L - ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "index", null, "quality", "health", "rows", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "index", null, "quality", "mezzanine", "rows", 6L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "index", null, "quality", "news", "rows", 2L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "index", null, "quality", "premium", "rows", 6L), - GroupByQueryRunnerTestHelper.createExpectedRow( - "2011-04-01", - "index", - null, - "quality", - "technology", - "rows", - 2L - ), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "index", null, "quality", "travel", "rows", 2L) - ); - - Map context = Maps.newHashMap(); - QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); - } - @Test public void testGroupByWithNonexistentDimension() throws Exception { @@ -4264,8 +4206,7 @@ public void testGroupByTimeExtractionNamedUnderUnderTime() new ExtractionDimensionSpec( Column.TIME_COLUMN_NAME, Column.TIME_COLUMN_NAME, - new TimeFormatExtractionFn("EEEE", null, null, null, false), - null + new TimeFormatExtractionFn("EEEE", null, null, null, false) ) ) ) @@ -5703,8 +5644,7 @@ public void testGroupByTimeExtraction() new ExtractionDimensionSpec( Column.TIME_COLUMN_NAME, "dayOfWeek", - new TimeFormatExtractionFn("EEEE", null, null, null, false), - null + new TimeFormatExtractionFn("EEEE", null, null, null, false) ) ) ) @@ -5965,8 +5905,7 @@ public ExtractionType getExtractionType() new TimeFormatExtractionFn("EEEE", null, null, null, false), nullWednesdays, } - ), - null + ) ) ) ) @@ -6277,8 +6216,7 @@ public void testBySegmentResultsUnOptimizedDimextraction() false ), false, null, false, false - ), - null + ) ) ) ) @@ -6352,8 +6290,7 @@ public void testBySegmentResultsOptimizedDimextraction() false ), false, null, true, false - ), - null + ) ) ) ) @@ -6930,15 +6867,995 @@ public void testGroupByCardinalityAggOnFloat() .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - // CardinalityAggregator currently treats non-String columns as having all nulls, so cardinality is 1 for - // the 'index' column List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "spot", "rows", 9L, "numVals", 1.0002442201269182d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "total_market", "rows", 2L, "numVals", 1.0002442201269182d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "upfront", "rows", 2L, "numVals", 1.0002442201269182d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "spot", "rows", 9L, "numVals", 1.0002442201269182d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "total_market", "rows", 2L, "numVals", 1.0002442201269182d), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "upfront", "rows", 2L, "numVals", 1.0002442201269182d) + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "spot", "rows", 9L, "numVals", 9.019833517963864d), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "total_market", "rows", 2L, "numVals", 2.000977198748901d), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "upfront", "rows", 2L, "numVals", 2.000977198748901d), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "spot", "rows", 9L, "numVals", 9.019833517963864d), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "total_market", "rows", 2L, "numVals", 2.000977198748901d), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "upfront", "rows", 2L, "numVals", 2.000977198748901d) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByLongColumn() + { + if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); + } + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG))) + .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .addOrderByColumn(new OrderByColumnSpec( + "ql_alias", + OrderByColumnSpec.Direction.ASCENDING, + StringComparators.NUMERIC + )) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + Assert.assertEquals( + Functions.>identity(), + query.getLimitSpec().build( + query.getDimensions(), + query.getAggregatorSpecs(), + query.getPostAggregatorSpecs() + ) + ); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "ql_alias", + 1200L, + "rows", + 1L, + "idx", + 158L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "ql_alias", + 1200L, + "rows", + 1L, + "idx", + 166L + ) + ); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByLongColumnDescending() + { + if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); + } + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG))) + .setDimFilter(new InDimFilter("quality", Arrays.asList("entertainment", "technology"), null)) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .addOrderByColumn(new OrderByColumnSpec( + "ql_alias", + OrderByColumnSpec.Direction.DESCENDING, + StringComparators.NUMERIC + )) + .setGranularity(QueryRunnerTestHelper.allGran) + .build(); + + Assert.assertNotEquals( + Functions.>identity(), + query.getLimitSpec().build( + query.getDimensions(), + query.getAggregatorSpecs(), + query.getPostAggregatorSpecs() + ) + ); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "ql_alias", + 1700L, + "rows", + 2L, + "idx", + 175L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "ql_alias", + 1200L, + "rows", + 2L, + "idx", + 324L + ) + ); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByLongColumnWithExFn() + { + if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); + } + + String jsFn = "function(str) { return 'super-' + str; }"; + ExtractionFn jsExtractionFn = new JavaScriptExtractionFn(jsFn, false, JavaScriptConfig.getDefault()); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new ExtractionDimensionSpec("qualityLong", "ql_alias", jsExtractionFn))) + .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "ql_alias", + "super-1200", + "rows", + 1L, + "idx", + 158L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "ql_alias", + "super-1200", + "rows", + 1L, + "idx", + 166L + ) + ); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByLongTimeColumn() + { + if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); + } + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("__time", "time_alias", ValueType.LONG))) + .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "time_alias", + 1301616000000L, + "rows", + 1L, + "idx", + 158L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "time_alias", + 1301702400000L, + "rows", + 1L, + "idx", + 166L + ) + ); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByLongTimeColumnWithExFn() + { + String jsFn = "function(str) { return 'super-' + str; }"; + ExtractionFn jsExtractionFn = new JavaScriptExtractionFn(jsFn, false, JavaScriptConfig.getDefault()); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new ExtractionDimensionSpec("__time", "time_alias", jsExtractionFn))) + .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "time_alias", + "super-1301616000000", + "rows", + 1L, + "idx", + 158L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "time_alias", + "super-1301702400000", + "rows", + 1L, + "idx", + 166L + ) + ); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByFloatColumn() + { + if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); + } + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("index", "index_alias", ValueType.FLOAT))) + .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .addOrderByColumn(new OrderByColumnSpec( + "index_alias", + OrderByColumnSpec.Direction.ASCENDING, + StringComparators.NUMERIC + )) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + Assert.assertEquals( + Functions.>identity(), + query.getLimitSpec().build( + query.getDimensions(), + query.getAggregatorSpecs(), + query.getPostAggregatorSpecs() + ) + ); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "index_alias", + 158.747224f, + "rows", + 1L, + "idx", + 158L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "index_alias", + 166.016049f, + "rows", + 1L, + "idx", + 166L + ) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByFloatColumnDescending() + { + if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); + } + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("qualityFloat", "qf_alias", ValueType.FLOAT))) + .setDimFilter(new InDimFilter("quality", Arrays.asList("entertainment", "technology"), null)) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .addOrderByColumn(new OrderByColumnSpec( + "qf_alias", + OrderByColumnSpec.Direction.DESCENDING, + StringComparators.NUMERIC + )) + .setGranularity(QueryRunnerTestHelper.allGran) + .build(); + + Assert.assertNotEquals( + Functions.>identity(), + query.getLimitSpec().build( + query.getDimensions(), + query.getAggregatorSpecs(), + query.getPostAggregatorSpecs() + ) + ); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "qf_alias", + 17000.0f, + "rows", + 2L, + "idx", + 175L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "qf_alias", + 12000.0f, + "rows", + 2L, + "idx", + 324L + ) + ); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByFloatColumnWithExFn() + { + if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); + } + + String jsFn = "function(str) { return 'super-' + str; }"; + ExtractionFn jsExtractionFn = new JavaScriptExtractionFn(jsFn, false, JavaScriptConfig.getDefault()); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new ExtractionDimensionSpec("index", "index_alias", jsExtractionFn))) + .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults; + + expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "index_alias", + "super-158.74722290039062", + "rows", + 1L, + "idx", + 158L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "index_alias", + "super-166.01605224609375", + "rows", + 1L, + "idx", + 166L + ) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByWithHavingSpecOnLongAndFloat() + { + if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); + } + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions( + Lists.newArrayList( + new DefaultDimensionSpec("market", "alias"), + new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG), + new DefaultDimensionSpec("__time", "time_alias", ValueType.LONG), + new DefaultDimensionSpec("index", "index_alias", ValueType.FLOAT) + ) + ) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount + ) + ) + .setHavingSpec( + new DimFilterHavingSpec( + new AndDimFilter( + Lists.newArrayList( + new SelectorDimFilter("ql_alias", "1400", null), + new SelectorDimFilter("time_alias", "1301616000000", null), + new BoundDimFilter( + "index_alias", + "1310.0", + "1320.0", + true, + true, + null, + null, + StringComparators.NUMERIC + ) + ) + ) + ) + ) + .setGranularity(QueryRunnerTestHelper.allGran) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", "total_market", + "time_alias", 1301616000000L, + "index_alias", 1314.8397, + "ql_alias", 1400L, + "rows", 1L + ) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByLongAndFloatOutputAsString() + { + if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); + } + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions( + Lists.newArrayList( + new DefaultDimensionSpec("qualityLong", "ql_alias"), + new DefaultDimensionSpec("qualityFloat", "qf_alias") + ) + ) + .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "ql_alias", + "1200", + "qf_alias", + "12000.0", + "rows", + 1L, + "idx", + 158L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "ql_alias", + "1200", + "qf_alias", + "12000.0", + "rows", + 1L, + "idx", + 166L + ) + ); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByNumericStringsAsNumeric() + { + if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); + } + + GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions( + Lists.newArrayList( + new DefaultDimensionSpec("qualityLong", "ql_alias"), + new DefaultDimensionSpec("qualityFloat", "qf_alias"), + new DefaultDimensionSpec(Column.TIME_COLUMN_NAME, "time_alias") + ) + ) + .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + GroupByQuery outerQuery = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions( + Lists.newArrayList( + new DefaultDimensionSpec("time_alias", "time_alias2", ValueType.LONG), + new DefaultDimensionSpec("ql_alias", "ql_alias_long", ValueType.LONG), + new DefaultDimensionSpec("qf_alias", "qf_alias_float", ValueType.FLOAT), + new DefaultDimensionSpec("ql_alias", "ql_alias_float", ValueType.FLOAT) + ) + ) + .setAggregatorSpecs( + Arrays.asList( + new CountAggregatorFactory("count") + ) + ) + .setGranularity(QueryRunnerTestHelper.allGran) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "time_alias2", 1301616000000L, + "ql_alias_long", 1200L, + "qf_alias_float", 12000.0, + "ql_alias_float", 1200.0, + "count", 1L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "time_alias2", 1301702400000L, + "ql_alias_long", 1200L, + "qf_alias_float", 12000.0, + "ql_alias_float", 1200.0, + "count", 1L + ) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, outerQuery); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByNumericStringsAsNumericWithDecoration() + { + if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); + } + + // rows with `technology` have `170000` in the qualityNumericString field + RegexFilteredDimensionSpec regexSpec = new RegexFilteredDimensionSpec( + new DefaultDimensionSpec("qualityNumericString", "ql", ValueType.LONG), + "170000" + ); + + ListFilteredDimensionSpec listFilteredSpec = new ListFilteredDimensionSpec( + new DefaultDimensionSpec("qualityNumericString", "qf", ValueType.FLOAT), + Sets.newHashSet("170000"), + true + ); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions( + Lists.newArrayList( + regexSpec, + listFilteredSpec + ) + ) + .setDimFilter(new InDimFilter("quality", Arrays.asList("entertainment", "technology"), null)) + .setAggregatorSpecs( + Arrays.asList( + new CountAggregatorFactory("count") + ) + ) + .setGranularity(QueryRunnerTestHelper.allGran) + .build(); + + // "entertainment" rows are excluded by the decorated specs, they become empty rows + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "ql", 0L, + "qf", 0.0, + "count", 2L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "ql", 170000L, + "qf", 170000.0, + "count", 2L + ) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByDecorationOnNumerics() + { + if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); + } + + RegexFilteredDimensionSpec regexSpec = new RegexFilteredDimensionSpec( + new DefaultDimensionSpec("qualityLong", "ql", ValueType.LONG), + "1700" + ); + + ListFilteredDimensionSpec listFilteredSpec = new ListFilteredDimensionSpec( + new DefaultDimensionSpec("qualityFloat", "qf", ValueType.FLOAT), + Sets.newHashSet("17000.0"), + true + ); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions( + Lists.newArrayList( + regexSpec, + listFilteredSpec + ) + ) + .setDimFilter(new InDimFilter("quality", Arrays.asList("entertainment", "technology"), null)) + .setAggregatorSpecs( + Arrays.asList( + new CountAggregatorFactory("count") + ) + ) + .setGranularity(QueryRunnerTestHelper.allGran) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "ql", 0L, + "qf", 0.0, + "count", 2L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "ql", 1700L, + "qf", 17000.0, + "count", 2L + ) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByNestedWithInnerQueryNumerics() + { + if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); + } + + GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions( + Lists.newArrayList( + new DefaultDimensionSpec("quality", "alias"), + new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG), + new DefaultDimensionSpec("qualityFloat", "qf_alias", ValueType.FLOAT) + ) + ) + .setDimFilter( + new InDimFilter( + "quality", + Lists.newArrayList("entertainment"), + null + ) + ) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + GroupByQuery outerQuery = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions( + Lists.newArrayList( + new DefaultDimensionSpec("ql_alias", "quallong", ValueType.LONG), + new DefaultDimensionSpec("qf_alias", "qualfloat", ValueType.FLOAT) + ) + ) + .setDimFilter( + new AndDimFilter( + Lists.newArrayList( + new SelectorDimFilter("ql_alias", "1200", null), + new BoundDimFilter( + "qf_alias", + "11095.0", + "12005.0", + true, + true, + null, + null, + StringComparators.NUMERIC + ) + ) + ) + ) + .setAggregatorSpecs( + Arrays.asList( + new LongSumAggregatorFactory("ql_alias_sum", "ql_alias"), + new DoubleSumAggregatorFactory("qf_alias_sum", "qf_alias") + ) + ) + .setGranularity(QueryRunnerTestHelper.allGran) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quallong", 1200L, + "qualfloat", 12000.0, + "ql_alias_sum", 2400L, + "qf_alias_sum", 24000.0 + ) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, outerQuery); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByNestedWithInnerQueryNumericsWithLongTime() + { + if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); + } + + GroupByQuery subQuery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions( + Lists.newArrayList( + new DefaultDimensionSpec("market", "alias"), + new DefaultDimensionSpec("__time", "time_alias", ValueType.LONG), + new DefaultDimensionSpec("index", "index_alias", ValueType.FLOAT) + ) + ) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount + ) + ) + .setGranularity(QueryRunnerTestHelper.allGran) + .build(); + + GroupByQuery outerQuery = GroupByQuery + .builder() + .setDataSource(subQuery) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions( + Lists.newArrayList( + new DefaultDimensionSpec("alias", "market"), + new DefaultDimensionSpec("time_alias", "time_alias2", ValueType.LONG) + ) + ) + .setAggregatorSpecs( + Arrays.asList( + new LongMaxAggregatorFactory("time_alias_max", "time_alias"), + new DoubleMaxAggregatorFactory("index_alias_max", "index_alias") + ) + ) + .setGranularity(QueryRunnerTestHelper.allGran) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "market", "spot", + "time_alias2", 1301616000000L, + "time_alias_max", 1301616000000L, + "index_alias_max", 158.74722290039062 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "market", "spot", + "time_alias2", 1301702400000L, + "time_alias_max", 1301702400000L, + "index_alias_max", 166.01605224609375 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "market", "total_market", + "time_alias2", 1301616000000L, + "time_alias_max", 1301616000000L, + "index_alias_max", 1522.043701171875 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "market", "total_market", + "time_alias2", 1301702400000L, + "time_alias_max", 1301702400000L, + "index_alias_max", 1321.375 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "market", "upfront", + "time_alias2", 1301616000000L, + "time_alias_max", 1301616000000L, + "index_alias_max", 1447.3411865234375 + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "market", "upfront", + "time_alias2", 1301702400000L, + "time_alias_max", 1301702400000L, + "index_alias_max", 1144.3424072265625 + ) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, outerQuery); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testGroupByStringOutputAsLong() + { + if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); + } + + ExtractionFn strlenFn = StrlenExtractionFn.instance(); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new ExtractionDimensionSpec( + QueryRunnerTestHelper.qualityDimension, + "alias", + ValueType.LONG, + strlenFn + ))) + .setDimFilter(new SelectorDimFilter(QueryRunnerTestHelper.qualityDimension, "entertainment", null)) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "alias", + 13L, + "rows", + 1L, + "idx", + 158L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-02", + "alias", + 13L, + "rows", + 1L, + "idx", + 166L + ) ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); 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 d84db7f6f330..7130bad3661a 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java @@ -199,7 +199,7 @@ public SegmentMetadataQueryTest( null, null ) - ), mmap1 ? 93744 : 94517, + ), mmap1 ? 123969 : 124664, 1209, null, null, @@ -243,7 +243,7 @@ public SegmentMetadataQueryTest( null ) // null_column will be included only for incremental index, which makes a little bigger result than expected - ), mmap2 ? 93744 : 94517, + ), mmap2 ? 123969 : 124664, 1209, null, null, diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataUnionQueryTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataUnionQueryTest.java index 701d28159bdd..da4eedb3595e 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataUnionQueryTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataUnionQueryTest.java @@ -110,7 +110,7 @@ public void testSegmentMetadataUnionQuery() null ) ), - mmap ? 374976 : 378068, + mmap ? 495876 : 498656, 4836, null, null, 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 3f3a37703255..ac1f7d91c39f 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java @@ -25,12 +25,16 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; +import io.druid.js.JavaScriptConfig; import io.druid.query.Druids; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; +import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.ExtractionDimensionSpec; +import io.druid.query.extraction.ExtractionFn; +import io.druid.query.extraction.JavaScriptExtractionFn; import io.druid.query.extraction.MapLookupExtractor; import io.druid.query.extraction.TimeFormatExtractionFn; import io.druid.query.filter.AndDimFilter; @@ -47,6 +51,7 @@ import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.segment.TestHelper; import io.druid.segment.column.Column; +import io.druid.segment.column.ValueType; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; @@ -369,8 +374,7 @@ public void testSearchWithExtractionFilter1() new ExtractionDimensionSpec( QueryRunnerTestHelper.qualityDimension, null, - lookupExtractionFn, - null + lookupExtractionFn ) ) .query("☃") @@ -636,6 +640,97 @@ public void testSearchOnTime() checkSearchQuery(searchQuery, expectedHits); } + @Test + public void testSearchOnLongColumn() + { + SearchQuery searchQuery = Druids.newSearchQueryBuilder() + .dimensions( + new DefaultDimensionSpec( + Column.TIME_COLUMN_NAME, Column.TIME_COLUMN_NAME, + ValueType.LONG + ) + ) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .query("1297123200000") + .build(); + + List expectedHits = Lists.newLinkedList(); + expectedHits.add(new SearchHit(Column.TIME_COLUMN_NAME, "1297123200000", 13)); + checkSearchQuery(searchQuery, expectedHits); + } + + @Test + public void testSearchOnLongColumnWithExFn() + { + String jsFn = "function(str) { return 'super-' + str; }"; + ExtractionFn jsExtractionFn = new JavaScriptExtractionFn(jsFn, false, JavaScriptConfig.getDefault()); + + SearchQuery searchQuery = Druids.newSearchQueryBuilder() + .dimensions( + new ExtractionDimensionSpec( + Column.TIME_COLUMN_NAME, Column.TIME_COLUMN_NAME, + jsExtractionFn + ) + ) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .query("1297123200000") + .build(); + + List expectedHits = Lists.newLinkedList(); + expectedHits.add(new SearchHit(Column.TIME_COLUMN_NAME, "super-1297123200000", 13)); + checkSearchQuery(searchQuery, expectedHits); + } + + @Test + public void testSearchOnFloatColumn() + { + SearchQuery searchQuery = Druids.newSearchQueryBuilder() + .dimensions( + new DefaultDimensionSpec( + QueryRunnerTestHelper.indexMetric, QueryRunnerTestHelper.indexMetric, + ValueType.FLOAT + ) + ) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .query("100.7") + .build(); + + List expectedHits = Lists.newLinkedList(); + expectedHits.add(new SearchHit(QueryRunnerTestHelper.indexMetric, "100.706055", 1)); + expectedHits.add(new SearchHit(QueryRunnerTestHelper.indexMetric, "100.7756", 1)); + checkSearchQuery(searchQuery, expectedHits); + } + + @Test + public void testSearchOnFloatColumnWithExFn() + { + String jsFn = "function(str) { return 'super-' + str; }"; + ExtractionFn jsExtractionFn = new JavaScriptExtractionFn(jsFn, false, JavaScriptConfig.getDefault()); + + SearchQuery searchQuery = Druids.newSearchQueryBuilder() + .dimensions( + new ExtractionDimensionSpec( + QueryRunnerTestHelper.indexMetric, QueryRunnerTestHelper.indexMetric, + jsExtractionFn + ) + ) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .query("100.7") + .build(); + + List expectedHits = Lists.newLinkedList(); + expectedHits.add(new SearchHit(QueryRunnerTestHelper.indexMetric, "super-100.7060546875", 1)); + expectedHits.add(new SearchHit(QueryRunnerTestHelper.indexMetric, "super-100.77559661865234", 1)); + checkSearchQuery(searchQuery, expectedHits); + } private void checkSearchQuery(Query searchQuery, List expectedResults) { diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerWithCaseTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerWithCaseTest.java index c46e7ce57708..99cfc0794726 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerWithCaseTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerWithCaseTest.java @@ -81,10 +81,10 @@ public static Iterable constructorFeeder() throws IOException configs[2].setSearchStrategy(AutoStrategy.NAME); CharSource input = CharSource.wrap( - "2011-01-12T00:00:00.000Z\tspot\tAutoMotive\tPREFERRED\ta\u0001preferred\t100.000000\n" + - "2011-01-12T00:00:00.000Z\tSPot\tbusiness\tpreferred\tb\u0001Preferred\t100.000000\n" + - "2011-01-12T00:00:00.000Z\tspot\tentertainment\tPREFERRed\te\u0001preferred\t100.000000\n" + - "2011-01-13T00:00:00.000Z\tspot\tautomotive\tpreferred\ta\u0001preferred\t94.874713" + "2011-01-12T00:00:00.000Z\tspot\tAutoMotive\t1000\t10000.0\t100000\tPREFERRED\ta\u0001preferred\t100.000000\n" + + "2011-01-12T00:00:00.000Z\tSPot\tbusiness\t1100\t11000.0\t110000\tpreferred\tb\u0001Preferred\t100.000000\n" + + "2011-01-12T00:00:00.000Z\tspot\tentertainment\t1200\t12000.0\t120000\tPREFERRed\te\u0001preferred\t100.000000\n" + + "2011-01-13T00:00:00.000Z\tspot\tautomotive\t1000\t10000.0\t100000\tpreferred\ta\u0001preferred\t94.874713" ); IncrementalIndex index1 = TestIndex.makeRealtimeIndex(input); diff --git a/processing/src/test/java/io/druid/query/select/MultiSegmentSelectQueryTest.java b/processing/src/test/java/io/druid/query/select/MultiSegmentSelectQueryTest.java index a9add340eb92..65d1aeb5015c 100644 --- a/processing/src/test/java/io/druid/query/select/MultiSegmentSelectQueryTest.java +++ b/processing/src/test/java/io/druid/query/select/MultiSegmentSelectQueryTest.java @@ -76,43 +76,44 @@ public class MultiSegmentSelectQueryTest QueryRunnerTestHelper.NOOP_QUERYWATCHER ); - // time modified version of druid.sample.tsv + // time modified version of druid.sample.numeric.tsv public static final String[] V_0112 = { - "2011-01-12T00:00:00.000Z spot automotive preferred apreferred 100.000000", - "2011-01-12T01:00:00.000Z spot business preferred bpreferred 100.000000", - "2011-01-12T02:00:00.000Z spot entertainment preferred epreferred 100.000000", - "2011-01-12T03:00:00.000Z spot health preferred hpreferred 100.000000", - "2011-01-12T04:00:00.000Z spot mezzanine preferred mpreferred 100.000000", - "2011-01-12T05:00:00.000Z spot news preferred npreferred 100.000000", - "2011-01-12T06:00:00.000Z spot premium preferred ppreferred 100.000000", - "2011-01-12T07:00:00.000Z spot technology preferred tpreferred 100.000000", - "2011-01-12T08:00:00.000Z spot travel preferred tpreferred 100.000000", - "2011-01-12T09:00:00.000Z total_market mezzanine preferred mpreferred 1000.000000", - "2011-01-12T10:00:00.000Z total_market premium preferred ppreferred 1000.000000", - "2011-01-12T11:00:00.000Z upfront mezzanine preferred mpreferred 800.000000 value", - "2011-01-12T12:00:00.000Z upfront premium preferred ppreferred 800.000000 value" + "2011-01-12T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 100.000000", + "2011-01-12T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 100.000000", + "2011-01-12T02:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 100.000000", + "2011-01-12T03:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 100.000000", + "2011-01-12T04:00:00.000Z spot mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 100.000000", + "2011-01-12T05:00:00.000Z spot news\t1500\t15000.0\t150000\tpreferred npreferred 100.000000", + "2011-01-12T06:00:00.000Z spot premium\t1600\t16000.0\t160000\tpreferred ppreferred 100.000000", + "2011-01-12T07:00:00.000Z spot technology\t1700\t17000.0\t170000\tpreferred tpreferred 100.000000", + "2011-01-12T08:00:00.000Z spot travel\t1800\t18000.0\t180000\tpreferred tpreferred 100.000000", + "2011-01-12T09:00:00.000Z total_market mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 1000.000000", + "2011-01-12T10:00:00.000Z total_market premium\t1600\t16000.0\t160000\tpreferred ppreferred 1000.000000", + "2011-01-12T11:00:00.000Z upfront mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 800.000000 value", + "2011-01-12T12:00:00.000Z upfront premium\t1600\t16000.0\t160000\tpreferred ppreferred 800.000000 value" }; + public static final String[] V_0113 = { - "2011-01-13T00:00:00.000Z spot automotive preferred apreferred 94.874713", - "2011-01-13T01:00:00.000Z spot business preferred bpreferred 103.629399", - "2011-01-13T02:00:00.000Z spot entertainment preferred epreferred 110.087299", - "2011-01-13T03:00:00.000Z spot health preferred hpreferred 114.947403", - "2011-01-13T04:00:00.000Z spot mezzanine preferred mpreferred 104.465767", - "2011-01-13T05:00:00.000Z spot news preferred npreferred 102.851683", - "2011-01-13T06:00:00.000Z spot premium preferred ppreferred 108.863011", - "2011-01-13T07:00:00.000Z spot technology preferred tpreferred 111.356672", - "2011-01-13T08:00:00.000Z spot travel preferred tpreferred 106.236928", - "2011-01-13T09:00:00.000Z total_market mezzanine preferred mpreferred 1040.945505", - "2011-01-13T10:00:00.000Z total_market premium preferred ppreferred 1689.012875", - "2011-01-13T11:00:00.000Z upfront mezzanine preferred mpreferred 826.060182 value", - "2011-01-13T12:00:00.000Z upfront premium preferred ppreferred 1564.617729 value" + "2011-01-13T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 94.874713", + "2011-01-13T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 103.629399", + "2011-01-13T02:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 110.087299", + "2011-01-13T03:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 114.947403", + "2011-01-13T04:00:00.000Z spot mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 104.465767", + "2011-01-13T05:00:00.000Z spot news\t1500\t15000.0\t150000\tpreferred npreferred 102.851683", + "2011-01-13T06:00:00.000Z spot premium\t1600\t16000.0\t160000\tpreferred ppreferred 108.863011", + "2011-01-13T07:00:00.000Z spot technology\t1700\t17000.0\t170000\tpreferred tpreferred 111.356672", + "2011-01-13T08:00:00.000Z spot travel\t1800\t18000.0\t180000\tpreferred tpreferred 106.236928", + "2011-01-13T09:00:00.000Z total_market mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 1040.945505", + "2011-01-13T10:00:00.000Z total_market premium\t1600\t16000.0\t160000\tpreferred ppreferred 1689.012875", + "2011-01-13T11:00:00.000Z upfront mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 826.060182 value", + "2011-01-13T12:00:00.000Z upfront premium\t1600\t16000.0\t160000\tpreferred ppreferred 1564.617729 value" }; public static final String[] V_OVERRIDE = { - "2011-01-12T04:00:00.000Z spot automotive preferred apreferred 999.000000", - "2011-01-12T05:00:00.000Z spot business preferred bpreferred 999.000000", - "2011-01-12T06:00:00.000Z spot entertainment preferred epreferred 999.000000", - "2011-01-12T07:00:00.000Z spot health preferred hpreferred 999.000000" + "2011-01-12T04:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 999.000000", + "2011-01-12T05:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 999.000000", + "2011-01-12T06:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 999.000000", + "2011-01-12T07:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 999.000000" }; private static Segment segment0; 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 258bb57a4f32..81ec8381c9f4 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java @@ -29,6 +29,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Sequences; +import io.druid.js.JavaScriptConfig; import io.druid.query.Druids; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; @@ -37,6 +38,8 @@ import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; import io.druid.query.dimension.ExtractionDimensionSpec; +import io.druid.query.extraction.ExtractionFn; +import io.druid.query.extraction.JavaScriptExtractionFn; import io.druid.query.extraction.MapLookupExtractor; import io.druid.query.filter.AndDimFilter; import io.druid.query.filter.DimFilter; @@ -44,6 +47,8 @@ import io.druid.query.lookup.LookupExtractionFn; import io.druid.query.spec.LegacySegmentSpec; import io.druid.query.spec.QuerySegmentSpec; +import io.druid.segment.column.Column; +import io.druid.segment.column.ValueType; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; @@ -64,36 +69,36 @@ @RunWith(Parameterized.class) public class SelectQueryRunnerTest { - // copied from druid.sample.tsv + // copied from druid.sample.numeric.tsv public static final String[] V_0112 = { - "2011-01-12T00:00:00.000Z spot automotive preferred apreferred 100.000000", - "2011-01-12T00:00:00.000Z spot business preferred bpreferred 100.000000", - "2011-01-12T00:00:00.000Z spot entertainment preferred epreferred 100.000000", - "2011-01-12T00:00:00.000Z spot health preferred hpreferred 100.000000", - "2011-01-12T00:00:00.000Z spot mezzanine preferred mpreferred 100.000000", - "2011-01-12T00:00:00.000Z spot news preferred npreferred 100.000000", - "2011-01-12T00:00:00.000Z spot premium preferred ppreferred 100.000000", - "2011-01-12T00:00:00.000Z spot technology preferred tpreferred 100.000000", - "2011-01-12T00:00:00.000Z spot travel preferred tpreferred 100.000000", - "2011-01-12T00:00:00.000Z total_market mezzanine preferred mpreferred 1000.000000", - "2011-01-12T00:00:00.000Z total_market premium preferred ppreferred 1000.000000", - "2011-01-12T00:00:00.000Z upfront mezzanine preferred mpreferred 800.000000 value", - "2011-01-12T00:00:00.000Z upfront premium preferred ppreferred 800.000000 value" + "2011-01-12T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 100.000000", + "2011-01-12T00:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 100.000000", + "2011-01-12T00:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 100.000000", + "2011-01-12T00:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 100.000000", + "2011-01-12T00:00:00.000Z spot mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 100.000000", + "2011-01-12T00:00:00.000Z spot news\t1500\t15000.0\t150000\tpreferred npreferred 100.000000", + "2011-01-12T00:00:00.000Z spot premium\t1600\t16000.0\t160000\tpreferred ppreferred 100.000000", + "2011-01-12T00:00:00.000Z spot technology\t1700\t17000.0\t170000\tpreferred tpreferred 100.000000", + "2011-01-12T00:00:00.000Z spot travel\t1800\t18000.0\t180000\tpreferred tpreferred 100.000000", + "2011-01-12T00:00:00.000Z total_market mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 1000.000000", + "2011-01-12T00:00:00.000Z total_market premium\t1600\t16000.0\t160000\tpreferred ppreferred 1000.000000", + "2011-01-12T00:00:00.000Z upfront mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 800.000000 value", + "2011-01-12T00:00:00.000Z upfront premium\t1600\t16000.0\t160000\tpreferred ppreferred 800.000000 value" }; public static final String[] V_0113 = { - "2011-01-13T00:00:00.000Z spot automotive preferred apreferred 94.874713", - "2011-01-13T00:00:00.000Z spot business preferred bpreferred 103.629399", - "2011-01-13T00:00:00.000Z spot entertainment preferred epreferred 110.087299", - "2011-01-13T00:00:00.000Z spot health preferred hpreferred 114.947403", - "2011-01-13T00:00:00.000Z spot mezzanine preferred mpreferred 104.465767", - "2011-01-13T00:00:00.000Z spot news preferred npreferred 102.851683", - "2011-01-13T00:00:00.000Z spot premium preferred ppreferred 108.863011", - "2011-01-13T00:00:00.000Z spot technology preferred tpreferred 111.356672", - "2011-01-13T00:00:00.000Z spot travel preferred tpreferred 106.236928", - "2011-01-13T00:00:00.000Z total_market mezzanine preferred mpreferred 1040.945505", - "2011-01-13T00:00:00.000Z total_market premium preferred ppreferred 1689.012875", - "2011-01-13T00:00:00.000Z upfront mezzanine preferred mpreferred 826.060182 value", - "2011-01-13T00:00:00.000Z upfront premium preferred ppreferred 1564.617729 value" + "2011-01-13T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 94.874713", + "2011-01-13T00:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 103.629399", + "2011-01-13T00:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 110.087299", + "2011-01-13T00:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 114.947403", + "2011-01-13T00:00:00.000Z spot mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 104.465767", + "2011-01-13T00:00:00.000Z spot news\t1500\t15000.0\t150000\tpreferred npreferred 102.851683", + "2011-01-13T00:00:00.000Z spot premium\t1600\t16000.0\t160000\tpreferred ppreferred 108.863011", + "2011-01-13T00:00:00.000Z spot technology\t1700\t17000.0\t170000\tpreferred tpreferred 111.356672", + "2011-01-13T00:00:00.000Z spot travel\t1800\t18000.0\t180000\tpreferred tpreferred 106.236928", + "2011-01-13T00:00:00.000Z total_market mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 1040.945505", + "2011-01-13T00:00:00.000Z total_market premium\t1600\t16000.0\t160000\tpreferred ppreferred 1689.012875", + "2011-01-13T00:00:00.000Z upfront mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 826.060182 value", + "2011-01-13T00:00:00.000Z upfront premium\t1600\t16000.0\t160000\tpreferred ppreferred 1564.617729 value" }; public static final QuerySegmentSpec I_0112_0114 = new LegacySegmentSpec( @@ -157,8 +162,8 @@ public void testFullOnSelect() PagingOffset offset = query.getPagingOffset(QueryRunnerTestHelper.segmentId); List> expectedResults = toExpected( toFullEvents(V_0112_0114), - Lists.newArrayList("market", "quality", "placement", "placementish", "partial_null_column", "null_column"), - Lists.newArrayList("index", "quality_uniques", "indexMin", "indexMaxPlusTen"), + Lists.newArrayList("market", "quality", "qualityNumericString", "placement", "placementish", "partial_null_column", "null_column"), + Lists.newArrayList("index", "quality_uniques", "qualityLong", "qualityFloat", "indexMin", "indexMaxPlusTen"), offset.startOffset(), offset.threshold() ); @@ -247,7 +252,7 @@ public void testFullOnSelectWithDimensionSpec() new SelectResultValue( ImmutableMap.of(QueryRunnerTestHelper.segmentId, 2), Sets.newHashSet("mar", "qual", "place"), - Sets.newHashSet("index", "quality_uniques", "indexMin", "indexMaxPlusTen"), + Sets.newHashSet("index", "quality_uniques", "qualityLong", "qualityFloat", "indexMin", "indexMaxPlusTen"), Arrays.asList( new EventHolder( QueryRunnerTestHelper.segmentId, @@ -293,7 +298,7 @@ public void testFullOnSelectWithDimensionSpec() new SelectResultValue( ImmutableMap.of(QueryRunnerTestHelper.segmentId, -3), Sets.newHashSet("mar", "qual", "place"), - Sets.newHashSet("index", "quality_uniques", "indexMin", "indexMaxPlusTen"), + Sets.newHashSet("index", "qualityLong", "qualityFloat", "quality_uniques", "indexMin", "indexMaxPlusTen"), Arrays.asList( new EventHolder( QueryRunnerTestHelper.segmentId, @@ -360,6 +365,9 @@ public void testSelectWithDimsAndMets() null, null, null, + null, + null, + null, QueryRunnerTestHelper.indexMetric + ":FLOAT" }, V_0112_0114 @@ -553,8 +561,8 @@ public void testFullSelectNoResults() new DateTime("2011-01-12T00:00:00.000Z"), new SelectResultValue( ImmutableMap.of(), - Sets.newHashSet("market", "quality", "placement", "placementish", "partial_null_column", "null_column"), - Sets.newHashSet("index", "quality_uniques", "indexMin", "indexMaxPlusTen"), + Sets.newHashSet("market", "quality", "qualityNumericString", "placement", "placementish", "partial_null_column", "null_column"), + Sets.newHashSet("index", "quality_uniques", "qualityLong", "qualityFloat", "indexMin", "indexMaxPlusTen"), Lists.newArrayList() ) ) @@ -597,6 +605,239 @@ public void testFullSelectNoDimensionAndMetric() verify(expectedResults, results); } + @Test + public void testFullOnSelectWithLongAndFloat() + { + List dimSpecs = Arrays.asList( + new DefaultDimensionSpec(QueryRunnerTestHelper.indexMetric, "floatIndex", ValueType.FLOAT), + new DefaultDimensionSpec(Column.TIME_COLUMN_NAME, "longTime", ValueType.LONG) + ); + + SelectQuery query = newTestQuery() + .dimensionSpecs(dimSpecs) + .metrics(Arrays.asList(Column.TIME_COLUMN_NAME, "index")) + .intervals(I_0112_0114) + .build(); + + HashMap context = new HashMap(); + Iterable> results = Sequences.toList( + runner.run(query, context), + Lists.>newArrayList() + ); + + List> expectedResultsAsc = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SelectResultValue( + ImmutableMap.of(QueryRunnerTestHelper.segmentId, 2), + Sets.newHashSet("null_column", "floatIndex", "longTime"), + Sets.newHashSet("__time", "index"), + Arrays.asList( + new EventHolder( + QueryRunnerTestHelper.segmentId, + 0, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z")) + .put("longTime", 1294790400000L) + .put("floatIndex", 100.0f) + .put(QueryRunnerTestHelper.indexMetric, 100.000000F) + .put(Column.TIME_COLUMN_NAME, 1294790400000L) + .build() + ), + new EventHolder( + QueryRunnerTestHelper.segmentId, + 1, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z")) + .put("longTime", 1294790400000L) + .put("floatIndex", 100.0f) + .put(QueryRunnerTestHelper.indexMetric, 100.000000F) + .put(Column.TIME_COLUMN_NAME, 1294790400000L) + .build() + ), + new EventHolder( + QueryRunnerTestHelper.segmentId, + 2, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z")) + .put("longTime", 1294790400000L) + .put("floatIndex", 100.0f) + .put(QueryRunnerTestHelper.indexMetric, 100.000000F) + .put(Column.TIME_COLUMN_NAME, 1294790400000L) + .build() + ) + ) + ) + ) + ); + + List> expectedResultsDsc = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SelectResultValue( + ImmutableMap.of(QueryRunnerTestHelper.segmentId, -3), + Sets.newHashSet("null_column", "floatIndex", "longTime"), + Sets.newHashSet("__time", "index"), + Arrays.asList( + new EventHolder( + QueryRunnerTestHelper.segmentId, + -1, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-13T00:00:00.000Z")) + .put("longTime", 1294876800000L) + .put("floatIndex", 1564.6177f) + .put(QueryRunnerTestHelper.indexMetric, 1564.6177f) + .put(Column.TIME_COLUMN_NAME, 1294876800000L) + .build() + ), + new EventHolder( + QueryRunnerTestHelper.segmentId, + -2, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-13T00:00:00.000Z")) + .put("longTime", 1294876800000L) + .put("floatIndex", 826.0602f) + .put(QueryRunnerTestHelper.indexMetric, 826.0602f) + .put(Column.TIME_COLUMN_NAME, 1294876800000L) + .build() + ), + new EventHolder( + QueryRunnerTestHelper.segmentId, + -3, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-13T00:00:00.000Z")) + .put("longTime", 1294876800000L) + .put("floatIndex", 1689.0128f) + .put(QueryRunnerTestHelper.indexMetric, 1689.0128f) + .put(Column.TIME_COLUMN_NAME, 1294876800000L) + .build() + ) + ) + ) + ) + ); + + verify(descending ? expectedResultsDsc : expectedResultsAsc, populateNullColumnAtLastForQueryableIndexCase(results, "null_column")); + } + + @Test + public void testFullOnSelectWithLongAndFloatWithExFn() + { + String jsFn = "function(str) { return 'super-' + str; }"; + ExtractionFn jsExtractionFn = new JavaScriptExtractionFn(jsFn, false, JavaScriptConfig.getDefault()); + + List dimSpecs = Arrays.asList( + new ExtractionDimensionSpec(QueryRunnerTestHelper.indexMetric, "floatIndex", jsExtractionFn), + new ExtractionDimensionSpec(Column.TIME_COLUMN_NAME, "longTime", jsExtractionFn) + ); + + SelectQuery query = newTestQuery() + .dimensionSpecs(dimSpecs) + .metrics(Arrays.asList(Column.TIME_COLUMN_NAME, "index")) + .intervals(I_0112_0114) + .build(); + + HashMap context = new HashMap(); + Iterable> results = Sequences.toList( + runner.run(query, context), + Lists.>newArrayList() + ); + + List> expectedResultsAsc = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SelectResultValue( + ImmutableMap.of(QueryRunnerTestHelper.segmentId, 2), + Sets.newHashSet("null_column", "floatIndex", "longTime"), + Sets.newHashSet("__time", "index"), + Arrays.asList( + new EventHolder( + QueryRunnerTestHelper.segmentId, + 0, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z")) + .put("longTime", "super-1294790400000") + .put("floatIndex", "super-100") + .put(QueryRunnerTestHelper.indexMetric, 100.000000F) + .put(Column.TIME_COLUMN_NAME, 1294790400000L) + .build() + ), + new EventHolder( + QueryRunnerTestHelper.segmentId, + 1, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z")) + .put("longTime", "super-1294790400000") + .put("floatIndex", "super-100") + .put(QueryRunnerTestHelper.indexMetric, 100.000000F) + .put(Column.TIME_COLUMN_NAME, 1294790400000L) + .build() + ), + new EventHolder( + QueryRunnerTestHelper.segmentId, + 2, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z")) + .put("longTime", "super-1294790400000") + .put("floatIndex", "super-100") + .put(QueryRunnerTestHelper.indexMetric, 100.000000F) + .put(Column.TIME_COLUMN_NAME, 1294790400000L) + .build() + ) + ) + ) + ) + ); + + List> expectedResultsDsc = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SelectResultValue( + ImmutableMap.of(QueryRunnerTestHelper.segmentId, -3), + Sets.newHashSet("null_column", "floatIndex", "longTime"), + Sets.newHashSet("__time", "index"), + Arrays.asList( + new EventHolder( + QueryRunnerTestHelper.segmentId, + -1, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-13T00:00:00.000Z")) + .put("longTime", "super-1294876800000") + .put("floatIndex", "super-1564.61767578125") + .put(QueryRunnerTestHelper.indexMetric, 1564.6177f) + .put(Column.TIME_COLUMN_NAME, 1294876800000L) + .build() + ), + new EventHolder( + QueryRunnerTestHelper.segmentId, + -2, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-13T00:00:00.000Z")) + .put("longTime", "super-1294876800000") + .put("floatIndex", "super-826.0601806640625") + .put(QueryRunnerTestHelper.indexMetric, 826.0602f) + .put(Column.TIME_COLUMN_NAME, 1294876800000L) + .build() + ), + new EventHolder( + QueryRunnerTestHelper.segmentId, + -3, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-13T00:00:00.000Z")) + .put("longTime", "super-1294876800000") + .put("floatIndex", "super-1689.0128173828125") + .put(QueryRunnerTestHelper.indexMetric, 1689.0128f) + .put(Column.TIME_COLUMN_NAME, 1294876800000L) + .build() + ) + ) + ) + ) + ); + + verify(descending ? expectedResultsDsc : expectedResultsAsc, populateNullColumnAtLastForQueryableIndexCase(results, "null_column")); + } + private Map toPagingIdentifier(int startDelta, boolean descending) { return ImmutableMap.of( @@ -610,6 +851,9 @@ private List>> toFullEvents(final String[]... valueSet) return toEvents(new String[]{EventHolder.timestampKey + ":TIME", QueryRunnerTestHelper.marketDimension + ":STRING", QueryRunnerTestHelper.qualityDimension + ":STRING", + "qualityLong" + ":LONG", + "qualityFloat" + ":FLOAT", + "qualityNumericString" + ":STRING", QueryRunnerTestHelper.placementDimension + ":STRING", QueryRunnerTestHelper.placementishDimension + ":STRINGS", QueryRunnerTestHelper.indexMetric + ":FLOAT", diff --git a/processing/src/test/java/io/druid/query/select/SelectQuerySpecTest.java b/processing/src/test/java/io/druid/query/select/SelectQuerySpecTest.java index 9c4ee4c58ec6..dd103ee990a0 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQuerySpecTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQuerySpecTest.java @@ -58,7 +58,9 @@ public void testSerializationLegacyString() throws Exception + "\"descending\":true," + "\"filter\":null," + "\"granularity\":{\"type\":\"all\"}," - + "\"dimensions\":[{\"type\":\"default\",\"dimension\":\"market\",\"outputName\":\"market\"},{\"type\":\"default\",\"dimension\":\"quality\",\"outputName\":\"quality\"}]," + + "\"dimensions\":" + + "[{\"type\":\"default\",\"dimension\":\"market\",\"outputName\":\"market\",\"outputType\":\"STRING\"}," + + "{\"type\":\"default\",\"dimension\":\"quality\",\"outputName\":\"quality\",\"outputType\":\"STRING\"}]," + "\"metrics\":[\"index\"]," + "\"virtualColumns\":[]," + "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":3,\"fromNext\":false}," 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 a785be275ed4..c860f014d8a5 100644 --- a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java @@ -90,21 +90,21 @@ public TimeBoundaryQueryRunnerTest( // Adapted from MultiSegmentSelectQueryTest, with modifications to make filtering meaningful public static final String[] V_0112 = { - "2011-01-12T01:00:00.000Z spot business preferred bpreferred 100.000000", - "2011-01-12T02:00:00.000Z spot entertainment preferred epreferred 100.000000", - "2011-01-13T00:00:00.000Z spot automotive preferred apreferred 100.000000", - "2011-01-13T01:00:00.000Z spot business preferred bpreferred 100.000000", + "2011-01-12T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 100.000000", + "2011-01-12T02:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 100.000000", + "2011-01-13T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 100.000000", + "2011-01-13T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 100.000000", }; public static final String[] V_0113 = { - "2011-01-14T00:00:00.000Z spot automotive preferred apreferred 94.874713", - "2011-01-14T02:00:00.000Z spot entertainment preferred epreferred 110.087299", - "2011-01-15T00:00:00.000Z spot automotive preferred apreferred 94.874713", - "2011-01-15T01:00:00.000Z spot business preferred bpreferred 103.629399", - "2011-01-16T00:00:00.000Z spot automotive preferred apreferred 94.874713", - "2011-01-16T01:00:00.000Z spot business preferred bpreferred 103.629399", - "2011-01-16T02:00:00.000Z spot entertainment preferred epreferred 110.087299", - "2011-01-17T01:00:00.000Z spot business preferred bpreferred 103.629399", - "2011-01-17T02:00:00.000Z spot entertainment preferred epreferred 110.087299", + "2011-01-14T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 94.874713", + "2011-01-14T02:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 110.087299", + "2011-01-15T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 94.874713", + "2011-01-15T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 103.629399", + "2011-01-16T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 94.874713", + "2011-01-16T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 103.629399", + "2011-01-16T02:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 110.087299", + "2011-01-17T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 103.629399", + "2011-01-17T02:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 110.087299", }; private static IncrementalIndex newIndex(String minTimeStamp) 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 e48eb397afae..a6cc15aec04f 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -27,6 +27,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import io.druid.collections.StupidPool; import io.druid.granularity.QueryGranularities; import io.druid.granularity.QueryGranularity; @@ -57,11 +58,13 @@ import io.druid.query.dimension.DimensionSpec; import io.druid.query.aggregation.last.LongLastAggregatorFactory; import io.druid.query.dimension.ExtractionDimensionSpec; +import io.druid.query.dimension.ListFilteredDimensionSpec; import io.druid.query.extraction.DimExtractionFn; import io.druid.query.extraction.ExtractionFn; import io.druid.query.extraction.JavaScriptExtractionFn; import io.druid.query.extraction.MapLookupExtractor; import io.druid.query.extraction.RegexDimExtractionFn; +import io.druid.query.extraction.StrlenExtractionFn; import io.druid.query.extraction.TimeFormatExtractionFn; import io.druid.query.filter.AndDimFilter; import io.druid.query.filter.DimFilter; @@ -73,10 +76,13 @@ import io.druid.query.timeseries.TimeseriesQuery; import io.druid.segment.TestHelper; import io.druid.segment.column.Column; +import io.druid.segment.column.ValueType; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -140,6 +146,9 @@ public ByteBuffer get() private final QueryRunner> runner; + @Rule + public ExpectedException expectedException = ExpectedException.none(); + public TopNQueryRunnerTest( QueryRunner> runner ) @@ -1696,8 +1705,7 @@ public void testTopNDimExtractionToOne() throws IOException new ExtractionDimensionSpec( QueryRunnerTestHelper.marketDimension, QueryRunnerTestHelper.marketDimension, - new JavaScriptExtractionFn("function(f) { return \"POTATO\"; }", false, JavaScriptConfig.getDefault()), - null + new JavaScriptExtractionFn("function(f) { return \"POTATO\"; }", false, JavaScriptConfig.getDefault()) ) ) .metric("rows") @@ -1755,7 +1763,7 @@ public void testTopNCollapsingDimExtraction() .dimension( new ExtractionDimensionSpec( QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.qualityDimension, - new RegexDimExtractionFn(".(.)", false, null), null + new RegexDimExtractionFn(".(.)", false, null) ) ) .metric("index") @@ -1825,8 +1833,7 @@ public void testTopNDimExtraction() new ExtractionDimensionSpec( QueryRunnerTestHelper.marketDimension, QueryRunnerTestHelper.marketDimension, - new RegexDimExtractionFn("(.)", false, null), - null + new RegexDimExtractionFn("(.)", false, null) ) ) .metric("rows") @@ -1890,8 +1897,7 @@ public void testTopNDimExtractionFastTopNOptimalWithReplaceMissing() false ), false, "MISSING", true, false - ), - null + ) ) ) .metric("rows") @@ -1955,8 +1961,7 @@ public void testTopNDimExtractionFastTopNUnOptimalWithReplaceMissing() false ), false, "MISSING", false, false - ), - null + ) ) ) .metric("rows") @@ -2021,8 +2026,7 @@ public void testTopNDimExtractionFastTopNOptimal() false ), true, null, true, false - ), - null + ) ) ) .metric("rows") @@ -2089,8 +2093,7 @@ public void testTopNDimExtractionFastTopNUnOptimal() false ), true, null, false, false - ), - null + ) ) ) .metric("rows") @@ -2156,8 +2159,7 @@ public void testTopNLexicographicDimExtractionOptimalNamespace() false ), true, null, true, false - ), - null + ) ) ) .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) @@ -2223,8 +2225,7 @@ public void testTopNLexicographicDimExtractionUnOptimalNamespace() false ), true, null, false, false - ), - null + ) ) ) .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) @@ -2291,8 +2292,7 @@ public void testTopNLexicographicDimExtractionOptimalNamespaceWithRunner() false ), true, null, true, false - ), - null + ) ) ) .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) @@ -2345,8 +2345,7 @@ public void testTopNLexicographicDimExtraction() new ExtractionDimensionSpec( QueryRunnerTestHelper.marketDimension, QueryRunnerTestHelper.marketDimension, - new RegexDimExtractionFn("(.)", false, null), - null + new RegexDimExtractionFn("(.)", false, null) ) ) .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) @@ -2399,8 +2398,7 @@ public void testInvertedTopNLexicographicDimExtraction2() new ExtractionDimensionSpec( QueryRunnerTestHelper.marketDimension, QueryRunnerTestHelper.marketDimension, - new RegexDimExtractionFn("..(.)", false, null), - null + new RegexDimExtractionFn("..(.)", false, null) ) ) .metric(new InvertedTopNMetricSpec(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))) @@ -2453,8 +2451,7 @@ public void testTopNLexicographicDimExtractionWithPreviousStop() new ExtractionDimensionSpec( QueryRunnerTestHelper.marketDimension, QueryRunnerTestHelper.marketDimension, - new RegexDimExtractionFn("(.)", false, null), - null + new RegexDimExtractionFn("(.)", false, null) ) ) .metric(new DimensionTopNMetricSpec("s", StringComparators.LEXICOGRAPHIC)) @@ -2524,7 +2521,7 @@ public ExtractionType getExtractionType() { return ExtractionType.MANY_TO_ONE; } - }, null + } ) ) .metric(new DimensionTopNMetricSpec("s", StringComparators.LEXICOGRAPHIC)) @@ -2571,8 +2568,7 @@ public void testInvertedTopNLexicographicDimExtractionWithPreviousStop() new ExtractionDimensionSpec( QueryRunnerTestHelper.marketDimension, QueryRunnerTestHelper.marketDimension, - new RegexDimExtractionFn("(.)", false, null), - null + new RegexDimExtractionFn("(.)", false, null) ) ) .metric(new InvertedTopNMetricSpec(new DimensionTopNMetricSpec("u", StringComparators.LEXICOGRAPHIC))) @@ -2618,8 +2614,7 @@ public void testInvertedTopNLexicographicDimExtractionWithPreviousStop2() new ExtractionDimensionSpec( QueryRunnerTestHelper.marketDimension, QueryRunnerTestHelper.marketDimension, - new RegexDimExtractionFn("..(.)", false, null), - null + new RegexDimExtractionFn("..(.)", false, null) ) ) .metric(new InvertedTopNMetricSpec(new DimensionTopNMetricSpec("p", StringComparators.LEXICOGRAPHIC))) @@ -2697,8 +2692,7 @@ public ExtractionType getExtractionType() new ExtractionDimensionSpec( QueryRunnerTestHelper.marketDimension, QueryRunnerTestHelper.marketDimension, - nullStringDimExtraction, - null + nullStringDimExtraction ) ) .build(); @@ -2787,8 +2781,7 @@ public ExtractionType getExtractionType() new ExtractionDimensionSpec( QueryRunnerTestHelper.marketDimension, QueryRunnerTestHelper.marketDimension, - emptyStringDimExtraction, - null + emptyStringDimExtraction ) ) .build(); @@ -3212,8 +3205,7 @@ public void testTopNTimeExtraction() new ExtractionDimensionSpec( Column.TIME_COLUMN_NAME, "dayOfWeek", - new TimeFormatExtractionFn("EEEE", null, null, null, false), - null + new TimeFormatExtractionFn("EEEE", null, null, null, false) ) ) .metric("index") @@ -3678,4 +3670,998 @@ public void testTopNWithExtractionFilterNoExistingValue() final Sequence> retval = runWithPreMergeAndMerge(topNQueryWithNULLValueExtraction); TestHelper.assertExpectedResults(expectedResults, retval); } + + @Test + public void testFullOnTopNFloatColumn() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(new DefaultDimensionSpec(QueryRunnerTestHelper.indexMetric, "index_alias", ValueType.FLOAT)) + .metric(QueryRunnerTestHelper.indexMetric) + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("index_alias", 1000.0f) + .put(QueryRunnerTestHelper.indexMetric, 2000.0D) + .put("rows", 2L) + .put("addRowsIndexConstant", 2003.0D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) + .put("maxIndex", 1000.0D) + .put("minIndex", 1000.0D) + .build(), + ImmutableMap.builder() + .put("index_alias", 1870.06103515625f) + .put(QueryRunnerTestHelper.indexMetric, 1870.06103515625D) + .put("rows", 1L) + .put("addRowsIndexConstant", 1872.06103515625D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 1870.06103515625D) + .build(), + ImmutableMap.builder() + .put("index_alias", 1862.7379150390625f) + .put(QueryRunnerTestHelper.indexMetric, 1862.7379150390625D) + .put("rows", 1L) + .put("addRowsIndexConstant", 1864.7379150390625D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1862.7379150390625D) + .put("minIndex", 1862.7379150390625D) + .build(), + ImmutableMap.builder() + .put("index_alias", 1743.9217529296875f) + .put(QueryRunnerTestHelper.indexMetric, 1743.9217529296875D) + .put("rows", 1L) + .put("addRowsIndexConstant", 1745.9217529296875D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1743.9217529296875D) + .put("minIndex", 1743.9217529296875D) + .build() + ) + ) + ) + ); + assertExpectedResults(expectedResults, query); + } + + @Test + public void testFullOnTopNFloatColumnWithExFn() + { + String jsFn = "function(str) { return 'super-' + str; }"; + ExtractionFn jsExtractionFn = new JavaScriptExtractionFn(jsFn, false, JavaScriptConfig.getDefault()); + + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(new ExtractionDimensionSpec(QueryRunnerTestHelper.indexMetric, "index_alias", jsExtractionFn)) + .metric(QueryRunnerTestHelper.indexMetric) + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("index_alias", "super-1000") + .put(QueryRunnerTestHelper.indexMetric, 2000.0D) + .put("rows", 2L) + .put("addRowsIndexConstant", 2003.0D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) + .put("maxIndex", 1000.0D) + .put("minIndex", 1000.0D) + .build(), + ImmutableMap.builder() + .put("index_alias", "super-1870.06103515625") + .put(QueryRunnerTestHelper.indexMetric, 1870.06103515625D) + .put("rows", 1L) + .put("addRowsIndexConstant", 1872.06103515625D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 1870.06103515625D) + .build(), + ImmutableMap.builder() + .put("index_alias", "super-1862.7379150390625") + .put(QueryRunnerTestHelper.indexMetric, 1862.7379150390625D) + .put("rows", 1L) + .put("addRowsIndexConstant", 1864.7379150390625D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1862.7379150390625D) + .put("minIndex", 1862.7379150390625D) + .build(), + ImmutableMap.builder() + .put("index_alias", "super-1743.9217529296875") + .put(QueryRunnerTestHelper.indexMetric, 1743.9217529296875D) + .put("rows", 1L) + .put("addRowsIndexConstant", 1745.9217529296875D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1743.9217529296875D) + .put("minIndex", 1743.9217529296875D) + .build() + ) + ) + ) + ); + assertExpectedResults(expectedResults, query); + } + + @Test + public void testFullOnTopNFloatColumnAsString() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(new DefaultDimensionSpec("qualityFloat", "qf_alias")) + .metric("maxIndex") + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("qf_alias", "14000.0") + .put(QueryRunnerTestHelper.indexMetric, 217725.42022705078D) + .put("rows", 279L) + .put("addRowsIndexConstant", 218005.42022705078D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 91.27055358886719D) + .build(), + ImmutableMap.builder() + .put("qf_alias", "16000.0") + .put(QueryRunnerTestHelper.indexMetric, 210865.67966461182D) + .put("rows", 279L) + .put("addRowsIndexConstant", 211145.67966461182D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1862.7379150390625D) + .put("minIndex", 99.2845230102539D) + .build(), + ImmutableMap.builder() + .put("qf_alias", "10000.0") + .put(QueryRunnerTestHelper.indexMetric, 12270.807106018066D) + .put("rows", 93L) + .put("addRowsIndexConstant", 12364.807106018066D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 277.2735290527344D) + .put("minIndex", 71.31593322753906D) + .build(), + ImmutableMap.builder() + .put("qf_alias", "12000.0") + .put(QueryRunnerTestHelper.indexMetric, 12086.472755432129D) + .put("rows", 93L) + .put("addRowsIndexConstant", 12180.472755432129D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 193.78756713867188D) + .put("minIndex", 84.71052551269531D) + .build() + ) + ) + ) + ); + assertExpectedResults(expectedResults, query); + } + + @Test + public void testFullOnTopNLongColumn() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG)) + .metric("maxIndex") + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("ql_alias", 1400L) + .put(QueryRunnerTestHelper.indexMetric, 217725.42022705078D) + .put("rows", 279L) + .put("addRowsIndexConstant", 218005.42022705078D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 91.27055358886719D) + .build(), + ImmutableMap.builder() + .put("ql_alias", 1600L) + .put(QueryRunnerTestHelper.indexMetric, 210865.67966461182D) + .put("rows", 279L) + .put("addRowsIndexConstant", 211145.67966461182D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1862.7379150390625D) + .put("minIndex", 99.2845230102539D) + .build(), + ImmutableMap.builder() + .put("ql_alias", 1000L) + .put(QueryRunnerTestHelper.indexMetric, 12270.807106018066D) + .put("rows", 93L) + .put("addRowsIndexConstant", 12364.807106018066D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 277.2735290527344D) + .put("minIndex", 71.31593322753906D) + .build(), + ImmutableMap.builder() + .put("ql_alias", 1200L) + .put(QueryRunnerTestHelper.indexMetric, 12086.472755432129D) + .put("rows", 93L) + .put("addRowsIndexConstant", 12180.472755432129D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 193.78756713867188D) + .put("minIndex", 84.71052551269531D) + .build() + ) + ) + ) + ); + assertExpectedResults(expectedResults, query); + } + + @Test + public void testFullOnTopNLongColumnWithExFn() + { + String jsFn = "function(str) { return 'super-' + str; }"; + ExtractionFn jsExtractionFn = new JavaScriptExtractionFn(jsFn, false, JavaScriptConfig.getDefault()); + + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(new ExtractionDimensionSpec("qualityLong", "ql_alias", jsExtractionFn)) + .metric("maxIndex") + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("ql_alias", "super-1400") + .put(QueryRunnerTestHelper.indexMetric, 217725.42022705078D) + .put("rows", 279L) + .put("addRowsIndexConstant", 218005.42022705078D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 91.27055358886719D) + .build(), + ImmutableMap.builder() + .put("ql_alias", "super-1600") + .put(QueryRunnerTestHelper.indexMetric, 210865.67966461182D) + .put("rows", 279L) + .put("addRowsIndexConstant", 211145.67966461182D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1862.7379150390625D) + .put("minIndex", 99.2845230102539D) + .build(), + ImmutableMap.builder() + .put("ql_alias", "super-1000") + .put(QueryRunnerTestHelper.indexMetric, 12270.807106018066D) + .put("rows", 93L) + .put("addRowsIndexConstant", 12364.807106018066D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 277.2735290527344D) + .put("minIndex", 71.31593322753906D) + .build(), + ImmutableMap.builder() + .put("ql_alias", "super-1200") + .put(QueryRunnerTestHelper.indexMetric, 12086.472755432129D) + .put("rows", 93L) + .put("addRowsIndexConstant", 12180.472755432129D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 193.78756713867188D) + .put("minIndex", 84.71052551269531D) + .build() + ) + ) + ) + ); + assertExpectedResults(expectedResults, query); + } + + @Test + public void testFullOnTopNLongColumnAsString() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(new DefaultDimensionSpec("qualityLong", "ql_alias")) + .metric("maxIndex") + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("ql_alias", "1400") + .put(QueryRunnerTestHelper.indexMetric, 217725.42022705078D) + .put("rows", 279L) + .put("addRowsIndexConstant", 218005.42022705078D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 91.27055358886719D) + .build(), + ImmutableMap.builder() + .put("ql_alias", "1600") + .put(QueryRunnerTestHelper.indexMetric, 210865.67966461182D) + .put("rows", 279L) + .put("addRowsIndexConstant", 211145.67966461182D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1862.7379150390625D) + .put("minIndex", 99.2845230102539D) + .build(), + ImmutableMap.builder() + .put("ql_alias", "1000") + .put(QueryRunnerTestHelper.indexMetric, 12270.807106018066D) + .put("rows", 93L) + .put("addRowsIndexConstant", 12364.807106018066D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 277.2735290527344D) + .put("minIndex", 71.31593322753906D) + .build(), + ImmutableMap.builder() + .put("ql_alias", "1200") + .put(QueryRunnerTestHelper.indexMetric, 12086.472755432129D) + .put("rows", 93L) + .put("addRowsIndexConstant", 12180.472755432129D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 193.78756713867188D) + .put("minIndex", 84.71052551269531D) + .build() + ) + ) + ) + ); + assertExpectedResults(expectedResults, query); + } + + @Test + public void testFullOnTopNNumericStringColumnAsLong() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(new DefaultDimensionSpec("qualityNumericString", "qns_alias", ValueType.LONG)) + .metric("maxIndex") + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("qns_alias", 140000L) + .put(QueryRunnerTestHelper.indexMetric, 217725.42022705078D) + .put("rows", 279L) + .put("addRowsIndexConstant", 218005.42022705078D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 91.27055358886719D) + .build(), + ImmutableMap.builder() + .put("qns_alias", 160000L) + .put(QueryRunnerTestHelper.indexMetric, 210865.67966461182D) + .put("rows", 279L) + .put("addRowsIndexConstant", 211145.67966461182D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1862.7379150390625D) + .put("minIndex", 99.2845230102539D) + .build(), + ImmutableMap.builder() + .put("qns_alias", 100000L) + .put(QueryRunnerTestHelper.indexMetric, 12270.807106018066D) + .put("rows", 93L) + .put("addRowsIndexConstant", 12364.807106018066D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 277.2735290527344D) + .put("minIndex", 71.31593322753906D) + .build(), + ImmutableMap.builder() + .put("qns_alias", 120000L) + .put(QueryRunnerTestHelper.indexMetric, 12086.472755432129D) + .put("rows", 93L) + .put("addRowsIndexConstant", 12180.472755432129D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 193.78756713867188D) + .put("minIndex", 84.71052551269531D) + .build() + ) + ) + ) + ); + assertExpectedResults(expectedResults, query); + } + + @Test + public void testFullOnTopNNumericStringColumnAsFloat() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(new DefaultDimensionSpec("qualityNumericString", "qns_alias", ValueType.FLOAT)) + .metric("maxIndex") + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("qns_alias", 140000.0f) + .put(QueryRunnerTestHelper.indexMetric, 217725.42022705078D) + .put("rows", 279L) + .put("addRowsIndexConstant", 218005.42022705078D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 91.27055358886719D) + .build(), + ImmutableMap.builder() + .put("qns_alias", 160000.0f) + .put(QueryRunnerTestHelper.indexMetric, 210865.67966461182D) + .put("rows", 279L) + .put("addRowsIndexConstant", 211145.67966461182D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1862.7379150390625D) + .put("minIndex", 99.2845230102539D) + .build(), + ImmutableMap.builder() + .put("qns_alias", 100000.0f) + .put(QueryRunnerTestHelper.indexMetric, 12270.807106018066D) + .put("rows", 93L) + .put("addRowsIndexConstant", 12364.807106018066D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 277.2735290527344D) + .put("minIndex", 71.31593322753906D) + .build(), + ImmutableMap.builder() + .put("qns_alias", 120000.0f) + .put(QueryRunnerTestHelper.indexMetric, 12086.472755432129D) + .put("rows", 93L) + .put("addRowsIndexConstant", 12180.472755432129D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 193.78756713867188D) + .put("minIndex", 84.71052551269531D) + .build() + ) + ) + ) + ); + assertExpectedResults(expectedResults, query); + } + + @Test + public void testFullOnTopNLongTimeColumn() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(new DefaultDimensionSpec(Column.TIME_COLUMN_NAME, "time_alias", ValueType.LONG)) + .metric("maxIndex") + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("time_alias", 1296345600000L) + .put(QueryRunnerTestHelper.indexMetric, 5497.331253051758D) + .put("rows", 13L) + .put("addRowsIndexConstant", 5511.331253051758D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 97.02391052246094D) + .build(), + ImmutableMap.builder() + .put("time_alias", 1298678400000L) + .put(QueryRunnerTestHelper.indexMetric, 6541.463027954102D) + .put("rows", 13L) + .put("addRowsIndexConstant", 6555.463027954102D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) + .put("maxIndex", 1862.7379150390625D) + .put("minIndex", 83.099365234375D) + .build(), + ImmutableMap.builder() + .put("time_alias", 1301529600000L) + .put(QueryRunnerTestHelper.indexMetric, 6814.467971801758D) + .put("rows", 13L) + .put("addRowsIndexConstant", 6828.467971801758D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) + .put("maxIndex", 1734.27490234375D) + .put("minIndex", 93.39083862304688D) + .build(), + ImmutableMap.builder() + .put("time_alias", 1294876800000L) + .put(QueryRunnerTestHelper.indexMetric, 6077.949111938477D) + .put("rows", 13L) + .put("addRowsIndexConstant", 6091.949111938477D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) + .put("maxIndex", 1689.0128173828125D) + .put("minIndex", 94.87471008300781D) + .build() + ) + ) + ) + ); + assertExpectedResults(expectedResults, query); + } + + @Test + public void testFullOnTopNLongTimeColumnWithExFn() + { + String jsFn = "function(str) { return 'super-' + str; }"; + ExtractionFn jsExtractionFn = new JavaScriptExtractionFn(jsFn, false, JavaScriptConfig.getDefault()); + + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(new ExtractionDimensionSpec(Column.TIME_COLUMN_NAME, "time_alias", jsExtractionFn)) + .metric("maxIndex") + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("time_alias", "super-1296345600000") + .put(QueryRunnerTestHelper.indexMetric, 5497.331253051758D) + .put("rows", 13L) + .put("addRowsIndexConstant", 5511.331253051758D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 97.02391052246094D) + .build(), + ImmutableMap.builder() + .put("time_alias", "super-1298678400000") + .put(QueryRunnerTestHelper.indexMetric, 6541.463027954102D) + .put("rows", 13L) + .put("addRowsIndexConstant", 6555.463027954102D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) + .put("maxIndex", 1862.7379150390625D) + .put("minIndex", 83.099365234375D) + .build(), + ImmutableMap.builder() + .put("time_alias", "super-1301529600000") + .put(QueryRunnerTestHelper.indexMetric, 6814.467971801758D) + .put("rows", 13L) + .put("addRowsIndexConstant", 6828.467971801758D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) + .put("maxIndex", 1734.27490234375D) + .put("minIndex", 93.39083862304688D) + .build(), + ImmutableMap.builder() + .put("time_alias", "super-1294876800000") + .put(QueryRunnerTestHelper.indexMetric, 6077.949111938477D) + .put("rows", 13L) + .put("addRowsIndexConstant", 6091.949111938477D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) + .put("maxIndex", 1689.0128173828125D) + .put("minIndex", 94.87471008300781D) + .build() + ) + ) + ) + ); + assertExpectedResults(expectedResults, query); + } + + @Test + public void testFullOnTopNDimExtractionAllNulls() + { + String jsFn = "function(str) { return null; }"; + ExtractionFn jsExtractionFn = new JavaScriptExtractionFn(jsFn, false, JavaScriptConfig.getDefault()); + + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(new ExtractionDimensionSpec( + QueryRunnerTestHelper.marketDimension, + QueryRunnerTestHelper.marketDimension, + jsExtractionFn + )) + .metric(QueryRunnerTestHelper.indexMetric) + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + Map expectedMap = new HashMap<>(); + expectedMap.put(QueryRunnerTestHelper.marketDimension, null); + expectedMap.put("rows", 1209L); + expectedMap.put("index", 503332.5071372986D); + expectedMap.put("addRowsIndexConstant", 504542.5071372986D); + expectedMap.put("uniques", 9.019833517963864); + expectedMap.put("maxIndex", 1870.06103515625D); + expectedMap.put("minIndex", 59.02102279663086D); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + expectedMap + ) + ) + ) + ); + assertExpectedResults(expectedResults, query); + } + + @Test + public void testFullOnTopNStringOutputAsLong() + { + ExtractionFn strlenFn = StrlenExtractionFn.instance(); + + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(new ExtractionDimensionSpec(QueryRunnerTestHelper.qualityDimension, "alias", ValueType.LONG, strlenFn)) + .metric("maxIndex") + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("alias", 9L) + .put(QueryRunnerTestHelper.indexMetric, 217725.42022705078D) + .put("rows", 279L) + .put("addRowsIndexConstant", 218005.42022705078D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 91.27055358886719D) + .build(), + ImmutableMap.builder() + .put("alias", 7L) + .put(QueryRunnerTestHelper.indexMetric, 210865.67966461182D) + .put("rows", 279L) + .put("addRowsIndexConstant", 211145.67966461182D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1862.7379150390625D) + .put("minIndex", 99.2845230102539D) + .build(), + ImmutableMap.builder() + .put("alias", 10L) + .put(QueryRunnerTestHelper.indexMetric, 20479.497562408447D) + .put("rows", 186L) + .put("addRowsIndexConstant", 20666.497562408447D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) + .put("maxIndex", 277.2735290527344D) + .put("minIndex", 59.02102279663086D) + .build(), + ImmutableMap.builder() + .put("alias", 13L) + .put(QueryRunnerTestHelper.indexMetric, 12086.472755432129D) + .put("rows", 93L) + .put("addRowsIndexConstant", 12180.472755432129D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 193.78756713867188D) + .put("minIndex", 84.71052551269531D) + .build() + ) + ) + ) + ); + assertExpectedResults(expectedResults, query); + } + + @Test + public void testFullOnTopNNumericStringColumnWithDecoration() + { + ListFilteredDimensionSpec filteredSpec = new ListFilteredDimensionSpec( + new DefaultDimensionSpec("qualityNumericString", "qns_alias", ValueType.LONG), + Sets.newHashSet("120000", "140000", "160000"), + true + ); + + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(filteredSpec) + .metric("maxIndex") + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("qns_alias", 140000L) + .put(QueryRunnerTestHelper.indexMetric, 217725.42022705078D) + .put("rows", 279L) + .put("addRowsIndexConstant", 218005.42022705078D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 91.27055358886719D) + .build(), + ImmutableMap.builder() + .put("qns_alias", 160000L) + .put(QueryRunnerTestHelper.indexMetric, 210865.67966461182D) + .put("rows", 279L) + .put("addRowsIndexConstant", 211145.67966461182D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1862.7379150390625D) + .put("minIndex", 99.2845230102539D) + .build(), + ImmutableMap.builder() + .put("qns_alias", 120000L) + .put(QueryRunnerTestHelper.indexMetric, 12086.472755432129D) + .put("rows", 93L) + .put("addRowsIndexConstant", 12180.472755432129D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 193.78756713867188D) + .put("minIndex", 84.71052551269531D) + .build() + ) + ) + ) + ); + assertExpectedResults(expectedResults, query); + } + + @Test + public void testFullOnTopNDecorationOnNumeric() + { + ListFilteredDimensionSpec filteredSpec = new ListFilteredDimensionSpec( + new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG), + Sets.newHashSet("1200", "1400", "1600"), + true + ); + + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(filteredSpec) + .metric("maxIndex") + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("ql_alias", 1400L) + .put(QueryRunnerTestHelper.indexMetric, 217725.42022705078D) + .put("rows", 279L) + .put("addRowsIndexConstant", 218005.42022705078D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 91.27055358886719D) + .build(), + ImmutableMap.builder() + .put("ql_alias", 1600L) + .put(QueryRunnerTestHelper.indexMetric, 210865.67966461182D) + .put("rows", 279L) + .put("addRowsIndexConstant", 211145.67966461182D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 1862.7379150390625D) + .put("minIndex", 99.2845230102539D) + .build(), + ImmutableMap.builder() + .put("ql_alias", 1200L) + .put(QueryRunnerTestHelper.indexMetric, 12086.472755432129D) + .put("rows", 93L) + .put("addRowsIndexConstant", 12180.472755432129D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_1) + .put("maxIndex", 193.78756713867188D) + .put("minIndex", 84.71052551269531D) + .build() + ) + ) + ) + ); + assertExpectedResults(expectedResults, query); + } } diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java index 612dbfc818bf..573cf717cd12 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java @@ -94,8 +94,7 @@ public void testQuerySerdeWithLookupExtractionFn() throws IOException new ExtractionDimensionSpec( marketDimension, marketDimension, - new LookupExtractionFn(new MapLookupExtractor(ImmutableMap.of("foo", "bar"), false), true, null, false, false), - null + new LookupExtractionFn(new MapLookupExtractor(ImmutableMap.of("foo", "bar"), false), true, null, false, false) ) ) .metric(new NumericTopNMetricSpec(indexMetric)) diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index efc7a6405aeb..b6022e4e8967 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -35,6 +35,7 @@ import io.druid.query.aggregation.DoubleMaxAggregatorFactory; import io.druid.query.aggregation.DoubleMinAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import io.druid.segment.incremental.IncrementalIndex; @@ -59,6 +60,9 @@ public class TestIndex "ts", "market", "quality", + "qualityLong", + "qualityFloat", + "qualityNumericString", "placement", "placementish", "index", @@ -71,6 +75,7 @@ public class TestIndex public static final String[] DIMENSIONS = new String[]{ "market", "quality", + "qualityNumericString", "placement", "placementish", "partial_null_column", @@ -88,7 +93,9 @@ public class TestIndex new DoubleSumAggregatorFactory(METRICS[0], METRICS[0]), new DoubleMinAggregatorFactory(METRICS[1], METRICS[0]), new DoubleMaxAggregatorFactory(METRICS[2], VIRTUAL_COLUMNS.getVirtualColumns()[0].getOutputName()), - new HyperUniquesAggregatorFactory("quality_uniques", "quality") + new HyperUniquesAggregatorFactory("quality_uniques", "quality"), + new LongSumAggregatorFactory("qualityLong", "qualityLong"), + new DoubleSumAggregatorFactory("qualityFloat", "qualityFloat") }; private static final IndexSpec indexSpec = new IndexSpec(); @@ -115,7 +122,7 @@ public static IncrementalIndex getIncrementalTestIndex() } } - return realtimeIndex = makeRealtimeIndex("druid.sample.tsv"); + return realtimeIndex = makeRealtimeIndex("druid.sample.numeric.tsv"); } public static IncrementalIndex getNoRollupIncrementalTestIndex() @@ -126,7 +133,7 @@ public static IncrementalIndex getNoRollupIncrementalTestIndex() } } - return noRollupRealtimeIndex = makeRealtimeIndex("druid.sample.tsv", false); + return noRollupRealtimeIndex = makeRealtimeIndex("druid.sample.numeric.tsv", false); } public static QueryableIndex getMMappedTestIndex() @@ -165,8 +172,8 @@ public static QueryableIndex mergedRealtimeIndex() } try { - IncrementalIndex top = makeRealtimeIndex("druid.sample.tsv.top"); - IncrementalIndex bottom = makeRealtimeIndex("druid.sample.tsv.bottom"); + IncrementalIndex top = makeRealtimeIndex("druid.sample.numeric.tsv.top"); + IncrementalIndex bottom = makeRealtimeIndex("druid.sample.numeric.tsv.bottom"); File tmpFile = File.createTempFile("yay", "who"); tmpFile.delete(); diff --git a/processing/src/test/java/io/druid/segment/filter/FilterPartitionTest.java b/processing/src/test/java/io/druid/segment/filter/FilterPartitionTest.java index d21477b279cb..0a99667570b7 100644 --- a/processing/src/test/java/io/druid/segment/filter/FilterPartitionTest.java +++ b/processing/src/test/java/io/druid/segment/filter/FilterPartitionTest.java @@ -37,6 +37,7 @@ import io.druid.query.filter.AndDimFilter; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.DimFilter; +import io.druid.query.filter.DruidFloatPredicate; import io.druid.query.filter.DruidLongPredicate; import io.druid.query.filter.DruidPredicateFactory; import io.druid.query.filter.Filter; @@ -142,6 +143,20 @@ public boolean applyLong(long input) } }; } + + @Override + public DruidFloatPredicate makeFloatPredicate() + { + return new DruidFloatPredicate() + { + @Override + public boolean applyFloat(float input) + { + return Objects.equals(valueOrNull, String.valueOf(input)); + } + }; + } + }; return new NoBitmapDimensionPredicateFilter(dimension, predicateFactory, extractionFn); diff --git a/processing/src/test/java/io/druid/segment/filter/FloatFilteringTest.java b/processing/src/test/java/io/druid/segment/filter/FloatFilteringTest.java new file mode 100644 index 000000000000..c63e97117c13 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/filter/FloatFilteringTest.java @@ -0,0 +1,430 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.filter; + +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +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 io.druid.data.input.InputRow; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.InputRowParser; +import io.druid.data.input.impl.MapInputRowParser; +import io.druid.data.input.impl.TimeAndDimsParseSpec; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.java.util.common.Pair; +import io.druid.js.JavaScriptConfig; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.extraction.MapLookupExtractor; +import io.druid.query.filter.BoundDimFilter; +import io.druid.query.filter.DimFilter; +import io.druid.query.filter.InDimFilter; +import io.druid.query.filter.JavaScriptDimFilter; +import io.druid.query.filter.RegexDimFilter; +import io.druid.query.filter.SearchQueryDimFilter; +import io.druid.query.filter.SelectorDimFilter; +import io.druid.query.lookup.LookupExtractionFn; +import io.druid.query.lookup.LookupExtractor; +import io.druid.query.ordering.StringComparators; +import io.druid.query.search.search.ContainsSearchQuerySpec; +import io.druid.segment.IndexBuilder; +import io.druid.segment.StorageAdapter; +import io.druid.segment.incremental.IncrementalIndexSchema; +import org.joda.time.DateTime; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +@RunWith(Parameterized.class) +public class FloatFilteringTest extends BaseFilterTest +{ + private static final String FLOAT_COLUMN = "flt"; + private static final String TIMESTAMP_COLUMN = "ts"; + private static int EXECUTOR_NUM_THREADS = 16; + private static int EXECUTOR_NUM_TASKS = 2000; + + private static final InputRowParser> PARSER = new MapInputRowParser( + new TimeAndDimsParseSpec( + new TimestampSpec(TIMESTAMP_COLUMN, "millis", new DateTime("2000")), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")), + null, + null + ) + ) + ); + + private static final List ROWS = ImmutableList.of( + PARSER.parse(ImmutableMap.of("ts", 1L, "dim0", "1", "flt", 1.0f, "dim1", "", "dim2", ImmutableList.of("a", "b"))), + PARSER.parse(ImmutableMap.of("ts", 2L, "dim0", "2", "flt", 2.0f, "dim1", "10", "dim2", ImmutableList.of())), + PARSER.parse(ImmutableMap.of("ts", 3L, "dim0", "3", "flt", 3.0f, "dim1", "2", "dim2", ImmutableList.of(""))), + PARSER.parse(ImmutableMap.of("ts", 4L, "dim0", "4", "flt", 4.0f, "dim1", "1", "dim2", ImmutableList.of("a"))), + PARSER.parse(ImmutableMap.of("ts", 5L, "dim0", "5", "flt", 5.0f, "dim1", "def", "dim2", ImmutableList.of("c"))), + PARSER.parse(ImmutableMap.of("ts", 6L, "dim0", "6", "flt", 6.0f, "dim1", "abc")) + ); + + public FloatFilteringTest( + String testName, + IndexBuilder indexBuilder, + Function> finisher, + boolean cnf, + boolean optimize + ) + { + super( + testName, + ROWS, + indexBuilder.schema( + new IncrementalIndexSchema.Builder() + .withMetrics( + new AggregatorFactory[]{ + new DoubleSumAggregatorFactory(FLOAT_COLUMN, FLOAT_COLUMN) + } + ).build() + ), + finisher, + cnf, + optimize + ); + } + + @AfterClass + public static void tearDown() throws Exception + { + BaseFilterTest.tearDown(FloatFilteringTest.class.getName()); + } + + @Test + public void testFloatColumnFiltering() + { + assertFilterMatches( + new SelectorDimFilter(FLOAT_COLUMN, "0", null), + ImmutableList.of() + ); + + assertFilterMatches( + new SelectorDimFilter(FLOAT_COLUMN, "0.0", null), + ImmutableList.of() + ); + + assertFilterMatches( + new SelectorDimFilter(FLOAT_COLUMN, "3", null), + ImmutableList.of("3") + ); + + assertFilterMatches( + new SelectorDimFilter(FLOAT_COLUMN, "3.0", null), + ImmutableList.of("3") + ); + + assertFilterMatches( + new BoundDimFilter(FLOAT_COLUMN, "2", "5", false, false, null, null, StringComparators.NUMERIC), + ImmutableList.of("2", "3", "4", "5") + ); + + assertFilterMatches( + new BoundDimFilter(FLOAT_COLUMN, "2.0", "5.0", false, false, null, null, StringComparators.NUMERIC), + ImmutableList.of("2", "3", "4", "5") + ); + + assertFilterMatches( + new BoundDimFilter(FLOAT_COLUMN, "1", "4", true, true, null, null, StringComparators.NUMERIC), + ImmutableList.of("2", "3") + ); + + assertFilterMatches( + new BoundDimFilter(FLOAT_COLUMN, "1.0", "4.0", true, true, null, null, StringComparators.NUMERIC), + ImmutableList.of("2", "3") + ); + + assertFilterMatches( + new InDimFilter(FLOAT_COLUMN, Arrays.asList("2", "4", "8"), null), + ImmutableList.of("2", "4") + ); + + assertFilterMatches( + new InDimFilter(FLOAT_COLUMN, Arrays.asList("2.0", "4.0", "8.0"), null), + ImmutableList.of("2", "4") + ); + + // cross the hashing threshold to test hashset implementation, filter on even values + List infilterValues = new ArrayList<>(InDimFilter.NUMERIC_HASHING_THRESHOLD * 2); + for (int i = 0; i < InDimFilter.NUMERIC_HASHING_THRESHOLD * 2; i++) { + infilterValues.add(String.valueOf(i * 2)); + } + assertFilterMatches( + new InDimFilter(FLOAT_COLUMN, infilterValues, null), + ImmutableList.of("2", "4", "6") + ); + + + String jsFn = "function(x) { return(x === 3 || x === 5) }"; + assertFilterMatches( + new JavaScriptDimFilter(FLOAT_COLUMN, jsFn, null, JavaScriptConfig.getDefault()), + ImmutableList.of("3", "5") + ); + + String jsFn2 = "function(x) { return(x === 3.0 || x === 5.0) }"; + assertFilterMatches( + new JavaScriptDimFilter(FLOAT_COLUMN, jsFn2, null, JavaScriptConfig.getDefault()), + ImmutableList.of("3", "5") + ); + + assertFilterMatches( + new RegexDimFilter(FLOAT_COLUMN, "4", null), + ImmutableList.of("4") + ); + + assertFilterMatches( + new RegexDimFilter(FLOAT_COLUMN, "4.0", null), + ImmutableList.of("4") + ); + + assertFilterMatches( + new SearchQueryDimFilter(FLOAT_COLUMN, new ContainsSearchQuerySpec("2", true), null), + ImmutableList.of("2") + ); + + assertFilterMatches( + new SearchQueryDimFilter(FLOAT_COLUMN, new ContainsSearchQuerySpec("2", true), null), + ImmutableList.of("2") + ); + } + + @Test + public void testFloatColumnFilteringWithNonNumbers() + { + assertFilterMatches( + new SelectorDimFilter(FLOAT_COLUMN, "", null), + ImmutableList.of() + ); + + assertFilterMatches( + new SelectorDimFilter(FLOAT_COLUMN, null, null), + ImmutableList.of() + ); + + assertFilterMatches( + new SelectorDimFilter(FLOAT_COLUMN, "abc", null), + ImmutableList.of() + ); + + assertFilterMatches( + new BoundDimFilter(FLOAT_COLUMN, "a", "b", false, false, null, null, StringComparators.NUMERIC), + ImmutableList.of() + ); + + assertFilterMatches( + new BoundDimFilter(FLOAT_COLUMN, " ", "4", false, false, null, null, StringComparators.NUMERIC), + ImmutableList.of("1", "2", "3", "4") + ); + + assertFilterMatches( + new BoundDimFilter(FLOAT_COLUMN, " ", "4", false, false, null, null, StringComparators.LEXICOGRAPHIC), + ImmutableList.of("1", "2", "3") + ); + + assertFilterMatches( + new BoundDimFilter(FLOAT_COLUMN, " ", "4.0", false, false, null, null, StringComparators.LEXICOGRAPHIC), + ImmutableList.of("1", "2", "3", "4") + ); + + assertFilterMatches( + new BoundDimFilter(FLOAT_COLUMN, " ", "A", false, false, null, null, StringComparators.NUMERIC), + ImmutableList.of() + ); + + assertFilterMatches( + new BoundDimFilter(FLOAT_COLUMN, " ", "A", false, false, null, null, StringComparators.LEXICOGRAPHIC), + ImmutableList.of("1", "2", "3", "4", "5", "6") + ); + } + + @Test + public void testFloatFilterWithExtractionFn() + { + final Map stringMap = new HashMap<>(); + stringMap.put("1.0", "Monday"); + stringMap.put("2.0", "Tuesday"); + stringMap.put("3.0", "Wednesday"); + stringMap.put("4.0", "Thursday"); + stringMap.put("5.0", "Friday"); + stringMap.put("6.0", "Saturday"); + LookupExtractor mapExtractor = new MapLookupExtractor(stringMap, false); + LookupExtractionFn exfn = new LookupExtractionFn(mapExtractor, false, "UNKNOWN", false, true); + + assertFilterMatches( + new SelectorDimFilter(FLOAT_COLUMN, "Monday", exfn), + ImmutableList.of("1") + ); + assertFilterMatches( + new SelectorDimFilter(FLOAT_COLUMN, "Notaday", exfn), + ImmutableList.of() + ); + + assertFilterMatches( + new BoundDimFilter(FLOAT_COLUMN, "Fridax", "Fridaz", false, false, null, exfn, StringComparators.ALPHANUMERIC), + ImmutableList.of("5") + ); + assertFilterMatches( + new BoundDimFilter(FLOAT_COLUMN, "Friday", "Friday", true, true, null, exfn, StringComparators.ALPHANUMERIC), + ImmutableList.of() + ); + + assertFilterMatches( + new InDimFilter(FLOAT_COLUMN, Arrays.asList("Caturday", "Saturday", "Tuesday"), exfn), + ImmutableList.of("2", "6") + ); + + // test InFilter HashSet implementation + List bigList = Arrays.asList( + "Saturday", "Tuesday", + "Caturday", "Xanaday", "Vojuday", "Gribaday", "Kipoday", "Dheferday", "Fakeday", "Qeearaday", + "Hello", "World", "1", "2", "3", "4", "5", "6", "7" + ); + assertFilterMatches( + new InDimFilter(FLOAT_COLUMN, bigList, exfn), + ImmutableList.of("2", "6") + ); + + String jsFn = "function(x) { return(x === 'Wednesday' || x === 'Thursday') }"; + assertFilterMatches( + new JavaScriptDimFilter(FLOAT_COLUMN, jsFn, exfn, JavaScriptConfig.getDefault()), + ImmutableList.of("3", "4") + ); + + assertFilterMatches( + new RegexDimFilter(FLOAT_COLUMN, ".*day", exfn), + ImmutableList.of("1", "2", "3", "4", "5", "6") + ); + + assertFilterMatches( + new SearchQueryDimFilter(FLOAT_COLUMN, new ContainsSearchQuerySpec("s", true), exfn), + ImmutableList.of("2", "3", "4") + ); + } + + @Test + public void testMultithreaded() + { + assertFilterMatchesMultithreaded( + new SelectorDimFilter(FLOAT_COLUMN, "3", null), + ImmutableList.of("3") + ); + + assertFilterMatchesMultithreaded( + new SelectorDimFilter(FLOAT_COLUMN, "3.0", null), + ImmutableList.of("3") + ); + + assertFilterMatchesMultithreaded( + new InDimFilter(FLOAT_COLUMN, Arrays.asList("2", "4", "8"), null), + ImmutableList.of("2", "4") + ); + + assertFilterMatchesMultithreaded( + new InDimFilter(FLOAT_COLUMN, Arrays.asList("2.0", "4.0", "8.0"), null), + ImmutableList.of("2", "4") + ); + + // cross the hashing threshold to test hashset implementation, filter on even values + List infilterValues = new ArrayList<>(InDimFilter.NUMERIC_HASHING_THRESHOLD * 2); + for (int i = 0; i < InDimFilter.NUMERIC_HASHING_THRESHOLD * 2; i++) { + infilterValues.add(String.valueOf(i * 2)); + } + assertFilterMatchesMultithreaded( + new InDimFilter(FLOAT_COLUMN, infilterValues, null), + ImmutableList.of("2", "4", "6") + ); + + assertFilterMatches( + new BoundDimFilter(FLOAT_COLUMN, "2", "5", false, false, null, null, StringComparators.NUMERIC), + ImmutableList.of("2", "3", "4", "5") + ); + + assertFilterMatches( + new BoundDimFilter(FLOAT_COLUMN, "2.0", "5.0", false, false, null, null, StringComparators.NUMERIC), + ImmutableList.of("2", "3", "4", "5") + ); + } + + private void assertFilterMatchesMultithreaded( + final DimFilter filter, + final List expectedRows + ) + { + testWithExecutor(filter, expectedRows); + } + + private Runnable makeFilterRunner( + final DimFilter filter, + final List expectedRows + ) + { + return new Runnable() + { + @Override + public void run() + { + assertFilterMatches(filter, expectedRows); + } + }; + } + + private void testWithExecutor( + final DimFilter filter, + final List expectedRows + ) + { + ListeningExecutorService executor = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(EXECUTOR_NUM_THREADS)); + + List> futures = new ArrayList<>(); + + for (int i = 0; i < EXECUTOR_NUM_TASKS; i++) { + Runnable runnable = makeFilterRunner(filter, expectedRows); + ListenableFuture fut = executor.submit(runnable); + futures.add(fut); + } + + try { + Futures.allAsList(futures).get(60, TimeUnit.SECONDS); + } + catch (Exception ex) { + Assert.fail(ex.getMessage()); + } + + executor.shutdown(); + } +} diff --git a/processing/src/test/java/io/druid/segment/filter/InvalidFilteringTest.java b/processing/src/test/java/io/druid/segment/filter/InvalidFilteringTest.java index 04bee940e9c3..57e73730486e 100644 --- a/processing/src/test/java/io/druid/segment/filter/InvalidFilteringTest.java +++ b/processing/src/test/java/io/druid/segment/filter/InvalidFilteringTest.java @@ -124,16 +124,6 @@ public void testFilterTheUnfilterable() ImmutableList.of("1", "2", "3", "4", "5", "6") ); - assertFilterMatches( - new SelectorDimFilter("dmax", "another string", null), - ImmutableList.of() - ); - - assertFilterMatches( - new SelectorDimFilter("dmax", null, null), - ImmutableList.of("1", "2", "3", "4", "5", "6") - ); - // predicate based matching assertFilterMatches( new InDimFilter("hyperion", Arrays.asList("hello", "world"), null), @@ -144,15 +134,5 @@ public void testFilterTheUnfilterable() new InDimFilter("hyperion", Arrays.asList("hello", "world", null), null), ImmutableList.of("1", "2", "3", "4", "5", "6") ); - - assertFilterMatches( - new InDimFilter("dmax", Arrays.asList("hello", "world"), null), - ImmutableList.of() - ); - - assertFilterMatches( - new InDimFilter("dmax", Arrays.asList("hello", "world", null), null), - ImmutableList.of("1", "2", "3", "4", "5", "6") - ); } } diff --git a/processing/src/test/java/io/druid/segment/filter/LongFilteringTest.java b/processing/src/test/java/io/druid/segment/filter/LongFilteringTest.java index 39771a042736..00d33a28e35d 100644 --- a/processing/src/test/java/io/druid/segment/filter/LongFilteringTest.java +++ b/processing/src/test/java/io/druid/segment/filter/LongFilteringTest.java @@ -155,8 +155,8 @@ public void testLongColumnFiltering() ); // cross the hashing threshold to test hashset implementation, filter on even values - List infilterValues = new ArrayList<>(InDimFilter.LONG_HASHING_THRESHOLD * 2); - for (int i = 0; i < InDimFilter.LONG_HASHING_THRESHOLD * 2; i++) { + List infilterValues = new ArrayList<>(InDimFilter.NUMERIC_HASHING_THRESHOLD * 2); + for (int i = 0; i < InDimFilter.NUMERIC_HASHING_THRESHOLD * 2; i++) { infilterValues.add(String.valueOf(i * 2)); } assertFilterMatches( @@ -303,8 +303,8 @@ public void testMultithreaded() ); // cross the hashing threshold to test hashset implementation, filter on even values - List infilterValues = new ArrayList<>(InDimFilter.LONG_HASHING_THRESHOLD * 2); - for (int i = 0; i < InDimFilter.LONG_HASHING_THRESHOLD * 2; i++) { + List infilterValues = new ArrayList<>(InDimFilter.NUMERIC_HASHING_THRESHOLD * 2); + for (int i = 0; i < InDimFilter.NUMERIC_HASHING_THRESHOLD * 2; i++) { infilterValues.add(String.valueOf(i * 2)); } assertFilterMatchesMultithreaded( diff --git a/processing/src/test/java/io/druid/segment/filter/TimeFilteringTest.java b/processing/src/test/java/io/druid/segment/filter/TimeFilteringTest.java index 253d6c7b3132..d5c34a4669d0 100644 --- a/processing/src/test/java/io/druid/segment/filter/TimeFilteringTest.java +++ b/processing/src/test/java/io/druid/segment/filter/TimeFilteringTest.java @@ -132,8 +132,8 @@ public void testTimeFilterAsLong() ); // cross the hashing threshold to test hashset implementation, filter on even values - List infilterValues = new ArrayList<>(InDimFilter.LONG_HASHING_THRESHOLD * 2); - for (int i = 0; i < InDimFilter.LONG_HASHING_THRESHOLD * 2; i++) { + List infilterValues = new ArrayList<>(InDimFilter.NUMERIC_HASHING_THRESHOLD * 2); + for (int i = 0; i < InDimFilter.NUMERIC_HASHING_THRESHOLD * 2; i++) { infilterValues.add(String.valueOf(i*2)); } assertFilterMatches( diff --git a/processing/src/test/resources/druid.sample.numeric.tsv b/processing/src/test/resources/druid.sample.numeric.tsv new file mode 100644 index 000000000000..a5de2973ece7 --- /dev/null +++ b/processing/src/test/resources/druid.sample.numeric.tsv @@ -0,0 +1,1209 @@ +2011-01-12T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 100.000000 +2011-01-12T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 100.000000 +2011-01-12T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 100.000000 +2011-01-12T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 100.000000 +2011-01-12T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 100.000000 +2011-01-12T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 100.000000 +2011-01-12T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 100.000000 +2011-01-12T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 100.000000 +2011-01-12T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 100.000000 +2011-01-12T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1000.000000 +2011-01-12T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1000.000000 +2011-01-12T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 800.000000 value +2011-01-12T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 800.000000 value +2011-01-13T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 94.874713 +2011-01-13T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 103.629399 +2011-01-13T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 110.087299 +2011-01-13T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 114.947403 +2011-01-13T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 104.465767 +2011-01-13T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 102.851683 +2011-01-13T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 108.863011 +2011-01-13T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 111.356672 +2011-01-13T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 106.236928 +2011-01-13T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1040.945505 +2011-01-13T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1689.012875 +2011-01-13T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 826.060182 value +2011-01-13T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1564.617729 value +2011-01-14T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 86.450372 +2011-01-14T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 102.670409 +2011-01-14T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 109.573474 +2011-01-14T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 94.000432 +2011-01-14T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 97.903068 +2011-01-14T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 101.380760 +2011-01-14T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 104.611784 +2011-01-14T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 114.974216 +2011-01-14T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 112.259958 +2011-01-14T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1049.141912 +2011-01-14T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1073.476545 +2011-01-14T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1006.402111 value +2011-01-14T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 869.643722 value +2011-01-15T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 82.840417 +2011-01-15T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 99.781645 +2011-01-15T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 100.551072 +2011-01-15T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 87.954346 +2011-01-15T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 100.582654 +2011-01-15T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 99.383407 +2011-01-15T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 102.366798 +2011-01-15T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 111.680229 +2011-01-15T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 120.481420 +2011-01-15T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1007.365510 +2011-01-15T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1545.708865 +2011-01-15T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 809.041763 value +2011-01-15T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1458.402661 value +2011-01-16T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 71.315931 +2011-01-16T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 100.591602 +2011-01-16T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 99.007588 +2011-01-16T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 93.085943 +2011-01-16T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 99.863171 +2011-01-16T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 100.192013 +2011-01-16T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 103.348007 +2011-01-16T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 87.280816 +2011-01-16T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 116.779610 +2011-01-16T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1016.965229 +2011-01-16T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1077.612663 +2011-01-16T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 852.437477 value +2011-01-16T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 879.988099 value +2011-01-17T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 105.442374 +2011-01-17T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 105.914315 +2011-01-17T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 99.189052 +2011-01-17T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 110.157325 +2011-01-17T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 103.692852 +2011-01-17T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 103.615039 +2011-01-17T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 106.696362 +2011-01-17T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 89.901887 +2011-01-17T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 134.415281 +2011-01-17T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1075.089574 +2011-01-17T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 953.995422 +2011-01-17T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 950.146770 value +2011-01-17T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 712.774595 value +2011-01-18T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 87.195139 +2011-01-18T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 107.244699 +2011-01-18T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 94.452739 +2011-01-18T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 103.018934 +2011-01-18T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 101.087367 +2011-01-18T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 104.724154 +2011-01-18T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 108.979936 +2011-01-18T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 88.764512 +2011-01-18T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 113.680094 +2011-01-18T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1022.783330 +2011-01-18T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 937.061939 +2011-01-18T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 846.267516 value +2011-01-18T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 682.885525 value +2011-01-19T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 85.681683 +2011-01-19T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 106.700550 +2011-01-19T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 92.314034 +2011-01-19T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 97.620315 +2011-01-19T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 105.087466 +2011-01-19T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 106.127830 +2011-01-19T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 104.568464 +2011-01-19T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 77.316731 +2011-01-19T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 109.772202 +2011-01-19T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1156.744712 +2011-01-19T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 849.877513 +2011-01-19T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1109.874950 value +2011-01-19T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 594.381703 value +2011-01-20T01:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 93.396274 +2011-01-20T01:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 106.367723 +2011-01-20T01:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 90.439020 +2011-01-20T01:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 96.112901 +2011-01-20T01:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 105.669498 +2011-01-20T01:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 105.225158 +2011-01-20T01:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 101.305541 +2011-01-20T01:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 77.759854 +2011-01-20T01:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 140.179069 +2011-01-20T01:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1066.208012 +2011-01-20T01:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 904.340636 +2011-01-20T01:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 870.115926 value +2011-01-20T01:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 677.510973 value +2011-01-22T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 95.235266 +2011-01-22T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 105.282866 +2011-01-22T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 93.681096 +2011-01-22T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 103.527592 +2011-01-22T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 104.184494 +2011-01-22T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 103.399677 +2011-01-22T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 99.284525 +2011-01-22T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 107.627793 +2011-01-22T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 137.109783 +2011-01-22T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1240.525484 +2011-01-22T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1343.232494 +2011-01-22T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1410.278128 value +2011-01-22T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1219.432170 value +2011-01-23T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 100.432710 +2011-01-23T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 107.348157 +2011-01-23T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 92.789692 +2011-01-23T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 96.826443 +2011-01-23T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 103.730730 +2011-01-23T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 106.418686 +2011-01-23T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 101.573522 +2011-01-23T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 110.467875 +2011-01-23T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 128.699746 +2011-01-23T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1088.943083 +2011-01-23T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1349.254415 +2011-01-23T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 979.306038 value +2011-01-23T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1224.501568 value +2011-01-24T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 96.671647 +2011-01-24T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 104.485760 +2011-01-24T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 88.748460 +2011-01-24T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 189.385952 +2011-01-24T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 97.906256 +2011-01-24T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 104.167373 +2011-01-24T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 101.581339 +2011-01-24T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 105.345921 +2011-01-24T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 131.695956 +2011-01-24T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1102.866656 +2011-01-24T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 939.244103 +2011-01-24T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1215.589859 value +2011-01-24T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 716.609179 value +2011-01-25T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 90.111413 +2011-01-25T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 101.624789 +2011-01-25T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 85.974579 +2011-01-25T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 180.575246 +2011-01-25T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 96.594588 +2011-01-25T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 102.907866 +2011-01-25T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 102.512878 +2011-01-25T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 102.044542 +2011-01-25T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 122.077247 +2011-01-25T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1109.875413 +2011-01-25T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 997.994544 +2011-01-25T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1301.023342 value +2011-01-25T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 786.363298 value +2011-01-26T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 84.906466 +2011-01-26T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 105.873942 +2011-01-26T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 84.710523 +2011-01-26T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 159.988606 +2011-01-26T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 96.046584 +2011-01-26T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 105.266058 +2011-01-26T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 101.088903 +2011-01-26T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 105.617702 +2011-01-26T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 122.160681 +2011-01-26T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1037.449471 +2011-01-26T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1686.419659 +2011-01-26T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 989.931541 value +2011-01-26T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1609.096706 value +2011-01-27T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 134.127106 +2011-01-27T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 104.882908 +2011-01-27T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 90.806201 +2011-01-27T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 170.735853 +2011-01-27T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 100.643435 +2011-01-27T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 104.609483 +2011-01-27T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 102.612747 +2011-01-27T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 116.979005 +2011-01-27T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 149.125271 +2011-01-27T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1074.006938 +2011-01-27T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1486.201299 +2011-01-27T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1023.295213 value +2011-01-27T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1367.638074 value +2011-01-28T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 123.006128 +2011-01-28T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 111.641077 +2011-01-28T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 99.681629 +2011-01-28T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 107.788998 +2011-01-28T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 106.075672 +2011-01-28T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 108.106449 +2011-01-28T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 103.822842 +2011-01-28T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 93.869236 +2011-01-28T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 158.739359 +2011-01-28T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1300.302260 +2011-01-28T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1021.334487 +2011-01-28T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1627.598064 value +2011-01-28T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 810.889422 value +2011-01-29T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 127.450345 +2011-01-29T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 100.992147 +2011-01-29T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 103.345166 +2011-01-29T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 114.905745 +2011-01-29T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 106.663538 +2011-01-29T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 101.998823 +2011-01-29T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 104.311418 +2011-01-29T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 109.549035 +2011-01-29T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 138.865014 +2011-01-29T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1314.619452 +2011-01-29T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 792.326066 +2011-01-29T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1685.500085 value +2011-01-29T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 545.990623 value +2011-01-30T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 124.943293 +2011-01-30T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 106.064111 +2011-01-30T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 108.415967 +2011-01-30T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 137.198397 +2011-01-30T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 97.023907 +2011-01-30T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 106.009926 +2011-01-30T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 104.298490 +2011-01-30T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 110.528451 +2011-01-30T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 137.932693 +2011-01-30T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1233.448863 +2011-01-30T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 805.930143 +2011-01-30T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1870.061029 value +2011-01-30T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 555.476028 value +2011-01-31T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 133.740047 +2011-01-31T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 103.492964 +2011-01-31T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 104.548387 +2011-01-31T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 124.171944 +2011-01-31T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 97.017604 +2011-01-31T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 103.832040 +2011-01-31T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 103.021032 +2011-01-31T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 85.125795 +2011-01-31T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 155.744951 +2011-01-31T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1184.920651 +2011-01-31T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1127.231000 +2011-01-31T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1643.340851 value +2011-01-31T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 943.497198 value +2011-02-01T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 132.123776 +2011-02-01T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 103.890175 +2011-02-01T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 103.652865 +2011-02-01T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 113.896016 +2011-02-01T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 98.909356 +2011-02-01T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 104.383662 +2011-02-01T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 102.480377 +2011-02-01T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 83.931272 +2011-02-01T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 134.014606 +2011-02-01T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1203.465595 +2011-02-01T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1100.904846 +2011-02-01T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1667.497773 value +2011-02-01T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 913.561076 value +2011-02-02T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 113.492245 +2011-02-02T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 104.963233 +2011-02-02T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 112.042996 +2011-02-02T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 102.281859 +2011-02-02T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 97.711139 +2011-02-02T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 105.578807 +2011-02-02T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 102.152053 +2011-02-02T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 96.706279 +2011-02-02T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 123.170962 +2011-02-02T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1097.211164 +2011-02-02T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1410.792943 +2011-02-02T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1218.561908 value +2011-02-02T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1273.707453 value +2011-02-03T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 85.770241 +2011-02-03T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 98.877952 +2011-02-03T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 106.425780 +2011-02-03T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 85.069784 +2011-02-03T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 100.559287 +2011-02-03T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 100.976362 +2011-02-03T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 102.349315 +2011-02-03T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 92.326431 +2011-02-03T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 134.140377 +2011-02-03T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1033.401241 +2011-02-03T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1283.166055 +2011-02-03T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 888.705280 value +2011-02-03T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1113.114125 value +2011-02-04T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 89.182906 +2011-02-04T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 106.888769 +2011-02-04T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 112.471918 +2011-02-04T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 99.837572 +2011-02-04T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 100.918373 +2011-02-04T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 106.050728 +2011-02-04T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 103.103922 +2011-02-04T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 93.973465 +2011-02-04T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 113.716758 +2011-02-04T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1025.633340 +2011-02-04T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1331.860983 +2011-02-04T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 864.568891 value +2011-02-04T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1308.582051 value +2011-02-05T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 93.001571 +2011-02-05T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 111.394244 +2011-02-05T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 117.030289 +2011-02-05T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 94.312960 +2011-02-05T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 105.660538 +2011-02-05T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 107.929804 +2011-02-05T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 100.646747 +2011-02-05T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 90.732978 +2011-02-05T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 114.723682 +2011-02-05T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1039.500513 +2011-02-05T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1332.468373 +2011-02-05T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 785.078869 value +2011-02-05T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1363.614929 value +2011-02-06T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 134.462521 +2011-02-06T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 110.897359 +2011-02-06T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 119.608310 +2011-02-06T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 93.585758 +2011-02-06T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 101.847544 +2011-02-06T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 110.053071 +2011-02-06T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 102.770913 +2011-02-06T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 93.620739 +2011-02-06T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 121.270562 +2011-02-06T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1011.205470 +2011-02-06T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1029.995236 +2011-02-06T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 787.125330 value +2011-02-06T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 826.039207 value +2011-02-07T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 130.194219 +2011-02-07T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 98.815847 +2011-02-07T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 112.924874 +2011-02-07T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 97.480779 +2011-02-07T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 97.434318 +2011-02-07T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 100.706057 +2011-02-07T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 102.705243 +2011-02-07T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 83.902353 +2011-02-07T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 115.246714 +2011-02-07T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1047.212887 +2011-02-07T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1057.079944 +2011-02-07T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1107.243787 value +2011-02-07T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 872.625669 value +2011-02-08T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 126.243536 +2011-02-08T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 93.190129 +2011-02-08T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 106.969799 +2011-02-08T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 97.432302 +2011-02-08T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 96.790543 +2011-02-08T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 97.085047 +2011-02-08T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 103.308255 +2011-02-08T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 75.735586 +2011-02-08T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 111.040150 +2011-02-08T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1064.972638 +2011-02-08T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1082.727640 +2011-02-08T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1188.369265 value +2011-02-08T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 911.956790 value +2011-02-09T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 129.221792 +2011-02-09T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 115.548444 +2011-02-09T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 111.729360 +2011-02-09T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 97.071703 +2011-02-09T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 96.478571 +2011-02-09T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 113.554588 +2011-02-09T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 105.498315 +2011-02-09T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 83.742151 +2011-02-09T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 112.646238 +2011-02-09T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 971.050764 +2011-02-09T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1320.638308 +2011-02-09T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 794.098825 value +2011-02-09T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1299.093262 value +2011-02-10T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 115.461691 +2011-02-10T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 118.062165 +2011-02-10T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 117.629065 +2011-02-10T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 97.235999 +2011-02-10T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 95.698374 +2011-02-10T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 115.824976 +2011-02-10T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 105.708103 +2011-02-10T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 91.750911 +2011-02-10T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 105.557241 +2011-02-10T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1070.165582 +2011-02-10T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1089.647884 +2011-02-10T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1212.928303 value +2011-02-10T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 901.327272 value +2011-02-11T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 129.187009 +2011-02-11T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 114.637486 +2011-02-11T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 114.960877 +2011-02-11T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 96.617339 +2011-02-11T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 100.111873 +2011-02-11T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 112.571724 +2011-02-11T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 105.672256 +2011-02-11T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 87.904114 +2011-02-11T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 102.864842 +2011-02-11T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 980.386611 +2011-02-11T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1179.695901 +2011-02-11T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 723.514254 value +2011-02-11T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1061.973330 value +2011-02-12T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 130.104979 +2011-02-12T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 115.758445 +2011-02-12T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 115.225386 +2011-02-12T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 96.457082 +2011-02-12T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 101.515571 +2011-02-12T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 114.877503 +2011-02-12T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 108.637522 +2011-02-12T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 88.142774 +2011-02-12T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 102.850696 +2011-02-12T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 959.236186 +2011-02-12T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1092.416967 +2011-02-12T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 602.979544 value +2011-02-12T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 879.406101 value +2011-02-13T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 119.490316 +2011-02-13T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 118.841176 +2011-02-13T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 119.907266 +2011-02-13T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 98.607490 +2011-02-13T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 100.905238 +2011-02-13T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 117.965974 +2011-02-13T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 112.514409 +2011-02-13T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 87.820236 +2011-02-13T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 106.033416 +2011-02-13T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 987.067381 +2011-02-13T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1103.458199 +2011-02-13T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 724.262526 value +2011-02-13T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 862.931321 value +2011-02-14T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 119.323168 +2011-02-14T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 115.628202 +2011-02-14T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 123.098262 +2011-02-14T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 103.008650 +2011-02-14T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 101.645725 +2011-02-14T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 117.110451 +2011-02-14T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 121.060464 +2011-02-14T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 73.717033 +2011-02-14T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 107.663239 +2011-02-14T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1091.223197 +2011-02-14T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1199.607472 +2011-02-14T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1133.135123 value +2011-02-14T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 948.657939 value +2011-02-15T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 123.485071 +2011-02-15T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 115.527003 +2011-02-15T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 121.563912 +2011-02-15T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 103.519393 +2011-02-15T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 105.269599 +2011-02-15T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 117.138956 +2011-02-15T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 121.411398 +2011-02-15T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 79.700998 +2011-02-15T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 108.428302 +2011-02-15T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1044.384300 +2011-02-15T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1183.240825 +2011-02-15T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 807.601674 value +2011-02-15T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 914.525048 value +2011-02-16T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 133.726576 +2011-02-16T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 116.432276 +2011-02-16T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 130.717934 +2011-02-16T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 105.762627 +2011-02-16T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 108.338531 +2011-02-16T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 117.334381 +2011-02-16T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 123.270869 +2011-02-16T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 98.918664 +2011-02-16T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 113.643571 +2011-02-16T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1289.097304 +2011-02-16T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1360.032423 +2011-02-16T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1553.348548 value +2011-02-16T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1208.456692 value +2011-02-17T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 147.942017 +2011-02-17T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 125.032692 +2011-02-17T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 126.982673 +2011-02-17T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 101.092779 +2011-02-17T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 103.940963 +2011-02-17T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 121.872569 +2011-02-17T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 120.050545 +2011-02-17T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 91.969195 +2011-02-17T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 112.150745 +2011-02-17T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 993.591221 +2011-02-17T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1021.071173 +2011-02-17T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 679.619354 value +2011-02-17T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 645.177645 value +2011-02-18T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 151.053296 +2011-02-18T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 127.611947 +2011-02-18T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 128.063524 +2011-02-18T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 100.849247 +2011-02-18T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 101.960196 +2011-02-18T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 124.513018 +2011-02-18T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 122.546253 +2011-02-18T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 92.174432 +2011-02-18T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 120.151389 +2011-02-18T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1105.383465 +2011-02-18T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1601.829436 +2011-02-18T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1120.088751 value +2011-02-18T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1649.533329 value +2011-02-19T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 163.351690 +2011-02-19T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 123.447481 +2011-02-19T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 133.726878 +2011-02-19T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 108.822138 +2011-02-19T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 102.919452 +2011-02-19T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 119.371511 +2011-02-19T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 125.052129 +2011-02-19T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 91.547944 +2011-02-19T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 121.733400 +2011-02-19T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1200.527201 +2011-02-19T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1600.723226 +2011-02-19T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1433.398801 value +2011-02-19T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1598.179271 value +2011-02-20T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 157.483005 +2011-02-20T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 129.409606 +2011-02-20T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 176.323916 +2011-02-20T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 109.790712 +2011-02-20T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 127.819268 +2011-02-20T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 122.082375 +2011-02-20T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 132.719065 +2011-02-20T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 105.985049 +2011-02-20T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 124.637709 +2011-02-20T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1317.458323 +2011-02-20T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1304.326111 +2011-02-20T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1192.563067 value +2011-02-20T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1022.854576 value +2011-02-21T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 155.632898 +2011-02-21T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 132.231346 +2011-02-21T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 193.787574 +2011-02-21T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 111.386745 +2011-02-21T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 154.627912 +2011-02-21T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 126.995117 +2011-02-21T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 138.092468 +2011-02-21T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 119.850150 +2011-02-21T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 119.739112 +2011-02-21T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1544.108134 +2011-02-21T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1488.737765 +2011-02-21T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1228.502469 value +2011-02-21T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1298.415763 value +2011-02-22T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 149.171056 +2011-02-22T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 124.524992 +2011-02-22T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 139.557139 +2011-02-22T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 108.370907 +2011-02-22T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 111.394542 +2011-02-22T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 122.510640 +2011-02-22T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 126.596847 +2011-02-22T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 86.333235 +2011-02-22T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 115.915849 +2011-02-22T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1224.827108 +2011-02-22T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1421.648704 +2011-02-22T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1345.964309 value +2011-02-22T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1291.897942 value +2011-02-23T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 165.273009 +2011-02-23T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 127.199815 +2011-02-23T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 145.588115 +2011-02-23T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 107.896489 +2011-02-23T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 113.141185 +2011-02-23T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 122.404192 +2011-02-23T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 124.305608 +2011-02-23T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 91.191071 +2011-02-23T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 123.961542 +2011-02-23T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1251.906228 +2011-02-23T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1414.619004 +2011-02-23T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1306.495696 value +2011-02-23T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1287.766687 value +2011-02-24T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 168.988478 +2011-02-24T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 123.553981 +2011-02-24T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 139.978575 +2011-02-24T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 104.951315 +2011-02-24T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 113.621184 +2011-02-24T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 118.862342 +2011-02-24T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 119.772575 +2011-02-24T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 91.962584 +2011-02-24T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 118.270052 +2011-02-24T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1350.175381 +2011-02-24T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 970.728273 +2011-02-24T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1631.584352 value +2011-02-24T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 578.795979 value +2011-02-25T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 172.335540 +2011-02-25T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 107.539869 +2011-02-25T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 140.941317 +2011-02-25T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 99.698015 +2011-02-25T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 99.460461 +2011-02-25T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 106.827630 +2011-02-25T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 115.932803 +2011-02-25T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 82.350556 +2011-02-25T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 120.124862 +2011-02-25T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1057.427269 +2011-02-25T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1073.967314 +2011-02-25T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1017.573185 value +2011-02-25T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 782.013486 value +2011-02-26T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 225.243186 +2011-02-26T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 103.643952 +2011-02-26T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 138.924835 +2011-02-26T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 103.667031 +2011-02-26T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 98.314744 +2011-02-26T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 105.352891 +2011-02-26T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 118.896845 +2011-02-26T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 83.099365 +2011-02-26T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 131.310541 +2011-02-26T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 996.433708 +2011-02-26T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1743.921750 +2011-02-26T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 829.916235 value +2011-02-26T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1862.737933 value +2011-02-27T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 277.273533 +2011-02-27T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 96.864384 +2011-02-27T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 136.394846 +2011-02-27T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 114.634278 +2011-02-27T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 103.226967 +2011-02-27T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 99.158839 +2011-02-27T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 121.929932 +2011-02-27T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 78.727950 +2011-02-27T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 136.163414 +2011-02-27T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1044.562903 +2011-02-27T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1474.591017 +2011-02-27T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 873.306547 value +2011-02-27T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1427.016724 value +2011-02-28T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 122.258195 +2011-02-28T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 97.218943 +2011-02-28T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 141.261324 +2011-02-28T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 112.528286 +2011-02-28T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 102.185098 +2011-02-28T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 99.505465 +2011-02-28T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 121.786785 +2011-02-28T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 72.163651 +2011-02-28T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 114.284569 +2011-02-28T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1159.278766 +2011-02-28T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1292.542896 +2011-02-28T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1430.257348 value +2011-02-28T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1101.918270 value +2011-03-01T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 153.059937 +2011-03-01T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 99.070796 +2011-03-01T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 143.424672 +2011-03-01T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 114.700932 +2011-03-01T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 105.453024 +2011-03-01T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 99.772347 +2011-03-01T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 123.251814 +2011-03-01T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 72.792970 +2011-03-01T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 116.975408 +2011-03-01T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1124.201419 +2011-03-01T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1243.354010 +2011-03-01T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1166.141121 value +2011-03-01T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1004.940887 value +2011-03-02T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 174.890520 +2011-03-02T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 98.432014 +2011-03-02T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 147.117434 +2011-03-02T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 112.968782 +2011-03-02T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 109.239196 +2011-03-02T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 100.600391 +2011-03-02T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 120.212473 +2011-03-02T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 82.823988 +2011-03-02T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 116.460744 +2011-03-02T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1051.808940 +2011-03-02T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1143.078414 +2011-03-02T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 740.183720 value +2011-03-02T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 865.777900 value +2011-03-03T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 127.994476 +2011-03-03T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 92.537499 +2011-03-03T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 140.215411 +2011-03-03T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 108.914095 +2011-03-03T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 108.784646 +2011-03-03T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 96.031371 +2011-03-03T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 115.393493 +2011-03-03T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 75.977564 +2011-03-03T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 114.188310 +2011-03-03T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1097.490771 +2011-03-03T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1010.370296 +2011-03-03T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 901.307577 value +2011-03-03T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 691.958920 value +2011-03-04T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 119.851231 +2011-03-04T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 93.634505 +2011-03-04T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 132.832331 +2011-03-04T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 110.018472 +2011-03-04T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 107.285615 +2011-03-04T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 97.535226 +2011-03-04T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 113.883056 +2011-03-04T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 81.131208 +2011-03-04T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 109.607245 +2011-03-04T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1326.829155 +2011-03-04T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1179.803776 +2011-03-04T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1674.331703 value +2011-03-04T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 975.577927 value +2011-03-05T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 136.941770 +2011-03-05T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 97.942645 +2011-03-05T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 145.393016 +2011-03-05T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 108.394611 +2011-03-05T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 112.522435 +2011-03-05T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 102.486832 +2011-03-05T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 114.691277 +2011-03-05T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 81.105110 +2011-03-05T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 117.904527 +2011-03-05T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1281.601175 +2011-03-05T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 994.731237 +2011-03-05T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1360.694785 value +2011-03-05T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 755.899363 value +2011-03-06T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 129.531062 +2011-03-06T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 99.508679 +2011-03-06T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 144.925734 +2011-03-06T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 113.069662 +2011-03-06T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 113.035167 +2011-03-06T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 102.536839 +2011-03-06T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 115.956859 +2011-03-06T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 81.612269 +2011-03-06T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 120.953163 +2011-03-06T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1081.650406 +2011-03-06T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1103.239788 +2011-03-06T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 771.348460 value +2011-03-06T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 869.308360 value +2011-03-07T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 111.909348 +2011-03-07T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 105.575929 +2011-03-07T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 150.452695 +2011-03-07T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 118.024245 +2011-03-07T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 111.106693 +2011-03-07T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 107.220218 +2011-03-07T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 121.582721 +2011-03-07T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 68.699125 +2011-03-07T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 106.884238 +2011-03-07T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1177.858403 +2011-03-07T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1152.547767 +2011-03-07T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1095.637520 value +2011-03-07T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 906.373797 value +2011-03-08T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 109.764955 +2011-03-08T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 98.972716 +2011-03-08T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 143.214331 +2011-03-08T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 119.777621 +2011-03-08T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 107.465492 +2011-03-08T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 101.652185 +2011-03-08T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 122.692722 +2011-03-08T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 70.866726 +2011-03-08T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 111.704071 +2011-03-08T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1117.953961 +2011-03-08T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1084.332554 +2011-03-08T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 988.893782 value +2011-03-08T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 835.762631 value +2011-03-09T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 139.260950 +2011-03-09T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 110.873407 +2011-03-09T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 138.466933 +2011-03-09T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 115.013313 +2011-03-09T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 105.613469 +2011-03-09T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 112.407868 +2011-03-09T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 121.220772 +2011-03-09T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 82.426362 +2011-03-09T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 107.998334 +2011-03-09T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1029.802500 +2011-03-09T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1121.385333 +2011-03-09T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 776.702940 value +2011-03-09T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 875.683406 value +2011-03-10T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 148.809150 +2011-03-10T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 105.214709 +2011-03-10T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 134.212714 +2011-03-10T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 114.717338 +2011-03-10T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 119.613508 +2011-03-10T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 107.127962 +2011-03-10T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 118.864028 +2011-03-10T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 79.793836 +2011-03-10T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 107.706257 +2011-03-10T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1244.849915 +2011-03-10T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1077.279402 +2011-03-10T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1070.836247 value +2011-03-10T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 835.461226 value +2011-03-11T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 135.820968 +2011-03-11T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 106.898536 +2011-03-11T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 135.038992 +2011-03-11T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 112.856230 +2011-03-11T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 120.497687 +2011-03-11T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 108.135811 +2011-03-11T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 118.298350 +2011-03-11T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 67.731170 +2011-03-11T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 108.186877 +2011-03-11T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1098.543170 +2011-03-11T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 998.650727 +2011-03-11T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 715.516125 value +2011-03-11T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 755.646538 value +2011-03-12T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 155.728048 +2011-03-12T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 113.493460 +2011-03-12T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 132.687079 +2011-03-12T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 112.554597 +2011-03-12T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 114.681603 +2011-03-12T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 115.572940 +2011-03-12T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 118.574721 +2011-03-12T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 74.394926 +2011-03-12T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 109.384493 +2011-03-12T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1088.807596 +2011-03-12T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1008.745525 +2011-03-12T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 771.100508 value +2011-03-12T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 764.508070 value +2011-03-13T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 149.637715 +2011-03-13T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 113.760384 +2011-03-13T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 120.113921 +2011-03-13T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 120.760130 +2011-03-13T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 103.227522 +2011-03-13T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 114.814070 +2011-03-13T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 120.620862 +2011-03-13T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 70.126017 +2011-03-13T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 108.579283 +2011-03-13T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 998.753955 +2011-03-13T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1129.723252 +2011-03-13T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 736.409261 value +2011-03-13T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 884.837267 value +2011-03-14T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 153.191744 +2011-03-14T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 109.461442 +2011-03-14T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 123.248581 +2011-03-14T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 120.487244 +2011-03-14T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 104.716583 +2011-03-14T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 111.688901 +2011-03-14T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 122.275869 +2011-03-14T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 59.021022 +2011-03-14T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 104.724023 +2011-03-14T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1075.243024 +2011-03-14T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1141.588400 +2011-03-14T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 918.722840 value +2011-03-14T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 893.985017 value +2011-03-15T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 145.963558 +2011-03-15T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 109.382273 +2011-03-15T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 121.386341 +2011-03-15T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 119.250945 +2011-03-15T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 103.583295 +2011-03-15T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 112.354294 +2011-03-15T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 122.038585 +2011-03-15T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 59.266595 +2011-03-15T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 103.134338 +2011-03-15T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1037.381049 +2011-03-15T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1099.197263 +2011-03-15T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 832.874861 value +2011-03-15T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 850.995007 value +2011-03-16T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 147.471464 +2011-03-16T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 110.565004 +2011-03-16T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 110.070846 +2011-03-16T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 115.750963 +2011-03-16T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 99.137980 +2011-03-16T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 112.577264 +2011-03-16T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 120.455865 +2011-03-16T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 69.329723 +2011-03-16T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 107.903885 +2011-03-16T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 981.577244 +2011-03-16T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1092.942008 +2011-03-16T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 767.973326 value +2011-03-16T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 848.339888 value +2011-03-17T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 148.905410 +2011-03-17T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 113.501786 +2011-03-17T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 109.666402 +2011-03-17T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 114.540037 +2011-03-17T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 105.996125 +2011-03-17T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 116.816958 +2011-03-17T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 122.740143 +2011-03-17T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 69.258523 +2011-03-17T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 110.797348 +2011-03-17T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1072.239320 +2011-03-17T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1154.415689 +2011-03-17T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 878.683776 value +2011-03-17T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 906.101957 value +2011-03-18T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 180.343171 +2011-03-18T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 110.037579 +2011-03-18T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 130.260926 +2011-03-18T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 113.490115 +2011-03-18T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 111.540639 +2011-03-18T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 113.238556 +2011-03-18T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 119.629977 +2011-03-18T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 68.573162 +2011-03-18T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 114.564808 +2011-03-18T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1311.178603 +2011-03-18T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1176.605164 +2011-03-18T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1403.830217 value +2011-03-18T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 936.429632 value +2011-03-19T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 177.514270 +2011-03-19T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 109.788875 +2011-03-19T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 134.147573 +2011-03-19T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 117.197085 +2011-03-19T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 112.999693 +2011-03-19T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 112.236468 +2011-03-19T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 120.638001 +2011-03-19T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 72.369471 +2011-03-19T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 115.384807 +2011-03-19T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1107.220174 +2011-03-19T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1102.698977 +2011-03-19T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 846.288386 value +2011-03-19T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 856.490089 value +2011-03-20T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 178.454262 +2011-03-20T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 123.507497 +2011-03-20T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 157.749330 +2011-03-20T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 117.851058 +2011-03-20T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 128.274705 +2011-03-20T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 125.496367 +2011-03-20T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 129.519442 +2011-03-20T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 85.013155 +2011-03-20T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 128.705337 +2011-03-20T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1285.090048 +2011-03-20T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1217.547439 +2011-03-20T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1032.257527 value +2011-03-20T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 954.754185 value +2011-03-21T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 182.035296 +2011-03-21T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 124.411632 +2011-03-21T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 157.153730 +2011-03-21T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 122.462424 +2011-03-21T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 128.149976 +2011-03-21T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 125.243882 +2011-03-21T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 131.807919 +2011-03-21T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 75.936640 +2011-03-21T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 123.653645 +2011-03-21T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1283.957016 +2011-03-21T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1178.830164 +2011-03-21T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1031.990042 value +2011-03-21T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 907.021565 value +2011-03-22T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 177.460613 +2011-03-22T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 121.270611 +2011-03-22T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 151.407583 +2011-03-22T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 124.400780 +2011-03-22T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 126.415884 +2011-03-22T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 124.970533 +2011-03-22T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 133.124963 +2011-03-22T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 79.948248 +2011-03-22T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 122.357549 +2011-03-22T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1301.778098 +2011-03-22T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1119.247202 +2011-03-22T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1110.788895 value +2011-03-22T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 843.952139 value +2011-03-23T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 154.019632 +2011-03-23T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 126.764513 +2011-03-23T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 158.592715 +2011-03-23T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 118.972310 +2011-03-23T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 126.672392 +2011-03-23T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 129.864384 +2011-03-23T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 131.786598 +2011-03-23T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 102.603152 +2011-03-23T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 123.754240 +2011-03-23T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1403.338838 +2011-03-23T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1156.601892 +2011-03-23T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1362.650586 value +2011-03-23T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 884.801502 value +2011-03-24T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 135.569784 +2011-03-24T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 123.895027 +2011-03-24T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 156.650862 +2011-03-24T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 119.777998 +2011-03-24T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 115.945757 +2011-03-24T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 125.044877 +2011-03-24T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 129.023978 +2011-03-24T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 88.521042 +2011-03-24T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 124.062061 +2011-03-24T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1429.580257 +2011-03-24T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1137.842315 +2011-03-24T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1684.268799 value +2011-03-24T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 873.652030 value +2011-03-25T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 140.577121 +2011-03-25T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 125.766952 +2011-03-25T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 157.432108 +2011-03-25T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 119.400783 +2011-03-25T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 114.706960 +2011-03-25T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 126.057960 +2011-03-25T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 128.943094 +2011-03-25T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 89.408906 +2011-03-25T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 136.136598 +2011-03-25T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1124.935193 +2011-03-25T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1256.499779 +2011-03-25T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 855.717712 value +2011-03-25T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 996.564152 value +2011-03-26T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 148.957194 +2011-03-26T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 130.824022 +2011-03-26T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 162.815450 +2011-03-26T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 118.463523 +2011-03-26T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 117.449116 +2011-03-26T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 142.972964 +2011-03-26T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 139.214665 +2011-03-26T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 101.686196 +2011-03-26T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 138.663182 +2011-03-26T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1217.877395 +2011-03-26T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1247.890809 +2011-03-26T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1061.678577 value +2011-03-26T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 962.235801 value +2011-03-27T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 144.056669 +2011-03-27T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 135.183271 +2011-03-27T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 163.161361 +2011-03-27T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 130.599006 +2011-03-27T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 114.952545 +2011-03-27T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 139.294248 +2011-03-27T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 142.430177 +2011-03-27T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 108.489598 +2011-03-27T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 141.820068 +2011-03-27T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1185.709973 +2011-03-27T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1345.781728 +2011-03-27T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1019.898509 value +2011-03-27T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1056.419292 value +2011-03-28T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 156.155294 +2011-03-28T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 124.336139 +2011-03-28T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 163.100154 +2011-03-28T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 131.191818 +2011-03-28T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 115.709767 +2011-03-28T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 127.403089 +2011-03-28T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 140.941296 +2011-03-28T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 103.578536 +2011-03-28T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 130.880788 +2011-03-28T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1250.166788 +2011-03-28T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1390.754050 +2011-03-28T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1198.723103 value +2011-03-28T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1108.136072 value +2011-03-29T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 134.084672 +2011-03-29T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 117.626804 +2011-03-29T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 158.073319 +2011-03-29T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 128.074393 +2011-03-29T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 109.678515 +2011-03-29T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 122.620188 +2011-03-29T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 144.446039 +2011-03-29T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 91.604463 +2011-03-29T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 132.477651 +2011-03-29T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1224.116225 +2011-03-29T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1361.080245 +2011-03-29T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1289.009485 value +2011-03-29T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1069.431801 value +2011-03-30T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 135.942820 +2011-03-30T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 120.283054 +2011-03-30T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 167.960620 +2011-03-30T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 121.215839 +2011-03-30T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 116.587746 +2011-03-30T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 122.612114 +2011-03-30T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 138.827311 +2011-03-30T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 93.331887 +2011-03-30T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 128.645571 +2011-03-30T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1190.933753 +2011-03-30T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1310.797070 +2011-03-30T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1001.134025 value +2011-03-30T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1030.499562 value +2011-03-31T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 151.752485 +2011-03-31T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 124.414321 +2011-03-31T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 175.778647 +2011-03-31T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 120.607382 +2011-03-31T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 117.060598 +2011-03-31T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 125.243245 +2011-03-31T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 150.247713 +2011-03-31T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 93.390841 +2011-03-31T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 125.839686 +2011-03-31T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1466.209327 +2011-03-31T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1366.447617 +2011-03-31T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1734.274909 value +2011-03-31T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1063.201156 value +2011-04-01T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 135.885094 +2011-04-01T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 118.570340 +2011-04-01T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 158.747224 +2011-04-01T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 120.134704 +2011-04-01T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 109.705815 +2011-04-01T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 121.583581 +2011-04-01T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 144.507368 +2011-04-01T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 78.622547 +2011-04-01T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 119.922742 +2011-04-01T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1314.839715 +2011-04-01T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1522.043733 +2011-04-01T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1447.341160 value +2011-04-01T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1234.247546 value +2011-04-02T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 147.425935 +2011-04-02T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 112.987027 +2011-04-02T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 166.016049 +2011-04-02T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 113.446008 +2011-04-02T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 110.931934 +2011-04-02T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 114.290141 +2011-04-02T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 135.301506 +2011-04-02T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 97.387433 +2011-04-02T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 126.411364 +2011-04-02T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1193.556278 +2011-04-02T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1321.375057 +2011-04-02T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1144.342401 value +2011-04-02T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1049.738585 value +2011-04-03T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 122.971856 +2011-04-03T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 105.735462 +2011-04-03T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 153.927965 +2011-04-03T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 103.532351 +2011-04-03T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 107.047773 +2011-04-03T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 107.919674 +2011-04-03T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 122.141707 +2011-04-03T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 80.861743 +2011-04-03T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 117.247070 +2011-04-03T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1055.783661 +2011-04-03T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1021.638673 +2011-04-03T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 811.991286 value +2011-04-03T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 768.423077 value +2011-04-04T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 110.919829 +2011-04-04T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 107.613577 +2011-04-04T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 146.729242 +2011-04-04T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 105.375351 +2011-04-04T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 110.573670 +2011-04-04T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 114.382255 +2011-04-04T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 125.285894 +2011-04-04T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 72.668430 +2011-04-04T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 117.703023 +2011-04-04T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1197.008423 +2011-04-04T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1131.531986 +2011-04-04T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1151.069173 value +2011-04-04T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 877.079396 value +2011-04-05T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 113.318712 +2011-04-05T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 105.615563 +2011-04-05T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 141.713507 +2011-04-05T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 106.207931 +2011-04-05T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 109.890586 +2011-04-05T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 110.012987 +2011-04-05T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 124.478408 +2011-04-05T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 86.683603 +2011-04-05T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 117.051694 +2011-04-05T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1192.144303 +2011-04-05T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1154.289559 +2011-04-05T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1146.423036 value +2011-04-05T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 902.615706 value +2011-04-06T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 115.334018 +2011-04-06T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 109.700256 +2011-04-06T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 147.553562 +2011-04-06T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 100.775597 +2011-04-06T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 108.659345 +2011-04-06T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 113.408308 +2011-04-06T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 121.079585 +2011-04-06T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 92.336403 +2011-04-06T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 112.921482 +2011-04-06T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1272.677122 +2011-04-06T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1141.514652 +2011-04-06T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1355.843374 value +2011-04-06T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 897.393445 value +2011-04-07T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 117.508062 +2011-04-07T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 115.418054 +2011-04-07T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 141.565031 +2011-04-07T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 95.562446 +2011-04-07T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 102.933171 +2011-04-07T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 122.696621 +2011-04-07T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 119.763135 +2011-04-07T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 84.357701 +2011-04-07T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 117.432760 +2011-04-07T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1190.896088 +2011-04-07T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1009.363132 +2011-04-07T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1260.143027 value +2011-04-07T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 762.862488 value +2011-04-08T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 120.973860 +2011-04-08T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 116.248860 +2011-04-08T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 146.830618 +2011-04-08T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 96.226609 +2011-04-08T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 95.447888 +2011-04-08T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 120.709912 +2011-04-08T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 120.309688 +2011-04-08T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 94.631354 +2011-04-08T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 122.743898 +2011-04-08T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1006.913816 +2011-04-08T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1032.599837 +2011-04-08T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 935.168026 value +2011-04-08T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 782.107861 value +2011-04-09T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 116.080323 +2011-04-09T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 116.060759 +2011-04-09T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 158.682525 +2011-04-09T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 95.509796 +2011-04-09T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 110.007248 +2011-04-09T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 121.905685 +2011-04-09T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 126.369367 +2011-04-09T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 92.905070 +2011-04-09T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 123.791320 +2011-04-09T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1137.385764 +2011-04-09T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1030.075553 +2011-04-09T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 996.205369 value +2011-04-09T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 767.692135 value +2011-04-10T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 113.221448 +2011-04-10T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 95.570457 +2011-04-10T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 131.766616 +2011-04-10T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 99.950855 +2011-04-10T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 91.470524 +2011-04-10T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 99.393076 +2011-04-10T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 123.207579 +2011-04-10T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 84.898691 +2011-04-10T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 114.353962 +2011-04-10T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1005.253077 +2011-04-10T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1030.094757 +2011-04-10T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1031.741509 value +2011-04-10T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 775.965555 value +2011-04-11T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 130.165796 +2011-04-11T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 107.765101 +2011-04-11T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 142.751726 +2011-04-11T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 104.847285 +2011-04-11T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 95.272956 +2011-04-11T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 106.229286 +2011-04-11T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 126.823859 +2011-04-11T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 89.250155 +2011-04-11T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 122.049678 +2011-04-11T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1112.794811 +2011-04-11T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1113.357530 +2011-04-11T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1374.968412 value +2011-04-11T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 853.163039 value +2011-04-12T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 122.386348 +2011-04-12T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 106.380995 +2011-04-12T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 141.932300 +2011-04-12T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 103.142372 +2011-04-12T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 97.340631 +2011-04-12T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 105.381244 +2011-04-12T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 125.189098 +2011-04-12T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 90.533391 +2011-04-12T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 122.128172 +2011-04-12T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1153.974725 +2011-04-12T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1069.640880 +2011-04-12T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1456.611830 value +2011-04-12T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 811.925240 value +2011-04-13T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 122.688340 +2011-04-13T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 105.739623 +2011-04-13T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 136.983407 +2011-04-13T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 100.860813 +2011-04-13T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 94.839191 +2011-04-13T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 105.261296 +2011-04-13T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 119.836611 +2011-04-13T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 91.972558 +2011-04-13T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 120.145572 +2011-04-13T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1016.137449 +2011-04-13T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 994.902292 +2011-04-13T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 989.032799 value +2011-04-13T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 744.744657 value +2011-04-14T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 111.179339 +2011-04-14T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 101.984377 +2011-04-14T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 133.606430 +2011-04-14T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 99.738319 +2011-04-14T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 91.270553 +2011-04-14T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 101.251756 +2011-04-14T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 118.285128 +2011-04-14T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 84.951300 +2011-04-14T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 119.768525 +2011-04-14T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1032.154263 +2011-04-14T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 999.586450 +2011-04-14T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1166.401205 value +2011-04-14T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 753.104985 value +2011-04-15T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 106.793700 +2011-04-15T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 94.469747 +2011-04-15T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 135.109191 +2011-04-15T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 99.596909 +2011-04-15T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 92.782760 +2011-04-15T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 97.859766 +2011-04-15T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 120.508160 +2011-04-15T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 89.646236 +2011-04-15T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 120.290348 +2011-04-15T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 994.752744 +2011-04-15T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1029.056992 +2011-04-15T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 962.731172 value +2011-04-15T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 780.271977 value diff --git a/processing/src/test/resources/druid.sample.numeric.tsv.bottom b/processing/src/test/resources/druid.sample.numeric.tsv.bottom new file mode 100644 index 000000000000..3875bb657cca --- /dev/null +++ b/processing/src/test/resources/druid.sample.numeric.tsv.bottom @@ -0,0 +1,732 @@ +2011-01-12T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 100.000000 +2011-01-12T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 100.000000 +2011-01-12T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 100.000000 +2011-01-12T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 100.000000 +2011-01-12T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1000.000000 +2011-01-12T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 800.000000 value +2011-01-13T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 94.874713 +2011-01-13T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 110.087299 +2011-01-13T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 104.465767 +2011-01-13T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 108.863011 +2011-01-13T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 106.236928 +2011-01-13T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1689.012875 +2011-01-13T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1564.617729 value +2011-01-14T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 102.670409 +2011-01-14T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 94.000432 +2011-01-14T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 101.380760 +2011-01-14T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 114.974216 +2011-01-14T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1049.141912 +2011-01-14T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1006.402111 value +2011-01-15T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 82.840417 +2011-01-15T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 100.551072 +2011-01-15T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 100.582654 +2011-01-15T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 102.366798 +2011-01-15T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 120.481420 +2011-01-15T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1545.708865 +2011-01-15T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1458.402661 value +2011-01-16T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 100.591602 +2011-01-16T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 93.085943 +2011-01-16T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 100.192013 +2011-01-16T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 87.280816 +2011-01-16T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1016.965229 +2011-01-16T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 852.437477 value +2011-01-17T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 105.442374 +2011-01-17T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 99.189052 +2011-01-17T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 103.692852 +2011-01-17T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 106.696362 +2011-01-17T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 134.415281 +2011-01-17T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 953.995422 +2011-01-17T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 712.774595 value +2011-01-18T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 107.244699 +2011-01-18T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 103.018934 +2011-01-18T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 104.724154 +2011-01-18T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 88.764512 +2011-01-18T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1022.783330 +2011-01-18T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 846.267516 value +2011-01-19T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 85.681683 +2011-01-19T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 92.314034 +2011-01-19T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 105.087466 +2011-01-19T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 104.568464 +2011-01-19T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 109.772202 +2011-01-19T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 849.877513 +2011-01-19T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 594.381703 value +2011-01-20T01:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 106.367723 +2011-01-20T01:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 96.112901 +2011-01-20T01:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 105.225158 +2011-01-20T01:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 77.759854 +2011-01-20T01:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1066.208012 +2011-01-20T01:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 870.115926 value +2011-01-22T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 105.282866 +2011-01-22T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 103.527592 +2011-01-22T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 103.399677 +2011-01-22T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 107.627793 +2011-01-22T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1240.525484 +2011-01-22T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1410.278128 value +2011-01-23T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 100.432710 +2011-01-23T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 92.789692 +2011-01-23T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 103.730730 +2011-01-23T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 101.573522 +2011-01-23T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 128.699746 +2011-01-23T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1349.254415 +2011-01-23T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1224.501568 value +2011-01-24T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 104.485760 +2011-01-24T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 189.385952 +2011-01-24T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 104.167373 +2011-01-24T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 105.345921 +2011-01-24T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1102.866656 +2011-01-24T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1215.589859 value +2011-01-25T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 90.111413 +2011-01-25T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 85.974579 +2011-01-25T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 96.594588 +2011-01-25T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 102.512878 +2011-01-25T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 122.077247 +2011-01-25T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 997.994544 +2011-01-25T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 786.363298 value +2011-01-26T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 105.873942 +2011-01-26T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 159.988606 +2011-01-26T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 105.266058 +2011-01-26T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 105.617702 +2011-01-26T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1037.449471 +2011-01-26T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 989.931541 value +2011-01-27T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 134.127106 +2011-01-27T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 90.806201 +2011-01-27T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 100.643435 +2011-01-27T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 102.612747 +2011-01-27T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 149.125271 +2011-01-27T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1486.201299 +2011-01-27T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1367.638074 value +2011-01-28T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 111.641077 +2011-01-28T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 107.788998 +2011-01-28T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 108.106449 +2011-01-28T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 93.869236 +2011-01-28T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1300.302260 +2011-01-28T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1627.598064 value +2011-01-29T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 127.450345 +2011-01-29T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 103.345166 +2011-01-29T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 106.663538 +2011-01-29T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 104.311418 +2011-01-29T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 138.865014 +2011-01-29T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 792.326066 +2011-01-29T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 545.990623 value +2011-01-30T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 106.064111 +2011-01-30T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 137.198397 +2011-01-30T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 106.009926 +2011-01-30T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 110.528451 +2011-01-30T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1233.448863 +2011-01-30T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1870.061029 value +2011-01-31T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 133.740047 +2011-01-31T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 104.548387 +2011-01-31T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 97.017604 +2011-01-31T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 103.021032 +2011-01-31T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 155.744951 +2011-01-31T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1127.231000 +2011-01-31T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 943.497198 value +2011-02-01T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 103.890175 +2011-02-01T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 113.896016 +2011-02-01T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 104.383662 +2011-02-01T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 83.931272 +2011-02-01T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1203.465595 +2011-02-01T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1667.497773 value +2011-02-02T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 113.492245 +2011-02-02T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 112.042996 +2011-02-02T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 97.711139 +2011-02-02T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 102.152053 +2011-02-02T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 123.170962 +2011-02-02T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1410.792943 +2011-02-02T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1273.707453 value +2011-02-03T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 98.877952 +2011-02-03T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 85.069784 +2011-02-03T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 100.976362 +2011-02-03T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 92.326431 +2011-02-03T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1033.401241 +2011-02-03T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 888.705280 value +2011-02-04T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 89.182906 +2011-02-04T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 112.471918 +2011-02-04T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 100.918373 +2011-02-04T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 103.103922 +2011-02-04T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 113.716758 +2011-02-04T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1331.860983 +2011-02-04T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1308.582051 value +2011-02-05T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 111.394244 +2011-02-05T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 94.312960 +2011-02-05T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 107.929804 +2011-02-05T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 90.732978 +2011-02-05T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1039.500513 +2011-02-05T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 785.078869 value +2011-02-06T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 134.462521 +2011-02-06T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 119.608310 +2011-02-06T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 101.847544 +2011-02-06T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 102.770913 +2011-02-06T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 121.270562 +2011-02-06T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1029.995236 +2011-02-06T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 826.039207 value +2011-02-07T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 98.815847 +2011-02-07T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 97.480779 +2011-02-07T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 100.706057 +2011-02-07T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 83.902353 +2011-02-07T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1047.212887 +2011-02-07T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1107.243787 value +2011-02-08T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 126.243536 +2011-02-08T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 106.969799 +2011-02-08T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 96.790543 +2011-02-08T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 103.308255 +2011-02-08T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 111.040150 +2011-02-08T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1082.727640 +2011-02-08T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 911.956790 value +2011-02-09T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 115.548444 +2011-02-09T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 97.071703 +2011-02-09T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 113.554588 +2011-02-09T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 83.742151 +2011-02-09T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 971.050764 +2011-02-09T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 794.098825 value +2011-02-10T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 115.461691 +2011-02-10T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 117.629065 +2011-02-10T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 95.698374 +2011-02-10T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 105.708103 +2011-02-10T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 105.557241 +2011-02-10T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1089.647884 +2011-02-10T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 901.327272 value +2011-02-11T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 114.637486 +2011-02-11T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 96.617339 +2011-02-11T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 112.571724 +2011-02-11T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 87.904114 +2011-02-11T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 980.386611 +2011-02-11T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 723.514254 value +2011-02-12T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 130.104979 +2011-02-12T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 115.225386 +2011-02-12T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 101.515571 +2011-02-12T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 108.637522 +2011-02-12T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 102.850696 +2011-02-12T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1092.416967 +2011-02-12T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 879.406101 value +2011-02-13T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 118.841176 +2011-02-13T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 98.607490 +2011-02-13T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 117.965974 +2011-02-13T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 87.820236 +2011-02-13T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 987.067381 +2011-02-13T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 724.262526 value +2011-02-14T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 119.323168 +2011-02-14T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 123.098262 +2011-02-14T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 101.645725 +2011-02-14T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 121.060464 +2011-02-14T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 107.663239 +2011-02-14T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1199.607472 +2011-02-14T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 948.657939 value +2011-02-15T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 115.527003 +2011-02-15T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 103.519393 +2011-02-15T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 117.138956 +2011-02-15T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 79.700998 +2011-02-15T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1044.384300 +2011-02-15T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 807.601674 value +2011-02-16T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 133.726576 +2011-02-16T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 130.717934 +2011-02-16T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 108.338531 +2011-02-16T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 123.270869 +2011-02-16T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 113.643571 +2011-02-16T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1360.032423 +2011-02-16T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1208.456692 value +2011-02-17T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 125.032692 +2011-02-17T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 101.092779 +2011-02-17T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 121.872569 +2011-02-17T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 91.969195 +2011-02-17T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 993.591221 +2011-02-17T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 679.619354 value +2011-02-18T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 151.053296 +2011-02-18T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 128.063524 +2011-02-18T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 101.960196 +2011-02-18T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 122.546253 +2011-02-18T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 120.151389 +2011-02-18T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1601.829436 +2011-02-18T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1649.533329 value +2011-02-19T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 123.447481 +2011-02-19T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 108.822138 +2011-02-19T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 119.371511 +2011-02-19T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 91.547944 +2011-02-19T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1200.527201 +2011-02-19T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1433.398801 value +2011-02-20T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 157.483005 +2011-02-20T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 176.323916 +2011-02-20T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 127.819268 +2011-02-20T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 132.719065 +2011-02-20T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 124.637709 +2011-02-20T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1304.326111 +2011-02-20T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1022.854576 value +2011-02-21T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 132.231346 +2011-02-21T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 111.386745 +2011-02-21T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 126.995117 +2011-02-21T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 119.850150 +2011-02-21T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1544.108134 +2011-02-21T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1228.502469 value +2011-02-22T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 149.171056 +2011-02-22T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 139.557139 +2011-02-22T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 111.394542 +2011-02-22T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 126.596847 +2011-02-22T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 115.915849 +2011-02-22T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1421.648704 +2011-02-22T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1291.897942 value +2011-02-23T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 127.199815 +2011-02-23T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 107.896489 +2011-02-23T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 122.404192 +2011-02-23T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 91.191071 +2011-02-23T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1251.906228 +2011-02-23T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1306.495696 value +2011-02-24T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 168.988478 +2011-02-24T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 139.978575 +2011-02-24T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 113.621184 +2011-02-24T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 119.772575 +2011-02-24T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 118.270052 +2011-02-24T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 970.728273 +2011-02-24T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 578.795979 value +2011-02-25T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 107.539869 +2011-02-25T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 99.698015 +2011-02-25T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 106.827630 +2011-02-25T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 82.350556 +2011-02-25T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1057.427269 +2011-02-25T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1017.573185 value +2011-02-26T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 225.243186 +2011-02-26T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 138.924835 +2011-02-26T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 98.314744 +2011-02-26T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 118.896845 +2011-02-26T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 131.310541 +2011-02-26T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1743.921750 +2011-02-26T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1862.737933 value +2011-02-27T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 96.864384 +2011-02-27T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 114.634278 +2011-02-27T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 99.158839 +2011-02-27T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 78.727950 +2011-02-27T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1044.562903 +2011-02-27T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 873.306547 value +2011-02-28T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 122.258195 +2011-02-28T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 141.261324 +2011-02-28T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 102.185098 +2011-02-28T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 121.786785 +2011-02-28T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 114.284569 +2011-02-28T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1292.542896 +2011-02-28T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1101.918270 value +2011-03-01T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 99.070796 +2011-03-01T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 114.700932 +2011-03-01T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 99.772347 +2011-03-01T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 72.792970 +2011-03-01T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1124.201419 +2011-03-01T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1166.141121 value +2011-03-02T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 174.890520 +2011-03-02T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 147.117434 +2011-03-02T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 109.239196 +2011-03-02T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 120.212473 +2011-03-02T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 116.460744 +2011-03-02T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1143.078414 +2011-03-02T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 865.777900 value +2011-03-03T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 92.537499 +2011-03-03T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 108.914095 +2011-03-03T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 96.031371 +2011-03-03T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 75.977564 +2011-03-03T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1097.490771 +2011-03-03T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 901.307577 value +2011-03-04T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 119.851231 +2011-03-04T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 132.832331 +2011-03-04T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 107.285615 +2011-03-04T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 113.883056 +2011-03-04T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 109.607245 +2011-03-04T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1179.803776 +2011-03-04T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 975.577927 value +2011-03-05T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 97.942645 +2011-03-05T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 108.394611 +2011-03-05T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 102.486832 +2011-03-05T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 81.105110 +2011-03-05T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1281.601175 +2011-03-05T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1360.694785 value +2011-03-06T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 129.531062 +2011-03-06T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 144.925734 +2011-03-06T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 113.035167 +2011-03-06T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 115.956859 +2011-03-06T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 120.953163 +2011-03-06T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1103.239788 +2011-03-06T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 869.308360 value +2011-03-07T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 105.575929 +2011-03-07T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 118.024245 +2011-03-07T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 107.220218 +2011-03-07T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 68.699125 +2011-03-07T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1177.858403 +2011-03-07T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1095.637520 value +2011-03-08T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 109.764955 +2011-03-08T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 143.214331 +2011-03-08T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 107.465492 +2011-03-08T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 122.692722 +2011-03-08T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 111.704071 +2011-03-08T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1084.332554 +2011-03-08T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 835.762631 value +2011-03-09T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 110.873407 +2011-03-09T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 115.013313 +2011-03-09T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 112.407868 +2011-03-09T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 82.426362 +2011-03-09T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1029.802500 +2011-03-09T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 776.702940 value +2011-03-10T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 148.809150 +2011-03-10T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 134.212714 +2011-03-10T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 119.613508 +2011-03-10T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 118.864028 +2011-03-10T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 107.706257 +2011-03-10T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1077.279402 +2011-03-10T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 835.461226 value +2011-03-11T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 106.898536 +2011-03-11T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 112.856230 +2011-03-11T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 108.135811 +2011-03-11T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 67.731170 +2011-03-11T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1098.543170 +2011-03-11T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 715.516125 value +2011-03-12T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 155.728048 +2011-03-12T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 132.687079 +2011-03-12T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 114.681603 +2011-03-12T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 118.574721 +2011-03-12T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 109.384493 +2011-03-12T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1008.745525 +2011-03-12T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 764.508070 value +2011-03-13T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 113.760384 +2011-03-13T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 120.760130 +2011-03-13T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 114.814070 +2011-03-13T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 70.126017 +2011-03-13T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 998.753955 +2011-03-13T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 736.409261 value +2011-03-14T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 153.191744 +2011-03-14T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 123.248581 +2011-03-14T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 104.716583 +2011-03-14T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 122.275869 +2011-03-14T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 104.724023 +2011-03-14T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1141.588400 +2011-03-14T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 893.985017 value +2011-03-15T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 109.382273 +2011-03-15T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 119.250945 +2011-03-15T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 112.354294 +2011-03-15T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 59.266595 +2011-03-15T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1037.381049 +2011-03-15T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 832.874861 value +2011-03-16T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 147.471464 +2011-03-16T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 110.070846 +2011-03-16T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 99.137980 +2011-03-16T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 120.455865 +2011-03-16T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 107.903885 +2011-03-16T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1092.942008 +2011-03-16T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 848.339888 value +2011-03-17T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 113.501786 +2011-03-17T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 114.540037 +2011-03-17T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 116.816958 +2011-03-17T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 69.258523 +2011-03-17T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1072.239320 +2011-03-17T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 878.683776 value +2011-03-18T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 180.343171 +2011-03-18T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 130.260926 +2011-03-18T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 111.540639 +2011-03-18T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 119.629977 +2011-03-18T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 114.564808 +2011-03-18T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1176.605164 +2011-03-18T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 936.429632 value +2011-03-19T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 109.788875 +2011-03-19T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 117.197085 +2011-03-19T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 112.236468 +2011-03-19T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 72.369471 +2011-03-19T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1107.220174 +2011-03-19T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 846.288386 value +2011-03-20T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 178.454262 +2011-03-20T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 157.749330 +2011-03-20T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 128.274705 +2011-03-20T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 129.519442 +2011-03-20T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 128.705337 +2011-03-20T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1217.547439 +2011-03-20T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 954.754185 value +2011-03-21T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 124.411632 +2011-03-21T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 122.462424 +2011-03-21T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 125.243882 +2011-03-21T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 75.936640 +2011-03-21T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1283.957016 +2011-03-21T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1031.990042 value +2011-03-22T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 177.460613 +2011-03-22T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 151.407583 +2011-03-22T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 126.415884 +2011-03-22T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 133.124963 +2011-03-22T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 122.357549 +2011-03-22T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1119.247202 +2011-03-22T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 843.952139 value +2011-03-23T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 126.764513 +2011-03-23T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 118.972310 +2011-03-23T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 129.864384 +2011-03-23T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 102.603152 +2011-03-23T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1403.338838 +2011-03-23T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1362.650586 value +2011-03-24T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 135.569784 +2011-03-24T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 156.650862 +2011-03-24T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 115.945757 +2011-03-24T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 129.023978 +2011-03-24T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 124.062061 +2011-03-24T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1137.842315 +2011-03-24T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 873.652030 value +2011-03-25T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 125.766952 +2011-03-25T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 119.400783 +2011-03-25T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 126.057960 +2011-03-25T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 89.408906 +2011-03-25T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1124.935193 +2011-03-25T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 855.717712 value +2011-03-26T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 148.957194 +2011-03-26T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 162.815450 +2011-03-26T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 117.449116 +2011-03-26T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 139.214665 +2011-03-26T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 138.663182 +2011-03-26T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1247.890809 +2011-03-26T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 962.235801 value +2011-03-27T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 135.183271 +2011-03-27T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 130.599006 +2011-03-27T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 114.952545 +2011-03-27T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 139.294248 +2011-03-27T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 142.430177 +2011-03-27T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 108.489598 +2011-03-27T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 141.820068 +2011-03-27T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1185.709973 +2011-03-27T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1345.781728 +2011-03-27T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1019.898509 value +2011-03-27T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1056.419292 value +2011-03-28T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 156.155294 +2011-03-28T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 124.336139 +2011-03-28T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 163.100154 +2011-03-28T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 131.191818 +2011-03-28T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 115.709767 +2011-03-28T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 127.403089 +2011-03-28T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 140.941296 +2011-03-28T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 103.578536 +2011-03-28T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 130.880788 +2011-03-28T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1250.166788 +2011-03-28T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1390.754050 +2011-03-28T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1198.723103 value +2011-03-28T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1108.136072 value +2011-03-29T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 134.084672 +2011-03-29T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 117.626804 +2011-03-29T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 158.073319 +2011-03-29T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 128.074393 +2011-03-29T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 109.678515 +2011-03-29T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 122.620188 +2011-03-29T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 144.446039 +2011-03-29T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 91.604463 +2011-03-29T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 132.477651 +2011-03-29T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1224.116225 +2011-03-29T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1361.080245 +2011-03-29T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1289.009485 value +2011-03-29T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1069.431801 value +2011-03-30T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 135.942820 +2011-03-30T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 120.283054 +2011-03-30T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 167.960620 +2011-03-30T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 121.215839 +2011-03-30T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 116.587746 +2011-03-30T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 122.612114 +2011-03-30T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 138.827311 +2011-03-30T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 93.331887 +2011-03-30T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 128.645571 +2011-03-30T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1190.933753 +2011-03-30T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1310.797070 +2011-03-30T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1001.134025 value +2011-03-30T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1030.499562 value +2011-03-31T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 151.752485 +2011-03-31T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 124.414321 +2011-03-31T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 175.778647 +2011-03-31T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 120.607382 +2011-03-31T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 117.060598 +2011-03-31T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 125.243245 +2011-03-31T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 150.247713 +2011-03-31T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 93.390841 +2011-03-31T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 125.839686 +2011-03-31T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1466.209327 +2011-03-31T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1366.447617 +2011-03-31T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1734.274909 value +2011-03-31T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1063.201156 value +2011-04-01T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 135.885094 +2011-04-01T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 118.570340 +2011-04-01T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 158.747224 +2011-04-01T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 120.134704 +2011-04-01T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 109.705815 +2011-04-01T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 121.583581 +2011-04-01T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 144.507368 +2011-04-01T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 78.622547 +2011-04-01T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 119.922742 +2011-04-01T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1314.839715 +2011-04-01T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1522.043733 +2011-04-01T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1447.341160 value +2011-04-01T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1234.247546 value +2011-04-02T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 147.425935 +2011-04-02T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 112.987027 +2011-04-02T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 166.016049 +2011-04-02T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 113.446008 +2011-04-02T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 110.931934 +2011-04-02T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 114.290141 +2011-04-02T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 135.301506 +2011-04-02T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 97.387433 +2011-04-02T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 126.411364 +2011-04-02T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1193.556278 +2011-04-02T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1321.375057 +2011-04-02T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1144.342401 value +2011-04-02T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1049.738585 value +2011-04-03T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 122.971856 +2011-04-03T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 105.735462 +2011-04-03T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 153.927965 +2011-04-03T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 103.532351 +2011-04-03T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 107.047773 +2011-04-03T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 107.919674 +2011-04-03T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 122.141707 +2011-04-03T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 80.861743 +2011-04-03T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 117.247070 +2011-04-03T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1055.783661 +2011-04-03T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1021.638673 +2011-04-03T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 811.991286 value +2011-04-03T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 768.423077 value +2011-04-04T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 110.919829 +2011-04-04T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 107.613577 +2011-04-04T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 146.729242 +2011-04-04T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 105.375351 +2011-04-04T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 110.573670 +2011-04-04T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 114.382255 +2011-04-04T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 125.285894 +2011-04-04T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 72.668430 +2011-04-04T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 117.703023 +2011-04-04T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1197.008423 +2011-04-04T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1131.531986 +2011-04-04T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1151.069173 value +2011-04-04T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 877.079396 value +2011-04-05T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 113.318712 +2011-04-05T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 105.615563 +2011-04-05T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 141.713507 +2011-04-05T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 106.207931 +2011-04-05T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 109.890586 +2011-04-05T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 110.012987 +2011-04-05T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 124.478408 +2011-04-05T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 86.683603 +2011-04-05T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 117.051694 +2011-04-05T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1192.144303 +2011-04-05T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1154.289559 +2011-04-05T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1146.423036 value +2011-04-05T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 902.615706 value +2011-04-06T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 115.334018 +2011-04-06T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 109.700256 +2011-04-06T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 147.553562 +2011-04-06T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 100.775597 +2011-04-06T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 108.659345 +2011-04-06T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 113.408308 +2011-04-06T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 121.079585 +2011-04-06T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 92.336403 +2011-04-06T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 112.921482 +2011-04-06T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1272.677122 +2011-04-06T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1141.514652 +2011-04-06T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1355.843374 value +2011-04-06T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 897.393445 value +2011-04-07T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 117.508062 +2011-04-07T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 115.418054 +2011-04-07T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 141.565031 +2011-04-07T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 95.562446 +2011-04-07T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 102.933171 +2011-04-07T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 122.696621 +2011-04-07T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 119.763135 +2011-04-07T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 84.357701 +2011-04-07T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 117.432760 +2011-04-07T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1190.896088 +2011-04-07T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1009.363132 +2011-04-07T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1260.143027 value +2011-04-07T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 762.862488 value +2011-04-08T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 120.973860 +2011-04-08T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 116.248860 +2011-04-08T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 146.830618 +2011-04-08T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 96.226609 +2011-04-08T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 95.447888 +2011-04-08T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 120.709912 +2011-04-08T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 120.309688 +2011-04-08T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 94.631354 +2011-04-08T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 122.743898 +2011-04-08T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1006.913816 +2011-04-08T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1032.599837 +2011-04-08T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 935.168026 value +2011-04-08T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 782.107861 value +2011-04-09T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 116.080323 +2011-04-09T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 116.060759 +2011-04-09T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 158.682525 +2011-04-09T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 95.509796 +2011-04-09T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 110.007248 +2011-04-09T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 121.905685 +2011-04-09T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 126.369367 +2011-04-09T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 92.905070 +2011-04-09T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 123.791320 +2011-04-09T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1137.385764 +2011-04-09T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1030.075553 +2011-04-09T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 996.205369 value +2011-04-09T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 767.692135 value +2011-04-10T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 113.221448 +2011-04-10T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 95.570457 +2011-04-10T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 131.766616 +2011-04-10T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 99.950855 +2011-04-10T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 91.470524 +2011-04-10T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 99.393076 +2011-04-10T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 123.207579 +2011-04-10T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 84.898691 +2011-04-10T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 114.353962 +2011-04-10T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1005.253077 +2011-04-10T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1030.094757 +2011-04-10T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1031.741509 value +2011-04-10T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 775.965555 value +2011-04-11T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 130.165796 +2011-04-11T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 107.765101 +2011-04-11T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 142.751726 +2011-04-11T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 104.847285 +2011-04-11T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 95.272956 +2011-04-11T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 106.229286 +2011-04-11T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 126.823859 +2011-04-11T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 89.250155 +2011-04-11T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 122.049678 +2011-04-11T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1112.794811 +2011-04-11T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1113.357530 +2011-04-11T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1374.968412 value +2011-04-11T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 853.163039 value +2011-04-12T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 122.386348 +2011-04-12T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 106.380995 +2011-04-12T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 141.932300 +2011-04-12T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 103.142372 +2011-04-12T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 97.340631 +2011-04-12T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 105.381244 +2011-04-12T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 125.189098 +2011-04-12T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 90.533391 +2011-04-12T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 122.128172 +2011-04-12T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1153.974725 +2011-04-12T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1069.640880 +2011-04-12T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1456.611830 value +2011-04-12T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 811.925240 value +2011-04-13T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 122.688340 +2011-04-13T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 105.739623 +2011-04-13T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 136.983407 +2011-04-13T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 100.860813 +2011-04-13T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 94.839191 +2011-04-13T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 105.261296 +2011-04-13T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 119.836611 +2011-04-13T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 91.972558 +2011-04-13T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 120.145572 +2011-04-13T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1016.137449 +2011-04-13T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 994.902292 +2011-04-13T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 989.032799 value +2011-04-13T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 744.744657 value +2011-04-14T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 111.179339 +2011-04-14T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 101.984377 +2011-04-14T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 133.606430 +2011-04-14T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 99.738319 +2011-04-14T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 91.270553 +2011-04-14T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 101.251756 +2011-04-14T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 118.285128 +2011-04-14T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 84.951300 +2011-04-14T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 119.768525 +2011-04-14T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1032.154263 +2011-04-14T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 999.586450 +2011-04-14T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1166.401205 value +2011-04-14T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 753.104985 value +2011-04-15T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 106.793700 +2011-04-15T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 94.469747 +2011-04-15T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 135.109191 +2011-04-15T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 99.596909 +2011-04-15T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 92.782760 +2011-04-15T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 97.859766 +2011-04-15T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 120.508160 +2011-04-15T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 89.646236 +2011-04-15T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 120.290348 +2011-04-15T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 994.752744 +2011-04-15T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1029.056992 +2011-04-15T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 962.731172 value +2011-04-15T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 780.271977 value diff --git a/processing/src/test/resources/druid.sample.numeric.tsv.top b/processing/src/test/resources/druid.sample.numeric.tsv.top new file mode 100644 index 000000000000..fa0c0228b167 --- /dev/null +++ b/processing/src/test/resources/druid.sample.numeric.tsv.top @@ -0,0 +1,477 @@ +2011-01-12T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 100.000000 +2011-01-12T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 100.000000 +2011-01-12T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 100.000000 +2011-01-12T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 100.000000 +2011-01-12T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 100.000000 +2011-01-12T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1000.000000 +2011-01-12T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 800.000000 value +2011-01-13T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 103.629399 +2011-01-13T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 114.947403 +2011-01-13T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 102.851683 +2011-01-13T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 111.356672 +2011-01-13T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1040.945505 +2011-01-13T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 826.060182 value +2011-01-14T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 86.450372 +2011-01-14T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 109.573474 +2011-01-14T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 97.903068 +2011-01-14T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 104.611784 +2011-01-14T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 112.259958 +2011-01-14T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1073.476545 +2011-01-14T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 869.643722 value +2011-01-15T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 99.781645 +2011-01-15T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 87.954346 +2011-01-15T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 99.383407 +2011-01-15T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 111.680229 +2011-01-15T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1007.365510 +2011-01-15T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 809.041763 value +2011-01-16T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 71.315931 +2011-01-16T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 99.007588 +2011-01-16T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 99.863171 +2011-01-16T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 103.348007 +2011-01-16T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 116.779610 +2011-01-16T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1077.612663 +2011-01-16T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 879.988099 value +2011-01-17T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 105.914315 +2011-01-17T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 110.157325 +2011-01-17T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 103.615039 +2011-01-17T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 89.901887 +2011-01-17T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1075.089574 +2011-01-17T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 950.146770 value +2011-01-18T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 87.195139 +2011-01-18T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 94.452739 +2011-01-18T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 101.087367 +2011-01-18T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 108.979936 +2011-01-18T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 113.680094 +2011-01-18T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 937.061939 +2011-01-18T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 682.885525 value +2011-01-19T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 106.700550 +2011-01-19T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 97.620315 +2011-01-19T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 106.127830 +2011-01-19T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 77.316731 +2011-01-19T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1156.744712 +2011-01-19T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1109.874950 value +2011-01-20T01:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 93.396274 +2011-01-20T01:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 90.439020 +2011-01-20T01:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 105.669498 +2011-01-20T01:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 101.305541 +2011-01-20T01:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 140.179069 +2011-01-20T01:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 904.340636 +2011-01-20T01:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 677.510973 value +2011-01-22T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 95.235266 +2011-01-22T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 93.681096 +2011-01-22T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 104.184494 +2011-01-22T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 99.284525 +2011-01-22T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 137.109783 +2011-01-22T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1343.232494 +2011-01-22T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1219.432170 value +2011-01-23T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 107.348157 +2011-01-23T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 96.826443 +2011-01-23T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 106.418686 +2011-01-23T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 110.467875 +2011-01-23T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1088.943083 +2011-01-23T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 979.306038 value +2011-01-24T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 96.671647 +2011-01-24T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 88.748460 +2011-01-24T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 97.906256 +2011-01-24T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 101.581339 +2011-01-24T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 131.695956 +2011-01-24T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 939.244103 +2011-01-24T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 716.609179 value +2011-01-25T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 101.624789 +2011-01-25T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 180.575246 +2011-01-25T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 102.907866 +2011-01-25T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 102.044542 +2011-01-25T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1109.875413 +2011-01-25T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1301.023342 value +2011-01-26T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 84.906466 +2011-01-26T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 84.710523 +2011-01-26T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 96.046584 +2011-01-26T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 101.088903 +2011-01-26T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 122.160681 +2011-01-26T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1686.419659 +2011-01-26T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1609.096706 value +2011-01-27T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 104.882908 +2011-01-27T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 170.735853 +2011-01-27T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 104.609483 +2011-01-27T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 116.979005 +2011-01-27T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1074.006938 +2011-01-27T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1023.295213 value +2011-01-28T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 123.006128 +2011-01-28T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 99.681629 +2011-01-28T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 106.075672 +2011-01-28T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 103.822842 +2011-01-28T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 158.739359 +2011-01-28T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1021.334487 +2011-01-28T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 810.889422 value +2011-01-29T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 100.992147 +2011-01-29T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 114.905745 +2011-01-29T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 101.998823 +2011-01-29T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 109.549035 +2011-01-29T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1314.619452 +2011-01-29T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1685.500085 value +2011-01-30T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 124.943293 +2011-01-30T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 108.415967 +2011-01-30T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 97.023907 +2011-01-30T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 104.298490 +2011-01-30T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 137.932693 +2011-01-30T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 805.930143 +2011-01-30T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 555.476028 value +2011-01-31T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 103.492964 +2011-01-31T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 124.171944 +2011-01-31T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 103.832040 +2011-01-31T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 85.125795 +2011-01-31T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1184.920651 +2011-01-31T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1643.340851 value +2011-02-01T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 132.123776 +2011-02-01T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 103.652865 +2011-02-01T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 98.909356 +2011-02-01T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 102.480377 +2011-02-01T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 134.014606 +2011-02-01T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1100.904846 +2011-02-01T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 913.561076 value +2011-02-02T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 104.963233 +2011-02-02T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 102.281859 +2011-02-02T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 105.578807 +2011-02-02T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 96.706279 +2011-02-02T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1097.211164 +2011-02-02T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1218.561908 value +2011-02-03T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 85.770241 +2011-02-03T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 106.425780 +2011-02-03T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 100.559287 +2011-02-03T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 102.349315 +2011-02-03T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 134.140377 +2011-02-03T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1283.166055 +2011-02-03T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1113.114125 value +2011-02-04T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 106.888769 +2011-02-04T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 99.837572 +2011-02-04T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 106.050728 +2011-02-04T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 93.973465 +2011-02-04T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1025.633340 +2011-02-04T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 864.568891 value +2011-02-05T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 93.001571 +2011-02-05T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 117.030289 +2011-02-05T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 105.660538 +2011-02-05T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 100.646747 +2011-02-05T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 114.723682 +2011-02-05T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1332.468373 +2011-02-05T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1363.614929 value +2011-02-06T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 110.897359 +2011-02-06T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 93.585758 +2011-02-06T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 110.053071 +2011-02-06T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 93.620739 +2011-02-06T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1011.205470 +2011-02-06T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 787.125330 value +2011-02-07T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 130.194219 +2011-02-07T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 112.924874 +2011-02-07T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 97.434318 +2011-02-07T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 102.705243 +2011-02-07T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 115.246714 +2011-02-07T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1057.079944 +2011-02-07T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 872.625669 value +2011-02-08T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 93.190129 +2011-02-08T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 97.432302 +2011-02-08T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 97.085047 +2011-02-08T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 75.735586 +2011-02-08T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1064.972638 +2011-02-08T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1188.369265 value +2011-02-09T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 129.221792 +2011-02-09T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 111.729360 +2011-02-09T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 96.478571 +2011-02-09T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 105.498315 +2011-02-09T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 112.646238 +2011-02-09T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1320.638308 +2011-02-09T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1299.093262 value +2011-02-10T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 118.062165 +2011-02-10T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 97.235999 +2011-02-10T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 115.824976 +2011-02-10T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 91.750911 +2011-02-10T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1070.165582 +2011-02-10T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1212.928303 value +2011-02-11T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 129.187009 +2011-02-11T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 114.960877 +2011-02-11T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 100.111873 +2011-02-11T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 105.672256 +2011-02-11T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 102.864842 +2011-02-11T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1179.695901 +2011-02-11T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1061.973330 value +2011-02-12T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 115.758445 +2011-02-12T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 96.457082 +2011-02-12T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 114.877503 +2011-02-12T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 88.142774 +2011-02-12T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 959.236186 +2011-02-12T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 602.979544 value +2011-02-13T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 119.490316 +2011-02-13T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 119.907266 +2011-02-13T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 100.905238 +2011-02-13T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 112.514409 +2011-02-13T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 106.033416 +2011-02-13T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1103.458199 +2011-02-13T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 862.931321 value +2011-02-14T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 115.628202 +2011-02-14T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 103.008650 +2011-02-14T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 117.110451 +2011-02-14T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 73.717033 +2011-02-14T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1091.223197 +2011-02-14T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1133.135123 value +2011-02-15T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 123.485071 +2011-02-15T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 121.563912 +2011-02-15T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 105.269599 +2011-02-15T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 121.411398 +2011-02-15T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 108.428302 +2011-02-15T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1183.240825 +2011-02-15T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 914.525048 value +2011-02-16T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 116.432276 +2011-02-16T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 105.762627 +2011-02-16T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 117.334381 +2011-02-16T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 98.918664 +2011-02-16T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1289.097304 +2011-02-16T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1553.348548 value +2011-02-17T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 147.942017 +2011-02-17T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 126.982673 +2011-02-17T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 103.940963 +2011-02-17T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 120.050545 +2011-02-17T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 112.150745 +2011-02-17T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1021.071173 +2011-02-17T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 645.177645 value +2011-02-18T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 127.611947 +2011-02-18T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 100.849247 +2011-02-18T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 124.513018 +2011-02-18T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 92.174432 +2011-02-18T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1105.383465 +2011-02-18T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1120.088751 value +2011-02-19T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 163.351690 +2011-02-19T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 133.726878 +2011-02-19T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 102.919452 +2011-02-19T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 125.052129 +2011-02-19T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 121.733400 +2011-02-19T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1600.723226 +2011-02-19T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1598.179271 value +2011-02-20T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 129.409606 +2011-02-20T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 109.790712 +2011-02-20T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 122.082375 +2011-02-20T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 105.985049 +2011-02-20T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1317.458323 +2011-02-20T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1192.563067 value +2011-02-21T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 155.632898 +2011-02-21T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 193.787574 +2011-02-21T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 154.627912 +2011-02-21T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 138.092468 +2011-02-21T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 119.739112 +2011-02-21T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1488.737765 +2011-02-21T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1298.415763 value +2011-02-22T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 124.524992 +2011-02-22T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 108.370907 +2011-02-22T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 122.510640 +2011-02-22T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 86.333235 +2011-02-22T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1224.827108 +2011-02-22T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1345.964309 value +2011-02-23T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 165.273009 +2011-02-23T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 145.588115 +2011-02-23T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 113.141185 +2011-02-23T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 124.305608 +2011-02-23T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 123.961542 +2011-02-23T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1414.619004 +2011-02-23T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1287.766687 value +2011-02-24T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 123.553981 +2011-02-24T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 104.951315 +2011-02-24T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 118.862342 +2011-02-24T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 91.962584 +2011-02-24T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1350.175381 +2011-02-24T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1631.584352 value +2011-02-25T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 172.335540 +2011-02-25T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 140.941317 +2011-02-25T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 99.460461 +2011-02-25T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 115.932803 +2011-02-25T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 120.124862 +2011-02-25T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1073.967314 +2011-02-25T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 782.013486 value +2011-02-26T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 103.643952 +2011-02-26T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 103.667031 +2011-02-26T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 105.352891 +2011-02-26T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 83.099365 +2011-02-26T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 996.433708 +2011-02-26T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 829.916235 value +2011-02-27T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 277.273533 +2011-02-27T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 136.394846 +2011-02-27T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 103.226967 +2011-02-27T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 121.929932 +2011-02-27T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 136.163414 +2011-02-27T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1474.591017 +2011-02-27T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1427.016724 value +2011-02-28T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 97.218943 +2011-02-28T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 112.528286 +2011-02-28T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 99.505465 +2011-02-28T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 72.163651 +2011-02-28T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1159.278766 +2011-02-28T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1430.257348 value +2011-03-01T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 153.059937 +2011-03-01T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 143.424672 +2011-03-01T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 105.453024 +2011-03-01T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 123.251814 +2011-03-01T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 116.975408 +2011-03-01T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1243.354010 +2011-03-01T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 1004.940887 value +2011-03-02T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 98.432014 +2011-03-02T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 112.968782 +2011-03-02T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 100.600391 +2011-03-02T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 82.823988 +2011-03-02T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1051.808940 +2011-03-02T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 740.183720 value +2011-03-03T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 127.994476 +2011-03-03T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 140.215411 +2011-03-03T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 108.784646 +2011-03-03T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 115.393493 +2011-03-03T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 114.188310 +2011-03-03T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1010.370296 +2011-03-03T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 691.958920 value +2011-03-04T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 93.634505 +2011-03-04T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 110.018472 +2011-03-04T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 97.535226 +2011-03-04T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 81.131208 +2011-03-04T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1326.829155 +2011-03-04T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1674.331703 value +2011-03-05T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 136.941770 +2011-03-05T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 145.393016 +2011-03-05T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 112.522435 +2011-03-05T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 114.691277 +2011-03-05T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 117.904527 +2011-03-05T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 994.731237 +2011-03-05T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 755.899363 value +2011-03-06T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 99.508679 +2011-03-06T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 113.069662 +2011-03-06T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 102.536839 +2011-03-06T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 81.612269 +2011-03-06T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1081.650406 +2011-03-06T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 771.348460 value +2011-03-07T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 111.909348 +2011-03-07T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 150.452695 +2011-03-07T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 111.106693 +2011-03-07T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 121.582721 +2011-03-07T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 106.884238 +2011-03-07T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1152.547767 +2011-03-07T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 906.373797 value +2011-03-08T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 98.972716 +2011-03-08T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 119.777621 +2011-03-08T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 101.652185 +2011-03-08T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 70.866726 +2011-03-08T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1117.953961 +2011-03-08T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 988.893782 value +2011-03-09T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 139.260950 +2011-03-09T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 138.466933 +2011-03-09T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 105.613469 +2011-03-09T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 121.220772 +2011-03-09T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 107.998334 +2011-03-09T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1121.385333 +2011-03-09T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 875.683406 value +2011-03-10T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 105.214709 +2011-03-10T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 114.717338 +2011-03-10T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 107.127962 +2011-03-10T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 79.793836 +2011-03-10T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1244.849915 +2011-03-10T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1070.836247 value +2011-03-11T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 135.820968 +2011-03-11T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 135.038992 +2011-03-11T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 120.497687 +2011-03-11T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 118.298350 +2011-03-11T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 108.186877 +2011-03-11T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 998.650727 +2011-03-11T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 755.646538 value +2011-03-12T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 113.493460 +2011-03-12T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 112.554597 +2011-03-12T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 115.572940 +2011-03-12T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 74.394926 +2011-03-12T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1088.807596 +2011-03-12T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 771.100508 value +2011-03-13T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 149.637715 +2011-03-13T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 120.113921 +2011-03-13T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 103.227522 +2011-03-13T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 120.620862 +2011-03-13T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 108.579283 +2011-03-13T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1129.723252 +2011-03-13T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 884.837267 value +2011-03-14T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 109.461442 +2011-03-14T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 120.487244 +2011-03-14T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 111.688901 +2011-03-14T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 59.021022 +2011-03-14T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1075.243024 +2011-03-14T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 918.722840 value +2011-03-15T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 145.963558 +2011-03-15T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 121.386341 +2011-03-15T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 103.583295 +2011-03-15T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 122.038585 +2011-03-15T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 103.134338 +2011-03-15T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1099.197263 +2011-03-15T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 850.995007 value +2011-03-16T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 110.565004 +2011-03-16T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 115.750963 +2011-03-16T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 112.577264 +2011-03-16T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 69.329723 +2011-03-16T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 981.577244 +2011-03-16T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 767.973326 value +2011-03-17T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 148.905410 +2011-03-17T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 109.666402 +2011-03-17T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 105.996125 +2011-03-17T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 122.740143 +2011-03-17T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 110.797348 +2011-03-17T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1154.415689 +2011-03-17T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 906.101957 value +2011-03-18T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 110.037579 +2011-03-18T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 113.490115 +2011-03-18T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 113.238556 +2011-03-18T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 68.573162 +2011-03-18T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1311.178603 +2011-03-18T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1403.830217 value +2011-03-19T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 177.514270 +2011-03-19T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 134.147573 +2011-03-19T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 112.999693 +2011-03-19T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 120.638001 +2011-03-19T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 115.384807 +2011-03-19T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1102.698977 +2011-03-19T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 856.490089 value +2011-03-20T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 123.507497 +2011-03-20T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 117.851058 +2011-03-20T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 125.496367 +2011-03-20T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 85.013155 +2011-03-20T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1285.090048 +2011-03-20T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1032.257527 value +2011-03-21T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 182.035296 +2011-03-21T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 157.153730 +2011-03-21T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 128.149976 +2011-03-21T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 131.807919 +2011-03-21T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 123.653645 +2011-03-21T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1178.830164 +2011-03-21T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 907.021565 value +2011-03-22T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 121.270611 +2011-03-22T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 124.400780 +2011-03-22T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 124.970533 +2011-03-22T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 79.948248 +2011-03-22T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1301.778098 +2011-03-22T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1110.788895 value +2011-03-23T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 154.019632 +2011-03-23T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 158.592715 +2011-03-23T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 126.672392 +2011-03-23T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 131.786598 +2011-03-23T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 123.754240 +2011-03-23T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1156.601892 +2011-03-23T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 884.801502 value +2011-03-24T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 123.895027 +2011-03-24T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 119.777998 +2011-03-24T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 125.044877 +2011-03-24T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 88.521042 +2011-03-24T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1429.580257 +2011-03-24T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1684.268799 value +2011-03-25T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 140.577121 +2011-03-25T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 157.432108 +2011-03-25T00:00:00.000Z spot mezzanine 1400 14000.0 140000 preferred mpreferred 114.706960 +2011-03-25T00:00:00.000Z spot premium 1600 16000.0 160000 preferred ppreferred 128.943094 +2011-03-25T00:00:00.000Z spot travel 1800 18000.0 180000 preferred tpreferred 136.136598 +2011-03-25T00:00:00.000Z total_market premium 1600 16000.0 160000 preferred ppreferred 1256.499779 +2011-03-25T00:00:00.000Z upfront premium 1600 16000.0 160000 preferred ppreferred 996.564152 value +2011-03-26T00:00:00.000Z spot business 1100 11000.0 110000 preferred bpreferred 130.824022 +2011-03-26T00:00:00.000Z spot health 1300 13000.0 130000 preferred hpreferred 118.463523 +2011-03-26T00:00:00.000Z spot news 1500 15000.0 150000 preferred npreferred 142.972964 +2011-03-26T00:00:00.000Z spot technology 1700 17000.0 170000 preferred tpreferred 101.686196 +2011-03-26T00:00:00.000Z total_market mezzanine 1400 14000.0 140000 preferred mpreferred 1217.877395 +2011-03-26T00:00:00.000Z upfront mezzanine 1400 14000.0 140000 preferred mpreferred 1061.678577 value +2011-03-27T00:00:00.000Z spot automotive 1000 10000.0 100000 preferred apreferred 144.056669 +2011-03-27T00:00:00.000Z spot entertainment 1200 12000.0 120000 preferred epreferred 163.161361 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 e52c71dfb164..4f36704df642 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -601,14 +601,14 @@ public void testQueryWithMultipleSegmentSpec() throws IOException, InterruptedEx interval_26_28, QueryRunnerTestHelper.makeQueryRunner( factory, - "druid.sample.tsv.top", + "druid.sample.numeric.tsv.top", null ) , interval_28_29, QueryRunnerTestHelper.makeQueryRunner( factory, - "druid.sample.tsv.bottom", + "druid.sample.numeric.tsv.bottom", null ) ); diff --git a/sql/src/main/java/io/druid/sql/calcite/aggregation/ApproxCountDistinctSqlAggregator.java b/sql/src/main/java/io/druid/sql/calcite/aggregation/ApproxCountDistinctSqlAggregator.java index beed16ae9a0e..653838e79f04 100644 --- a/sql/src/main/java/io/druid/sql/calcite/aggregation/ApproxCountDistinctSqlAggregator.java +++ b/sql/src/main/java/io/druid/sql/calcite/aggregation/ApproxCountDistinctSqlAggregator.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; +import io.druid.java.util.common.ISE; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory; @@ -31,9 +32,11 @@ import io.druid.segment.column.ValueType; import io.druid.sql.calcite.expression.Expressions; import io.druid.sql.calcite.expression.RowExtraction; +import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.table.RowSignature; import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlAggFunction; import org.apache.calcite.sql.SqlFunctionCategory; import org.apache.calcite.sql.SqlKind; @@ -65,13 +68,14 @@ public Aggregation toDruidAggregation( final DimFilter filter ) { + final RexNode rexNode = Expressions.fromFieldAccess( + rowSignature, + project, + Iterables.getOnlyElement(aggregateCall.getArgList()) + ); final RowExtraction rex = Expressions.toRowExtraction( rowSignature.getRowOrder(), - Expressions.fromFieldAccess( - rowSignature, - project, - Iterables.getOnlyElement(aggregateCall.getArgList()) - ) + rexNode ); if (rex == null) { return null; @@ -82,7 +86,13 @@ public Aggregation toDruidAggregation( if (rowSignature.getColumnType(rex.getColumn()) == ValueType.COMPLEX) { aggregatorFactory = new HyperUniquesAggregatorFactory(name, rex.getColumn()); } else { - final DimensionSpec dimensionSpec = rex.toDimensionSpec(rowSignature, null); + final SqlTypeName sqlTypeName = rexNode.getType().getSqlTypeName(); + final ValueType outputType = Calcites.getValueTypeForSqlTypeName(sqlTypeName); + if (outputType == null) { + throw new ISE("Cannot translate sqlTypeName[%s] to Druid type for field[%s]", sqlTypeName, name); + } + + final DimensionSpec dimensionSpec = rex.toDimensionSpec(rowSignature, null, ValueType.STRING); if (dimensionSpec == null) { return null; } diff --git a/sql/src/main/java/io/druid/sql/calcite/expression/RowExtraction.java b/sql/src/main/java/io/druid/sql/calcite/expression/RowExtraction.java index 776ada76f712..b8f9b44d579e 100644 --- a/sql/src/main/java/io/druid/sql/calcite/expression/RowExtraction.java +++ b/sql/src/main/java/io/druid/sql/calcite/expression/RowExtraction.java @@ -124,8 +124,14 @@ public boolean isFilterable(final RowSignature rowSignature) return Filters.FILTERABLE_TYPES.contains(rowSignature.getColumnType(column)); } - public DimensionSpec toDimensionSpec(final RowSignature rowSignature, final String outputName) + public DimensionSpec toDimensionSpec( + final RowSignature rowSignature, + final String outputName, + final ValueType outputType + ) { + Preconditions.checkNotNull(outputType, "outputType"); + final ValueType columnType = rowSignature.getColumnType(column); if (columnType == null) { return null; @@ -133,10 +139,16 @@ public DimensionSpec toDimensionSpec(final RowSignature rowSignature, final Stri if (columnType == ValueType.STRING || (column.equals(Column.TIME_COLUMN_NAME) && extractionFn != null)) { return extractionFn == null - ? new DefaultDimensionSpec(column, outputName) - : new ExtractionDimensionSpec(column, outputName, extractionFn); + ? new DefaultDimensionSpec(column, outputName, outputType) + : new ExtractionDimensionSpec(column, outputName, outputType, extractionFn); + } else if (columnType == ValueType.LONG || columnType == ValueType.FLOAT) { + if (extractionFn == null) { + return new DefaultDimensionSpec(column, outputName, outputType); + } else { + return new ExtractionDimensionSpec(column, outputName, outputType, extractionFn); + } } else { - // Can't create dimensionSpecs for non-string, non-time. + // Can't create dimensionSpecs for non-string, non-numeric columns return null; } } diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/Calcites.java b/sql/src/main/java/io/druid/sql/calcite/planner/Calcites.java index f56c53c0b20c..7133c0186cf2 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/Calcites.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/Calcites.java @@ -27,6 +27,7 @@ import com.google.common.primitives.Chars; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; +import io.druid.segment.column.ValueType; import io.druid.sql.calcite.rel.DruidConvention; import io.druid.sql.calcite.rel.DruidRel; import io.druid.sql.calcite.schema.DruidSchema; @@ -152,6 +153,21 @@ public static PlannerResult plan( } } + public static ValueType getValueTypeForSqlTypeName(SqlTypeName sqlTypeName) + { + if (SqlTypeName.APPROX_TYPES.contains(sqlTypeName)) { + return ValueType.FLOAT; + } else if (SqlTypeName.DATETIME_TYPES.contains(sqlTypeName) || SqlTypeName.EXACT_TYPES.contains(sqlTypeName)) { + return ValueType.LONG; + } else if (SqlTypeName.CHAR_TYPES.contains(sqlTypeName)) { + return ValueType.STRING; + } else if (SqlTypeName.OTHER == sqlTypeName) { + return ValueType.COMPLEX; + } else { + return null; + } + } + private static PlannerResult planWithDruidConvention( final Planner planner, final SqlExplain explain, diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryBuilder.java b/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryBuilder.java index 38ea175937e7..0c7c697fe660 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryBuilder.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryBuilder.java @@ -47,6 +47,7 @@ import io.druid.segment.column.ValueType; import io.druid.sql.calcite.expression.ExtractionFns; import io.druid.sql.calcite.filtration.Filtration; +import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.table.RowSignature; import org.apache.calcite.plan.RelTrait; import org.apache.calcite.rel.RelCollations; @@ -97,15 +98,8 @@ private DruidQueryBuilder( final SqlTypeName sqlTypeName = field.getType().getSqlTypeName(); final ValueType valueType; - if (SqlTypeName.APPROX_TYPES.contains(sqlTypeName)) { - valueType = ValueType.FLOAT; - } else if (SqlTypeName.DATETIME_TYPES.contains(sqlTypeName) || SqlTypeName.EXACT_TYPES.contains(sqlTypeName)) { - valueType = ValueType.LONG; - } else if (SqlTypeName.CHAR_TYPES.contains(sqlTypeName)) { - valueType = ValueType.STRING; - } else if (SqlTypeName.OTHER == sqlTypeName) { - valueType = ValueType.COMPLEX; - } else { + valueType = Calcites.getValueTypeForSqlTypeName(sqlTypeName); + if (valueType == null) { throw new ISE("Cannot translate sqlTypeName[%s] to Druid type for field[%s]", sqlTypeName, rowOrder.get(i)); } diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java index 12c09fe46141..0eb83ec0c14c 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java @@ -359,6 +359,8 @@ private static Object coerce(final Object value, final SqlTypeName sqlType) coercedValue = Strings.nullToEmpty((String) value); } else if (value instanceof NlsString) { coercedValue = ((NlsString) value).getValue(); + } else if (value instanceof Number) { + coercedValue = String.valueOf(value); } else { throw new ISE("Cannot coerce[%s] to %s", value.getClass().getName(), sqlType); } diff --git a/sql/src/main/java/io/druid/sql/calcite/rule/GroupByRules.java b/sql/src/main/java/io/druid/sql/calcite/rule/GroupByRules.java index 14fd569eec5d..0da5bce62f7d 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rule/GroupByRules.java +++ b/sql/src/main/java/io/druid/sql/calcite/rule/GroupByRules.java @@ -44,6 +44,7 @@ import io.druid.query.groupby.orderby.OrderByColumnSpec; import io.druid.query.ordering.StringComparator; import io.druid.query.ordering.StringComparators; +import io.druid.segment.column.ValueType; import io.druid.sql.calcite.aggregation.Aggregation; import io.druid.sql.calcite.aggregation.ApproxCountDistinctSqlAggregator; import io.druid.sql.calcite.aggregation.PostAggregatorFactory; @@ -51,6 +52,7 @@ import io.druid.sql.calcite.expression.Expressions; import io.druid.sql.calcite.expression.RowExtraction; import io.druid.sql.calcite.filtration.Filtration; +import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.rel.DruidNestedGroupBy; @@ -437,17 +439,25 @@ private static DruidRel applyAggregate( // nobody actually expects to see the literal. rowOrder.add(dimOutputName(dimOutputNameCounter++)); } else { + final RexNode rexNode = Expressions.fromFieldAccess(sourceRowSignature, project, i); final RowExtraction rex = Expressions.toRowExtraction( sourceRowSignature.getRowOrder(), - Expressions.fromFieldAccess(sourceRowSignature, project, i) + rexNode ); if (rex == null) { return null; } + final SqlTypeName sqlTypeName = rexNode.getType().getSqlTypeName(); + final ValueType outputType = Calcites.getValueTypeForSqlTypeName(sqlTypeName); + if (outputType == null) { + throw new ISE("Cannot translate sqlTypeName[%s] to Druid type for field[%s]", sqlTypeName, rowOrder.get(i)); + } + final DimensionSpec dimensionSpec = rex.toDimensionSpec( sourceRowSignature, - dimOutputName(dimOutputNameCounter++) + dimOutputName(dimOutputNameCounter++), + outputType ); if (dimensionSpec == null) { return null; diff --git a/sql/src/main/java/io/druid/sql/calcite/rule/SelectRules.java b/sql/src/main/java/io/druid/sql/calcite/rule/SelectRules.java index 8f4fd311a2c0..46336c6b0f3c 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rule/SelectRules.java +++ b/sql/src/main/java/io/druid/sql/calcite/rule/SelectRules.java @@ -30,6 +30,7 @@ import io.druid.segment.column.ValueType; import io.druid.sql.calcite.expression.Expressions; import io.druid.sql.calcite.expression.RowExtraction; +import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.rel.DruidRel; import io.druid.sql.calcite.rel.SelectProjection; import io.druid.sql.calcite.table.RowSignature; @@ -37,6 +38,8 @@ import org.apache.calcite.plan.RelOptRuleCall; import org.apache.calcite.rel.core.Project; import org.apache.calcite.rel.core.Sort; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.type.SqlTypeName; import java.util.List; @@ -88,9 +91,10 @@ public void onMatch(RelOptRuleCall call) int dimOutputNameCounter = 0; for (int i = 0; i < project.getRowType().getFieldCount(); i++) { + final RexNode rexNode = project.getChildExps().get(i); final RowExtraction rex = Expressions.toRowExtraction( sourceRowSignature.getRowOrder(), - project.getChildExps().get(i) + rexNode ); if (rex == null) { @@ -109,7 +113,12 @@ public void onMatch(RelOptRuleCall call) dimOutputNameCounter++; } while (sourceRowSignature.getColumnType(GroupByRules.dimOutputName(dimOutputNameCounter)) != null); final String outputName = GroupByRules.dimOutputName(dimOutputNameCounter); - final DimensionSpec dimensionSpec = rex.toDimensionSpec(sourceRowSignature, outputName); + final SqlTypeName sqlTypeName = rexNode.getType().getSqlTypeName(); + final ValueType outputType = Calcites.getValueTypeForSqlTypeName(sqlTypeName); + if (outputType == null) { + throw new ISE("Cannot translate sqlTypeName[%s] to Druid type for field[%s]", sqlTypeName, outputName); + } + final DimensionSpec dimensionSpec = rex.toDimensionSpec(sourceRowSignature, outputName, columnType); if (dimensionSpec == null) { // Really should have been possible due to the checks above. diff --git a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java index 156ddea40850..ae38cf1f9dcf 100644 --- a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java @@ -77,6 +77,7 @@ import io.druid.query.topn.NumericTopNMetricSpec; import io.druid.query.topn.TopNQueryBuilder; import io.druid.segment.column.Column; +import io.druid.segment.column.ValueType; import io.druid.sql.calcite.filtration.Filtration; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; @@ -543,6 +544,110 @@ public void testExplainSelfJoinWithFallback() throws Exception ); } + @Test + public void testGroupByLong() throws Exception + { + testQuery( + "SELECT cnt, COUNT(*) FROM druid.foo GROUP BY cnt", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(QSS(Filtration.eternity())) + .setGranularity(QueryGranularities.ALL) + .setDimensions(DIMS(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG))) + .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0"))) + .build() + ), + ImmutableList.of( + new Object[]{1L, 6L} + ) + ); + } + + @Test + public void testGroupByFloat() throws Exception + { + testQuery( + "SELECT m1, COUNT(*) FROM druid.foo GROUP BY m1", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(QSS(Filtration.eternity())) + .setGranularity(QueryGranularities.ALL) + .setDimensions(DIMS(new DefaultDimensionSpec("m1", "d0", ValueType.FLOAT))) + .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0"))) + .build() + ), + ImmutableList.of( + new Object[]{1.0d, 1L}, + new Object[]{2.0d, 1L}, + new Object[]{3.0d, 1L}, + new Object[]{4.0d, 1L}, + new Object[]{5.0d, 1L}, + new Object[]{6.0d, 1L} + ) + ); + } + + @Test + public void testFilterOnFloat() throws Exception + { + testQuery( + "SELECT COUNT(*) FROM druid.foo WHERE m1 = 1.0", + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(QSS(Filtration.eternity())) + .granularity(QueryGranularities.ALL) + .aggregators(AGGS(new CountAggregatorFactory("a0"))) + .filters(SELECTOR("m1", "1.0", null)) + .context(TIMESERIES_CONTEXT) + .build() + ), + ImmutableList.of( + new Object[]{1L} + ) + ); + } + + @Test + public void testHavingOnFloat() throws Exception + { + testQuery( + "SELECT dim1, SUM(m1) AS m1_sum FROM druid.foo GROUP BY dim1 HAVING SUM(m1) > 1", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(QSS(Filtration.eternity())) + .setGranularity(QueryGranularities.ALL) + .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0"))) + .setAggregatorSpecs(AGGS(new DoubleSumAggregatorFactory("a0", "m1"))) + .setHavingSpec( + new DimFilterHavingSpec( + new BoundDimFilter( + "a0", + "1", + null, + true, + false, + false, + null, + StringComparators.NUMERIC + ) + ) + ) + .build() + ), + ImmutableList.of( + new Object[]{"1", 4.0d}, + new Object[]{"10.1", 2.0d}, + new Object[]{"2", 3.0d}, + new Object[]{"abc", 6.0d}, + new Object[]{"def", 5.0d} + ) + ); + } + @Test public void testUnplannableQueries() throws Exception { @@ -554,9 +659,6 @@ public void testUnplannableQueries() throws Exception "SELECT (dim1 || ' ' || dim2) AS cc, COUNT(*) FROM druid.foo GROUP BY dim1 || ' ' || dim2", // Concat two dims "SELECT dim1 FROM druid.foo ORDER BY dim1", // SELECT query with order by "SELECT TRIM(dim1) FROM druid.foo", // TRIM function - "SELECT cnt, COUNT(*) FROM druid.foo GROUP BY cnt", // GROUP BY long - "SELECT m1, COUNT(*) FROM druid.foo GROUP BY m1", // GROUP BY float - "SELECT COUNT(*) FROM druid.foo WHERE m1 = 1.0", // Filter on float "SELECT COUNT(*) FROM druid.foo WHERE dim1 = dim2", // Filter on two columns equaling each other "SELECT COUNT(*) FROM druid.foo WHERE CHARACTER_LENGTH(dim1) = CHARACTER_LENGTH(dim2)", // Similar to above "SELECT CHARACTER_LENGTH(dim1) + 1 FROM druid.foo GROUP BY CHARACTER_LENGTH(dim1) + 1", // Group by math @@ -566,7 +668,6 @@ public void testUnplannableQueries() throws Exception + " SUM(cnt) AS cnt\n" + "FROM druid.foo\n" + "GROUP BY (CAST(__time AS DATE) + EXTRACT(HOUR FROM __time) * INTERVAL '1' HOUR)", // Time arithmetic - "SELECT dim1, SUM(m1) AS m1_sum FROM druid.foo GROUP BY dim1 HAVING SUM(m1) > 1", // HAVING on float "SELECT SUBSTRING(dim1, 2) FROM druid.foo GROUP BY dim1", // Project a dimension from GROUP BY "SELECT dim1 FROM druid.foo GROUP BY dim1 ORDER BY SUBSTRING(dim1, 2)" // ORDER BY projection ); @@ -2149,15 +2250,12 @@ public void testHistogramUsingSubqueryWithSort() throws Exception .setGranularity(QueryGranularities.ALL) .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0"))) .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) - .setPostAggregatorSpecs(ImmutableList.of( - new FieldAccessPostAggregator("a1", "a0") - )) .build() ) ) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) - .setDimensions(DIMS(new DefaultDimensionSpec("a1", "d0"))) + .setDimensions(DIMS(new DefaultDimensionSpec("a0", "d0"))) .setAggregatorSpecs(AGGS( new CountAggregatorFactory("a0") )) @@ -2411,16 +2509,16 @@ public void testGroupByFloor() throws Exception .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions(DIMS( - new ExtractionDimensionSpec("dim1", "d0", new BucketExtractionFn(1.0, 0.0)) + new ExtractionDimensionSpec("dim1", "d0", ValueType.FLOAT, new BucketExtractionFn(1.0, 0.0)) )) .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0"))) .build() ), ImmutableList.of( - new Object[]{null, 3L}, + new Object[]{0.0, 3L}, new Object[]{1.0, 1L}, - new Object[]{10.0, 1L}, - new Object[]{2.0, 1L} + new Object[]{2.0, 1L}, + new Object[]{10.0, 1L} ) ); } @@ -2437,7 +2535,7 @@ public void testGroupByFloorWithOrderBy() throws Exception .setGranularity(QueryGranularities.ALL) .setDimensions( DIMS( - new ExtractionDimensionSpec("dim1", "d0", new BucketExtractionFn(1.0, 0.0)) + new ExtractionDimensionSpec("dim1", "d0", ValueType.FLOAT, new BucketExtractionFn(1.0, 0.0)) ) ) .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0"))) @@ -2459,7 +2557,7 @@ public void testGroupByFloorWithOrderBy() throws Exception new Object[]{10.0, 1L}, new Object[]{2.0, 1L}, new Object[]{1.0, 1L}, - new Object[]{null, 3L} + new Object[]{0.0, 3L} ) ); } @@ -2482,6 +2580,7 @@ public void testGroupByFloorTimeAndOneOtherDimensionWithOrderBy() throws Excepti new ExtractionDimensionSpec( "__time", "d0", + ValueType.LONG, new TimeFormatExtractionFn(null, null, null, QueryGranularities.YEAR, true) ), new DefaultDimensionSpec("dim2", "d1") @@ -2541,6 +2640,7 @@ public void testGroupByStringLength() throws Exception new ExtractionDimensionSpec( "dim1", "d0", + ValueType.LONG, StrlenExtractionFn.instance() ) ) @@ -2634,6 +2734,7 @@ public void testGroupByExtractYear() throws Exception new ExtractionDimensionSpec( "__time", "d0", + ValueType.LONG, new TimeFormatExtractionFn("Y", null, null, QueryGranularities.NONE, true) ) ) @@ -2678,6 +2779,7 @@ public void testExtractFloorTime() throws Exception new ExtractionDimensionSpec( "__time", "d0", + ValueType.LONG, CASCADE( new TimeFormatExtractionFn(null, null, null, QueryGranularities.YEAR, true), new TimeFormatExtractionFn("Y", null, null, QueryGranularities.NONE, true) @@ -2718,6 +2820,7 @@ public void testTimeseriesWithLimitNoTopN() throws Exception new ExtractionDimensionSpec( "__time", "d0", + ValueType.LONG, new TimeFormatExtractionFn(null, null, null, QueryGranularities.MONTH, true) ) ) @@ -2764,6 +2867,7 @@ public void testTimeseriesWithLimit() throws Exception new ExtractionDimensionSpec( "__time", "d0", + ValueType.LONG, new TimeFormatExtractionFn(null, null, null, QueryGranularities.MONTH, true) ) ) @@ -2797,6 +2901,7 @@ public void testGroupByTimeAndOtherDimension() throws Exception new ExtractionDimensionSpec( "__time", "d0", + ValueType.LONG, new TimeFormatExtractionFn(null, null, null, QueryGranularities.MONTH, true) ) )