mirror of https://github.com/apache/druid.git
timeseries for druid + unit tests
This commit is contained in:
parent
69b929fad5
commit
499263386c
|
@ -20,6 +20,8 @@
|
||||||
package com.metamx.druid;
|
package com.metamx.druid;
|
||||||
|
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
|
import com.metamx.druid.aggregation.post.PostAggregator;
|
||||||
import com.metamx.druid.query.filter.AndDimFilter;
|
import com.metamx.druid.query.filter.AndDimFilter;
|
||||||
import com.metamx.druid.query.filter.DimFilter;
|
import com.metamx.druid.query.filter.DimFilter;
|
||||||
import com.metamx.druid.query.filter.NoopDimFilter;
|
import com.metamx.druid.query.filter.NoopDimFilter;
|
||||||
|
@ -32,6 +34,7 @@ import com.metamx.druid.query.search.SearchQuerySpec;
|
||||||
import com.metamx.druid.query.segment.LegacySegmentSpec;
|
import com.metamx.druid.query.segment.LegacySegmentSpec;
|
||||||
import com.metamx.druid.query.segment.QuerySegmentSpec;
|
import com.metamx.druid.query.segment.QuerySegmentSpec;
|
||||||
import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
|
import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
|
||||||
|
import com.metamx.druid.query.timeseries.TimeseriesQuery;
|
||||||
import com.metamx.druid.result.Result;
|
import com.metamx.druid.result.Result;
|
||||||
import com.metamx.druid.result.SearchResultValue;
|
import com.metamx.druid.result.SearchResultValue;
|
||||||
import com.metamx.druid.result.TimeBoundaryResultValue;
|
import com.metamx.druid.result.TimeBoundaryResultValue;
|
||||||
|
@ -276,6 +279,197 @@ public class Druids
|
||||||
return new NoopDimFilterBuilder();
|
return new NoopDimFilterBuilder();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A Builder for TimeseriesQuery.
|
||||||
|
* <p/>
|
||||||
|
* Required: dataSource(), intervals(), and aggregators() must be called before build()
|
||||||
|
* Optional: filters(), granularity(), postAggregators(), and context() can be called before build()
|
||||||
|
* <p/>
|
||||||
|
* Usage example:
|
||||||
|
* <pre><code>
|
||||||
|
* TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
* .dataSource("Example")
|
||||||
|
* .intervals("2012-01-01/2012-01-02")
|
||||||
|
* .aggregators(listofAggregators)
|
||||||
|
* .build();
|
||||||
|
* </code></pre>
|
||||||
|
*
|
||||||
|
* @see com.metamx.druid.query.timeseries.TimeseriesQuery
|
||||||
|
*/
|
||||||
|
public static class TimeseriesQueryBuilder
|
||||||
|
{
|
||||||
|
private String dataSource;
|
||||||
|
private QuerySegmentSpec querySegmentSpec;
|
||||||
|
private DimFilter dimFilter;
|
||||||
|
private QueryGranularity granularity;
|
||||||
|
private List<AggregatorFactory> aggregatorSpecs;
|
||||||
|
private List<PostAggregator> postAggregatorSpecs;
|
||||||
|
private Map<String, String> context;
|
||||||
|
|
||||||
|
private TimeseriesQueryBuilder()
|
||||||
|
{
|
||||||
|
dataSource = "";
|
||||||
|
querySegmentSpec = null;
|
||||||
|
dimFilter = null;
|
||||||
|
granularity = QueryGranularity.ALL;
|
||||||
|
aggregatorSpecs = Lists.newArrayList();
|
||||||
|
postAggregatorSpecs = Lists.newArrayList();
|
||||||
|
context = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQuery build()
|
||||||
|
{
|
||||||
|
return new TimeseriesQuery(
|
||||||
|
dataSource,
|
||||||
|
querySegmentSpec,
|
||||||
|
dimFilter,
|
||||||
|
granularity,
|
||||||
|
aggregatorSpecs,
|
||||||
|
postAggregatorSpecs,
|
||||||
|
context
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQueryBuilder copy(TimeseriesQuery query)
|
||||||
|
{
|
||||||
|
return new TimeseriesQueryBuilder()
|
||||||
|
.dataSource(query.getDataSource())
|
||||||
|
.intervals(query.getIntervals())
|
||||||
|
.filters(query.getDimensionsFilter())
|
||||||
|
.granularity(query.getGranularity())
|
||||||
|
.aggregators(query.getAggregatorSpecs())
|
||||||
|
.postAggregators(query.getPostAggregatorSpecs())
|
||||||
|
.context(query.getContext());
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQueryBuilder copy(TimeseriesQueryBuilder builder)
|
||||||
|
{
|
||||||
|
return new TimeseriesQueryBuilder()
|
||||||
|
.dataSource(builder.dataSource)
|
||||||
|
.intervals(builder.querySegmentSpec)
|
||||||
|
.filters(builder.dimFilter)
|
||||||
|
.granularity(builder.granularity)
|
||||||
|
.aggregators(builder.aggregatorSpecs)
|
||||||
|
.postAggregators(builder.postAggregatorSpecs)
|
||||||
|
.context(builder.context);
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getDataSource()
|
||||||
|
{
|
||||||
|
return dataSource;
|
||||||
|
}
|
||||||
|
|
||||||
|
public QuerySegmentSpec getQuerySegmentSpec()
|
||||||
|
{
|
||||||
|
return querySegmentSpec;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DimFilter getDimFilter()
|
||||||
|
{
|
||||||
|
return dimFilter;
|
||||||
|
}
|
||||||
|
|
||||||
|
public QueryGranularity getGranularity()
|
||||||
|
{
|
||||||
|
return granularity;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<AggregatorFactory> getAggregatorSpecs()
|
||||||
|
{
|
||||||
|
return aggregatorSpecs;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<PostAggregator> getPostAggregatorSpecs()
|
||||||
|
{
|
||||||
|
return postAggregatorSpecs;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Map<String, String> getContext()
|
||||||
|
{
|
||||||
|
return context;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQueryBuilder dataSource(String ds)
|
||||||
|
{
|
||||||
|
dataSource = ds;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQueryBuilder intervals(QuerySegmentSpec q)
|
||||||
|
{
|
||||||
|
querySegmentSpec = q;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQueryBuilder intervals(String s)
|
||||||
|
{
|
||||||
|
querySegmentSpec = new LegacySegmentSpec(s);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQueryBuilder intervals(List<Interval> l)
|
||||||
|
{
|
||||||
|
querySegmentSpec = new LegacySegmentSpec(l);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQueryBuilder filters(String dimensionName, String value)
|
||||||
|
{
|
||||||
|
dimFilter = new SelectorDimFilter(dimensionName, value);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQueryBuilder filters(String dimensionName, String value, String... values)
|
||||||
|
{
|
||||||
|
List<DimFilter> fields = Lists.<DimFilter>newArrayList(new SelectorDimFilter(dimensionName, value));
|
||||||
|
for (String val : values) {
|
||||||
|
fields.add(new SelectorDimFilter(dimensionName, val));
|
||||||
|
}
|
||||||
|
dimFilter = new OrDimFilter(fields);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQueryBuilder filters(DimFilter f)
|
||||||
|
{
|
||||||
|
dimFilter = f;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQueryBuilder granularity(String g)
|
||||||
|
{
|
||||||
|
granularity = QueryGranularity.fromString(g);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQueryBuilder granularity(QueryGranularity g)
|
||||||
|
{
|
||||||
|
granularity = g;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQueryBuilder aggregators(List<AggregatorFactory> a)
|
||||||
|
{
|
||||||
|
aggregatorSpecs = a;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQueryBuilder postAggregators(List<PostAggregator> p)
|
||||||
|
{
|
||||||
|
postAggregatorSpecs = p;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQueryBuilder context(Map<String, String> c)
|
||||||
|
{
|
||||||
|
context = c;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static TimeseriesQueryBuilder newTimeseriesQueryBuilder()
|
||||||
|
{
|
||||||
|
return new TimeseriesQueryBuilder();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A Builder for SearchQuery.
|
* A Builder for SearchQuery.
|
||||||
|
|
|
@ -0,0 +1,91 @@
|
||||||
|
/*
|
||||||
|
* Druid - a distributed column store.
|
||||||
|
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||||
|
*
|
||||||
|
* This program is free software; you can redistribute it and/or
|
||||||
|
* modify it under the terms of the GNU General Public License
|
||||||
|
* as published by the Free Software Foundation; either version 2
|
||||||
|
* of the License, or (at your option) any later version.
|
||||||
|
*
|
||||||
|
* This program is distributed in the hope that it will be useful,
|
||||||
|
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||||
|
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||||
|
* GNU General Public License for more details.
|
||||||
|
*
|
||||||
|
* You should have received a copy of the GNU General Public License
|
||||||
|
* along with this program; if not, write to the Free Software
|
||||||
|
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package com.metamx.druid;
|
||||||
|
|
||||||
|
import com.metamx.common.guava.nary.BinaryFn;
|
||||||
|
import com.metamx.druid.AllGranularity;
|
||||||
|
import com.metamx.druid.QueryGranularity;
|
||||||
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
|
import com.metamx.druid.aggregation.post.PostAggregator;
|
||||||
|
import com.metamx.druid.result.Result;
|
||||||
|
import com.metamx.druid.result.TimeseriesResultValue;
|
||||||
|
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class TimeseriesBinaryFn
|
||||||
|
implements BinaryFn<Result<TimeseriesResultValue>, Result<TimeseriesResultValue>, Result<TimeseriesResultValue>>
|
||||||
|
{
|
||||||
|
private final QueryGranularity gran;
|
||||||
|
private final List<AggregatorFactory> aggregations;
|
||||||
|
private final List<PostAggregator> postAggregations;
|
||||||
|
|
||||||
|
public TimeseriesBinaryFn(
|
||||||
|
QueryGranularity granularity,
|
||||||
|
List<AggregatorFactory> aggregations,
|
||||||
|
List<PostAggregator> postAggregations
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.gran = granularity;
|
||||||
|
this.aggregations = aggregations;
|
||||||
|
this.postAggregations = postAggregations;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Result<TimeseriesResultValue> apply(Result<TimeseriesResultValue> arg1, Result<TimeseriesResultValue> arg2)
|
||||||
|
{
|
||||||
|
if (arg1 == null) {
|
||||||
|
return arg2;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (arg2 == null) {
|
||||||
|
return arg1;
|
||||||
|
}
|
||||||
|
|
||||||
|
TimeseriesResultValue arg1Val = arg1.getValue();
|
||||||
|
TimeseriesResultValue arg2Val = arg2.getValue();
|
||||||
|
|
||||||
|
Map<String, Object> retVal = new LinkedHashMap<String, Object>();
|
||||||
|
|
||||||
|
for (AggregatorFactory factory : aggregations) {
|
||||||
|
final String metricName = factory.getName();
|
||||||
|
retVal.put(metricName, factory.combine(arg1Val.getMetric(metricName), arg2Val.getMetric(metricName)));
|
||||||
|
}
|
||||||
|
|
||||||
|
for (PostAggregator pf : postAggregations) {
|
||||||
|
final String metricName = pf.getName();
|
||||||
|
retVal.put(metricName, pf.compute(retVal));
|
||||||
|
}
|
||||||
|
|
||||||
|
return (gran instanceof AllGranularity) ?
|
||||||
|
new Result<TimeseriesResultValue>(
|
||||||
|
arg1.getTimestamp(),
|
||||||
|
new TimeseriesResultValue(retVal)
|
||||||
|
) :
|
||||||
|
new Result<TimeseriesResultValue>(
|
||||||
|
gran.toDateTime(gran.truncate(arg1.getTimestamp().getMillis())),
|
||||||
|
new TimeseriesResultValue(retVal)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,150 @@
|
||||||
|
/*
|
||||||
|
* Druid - a distributed column store.
|
||||||
|
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||||
|
*
|
||||||
|
* This program is free software; you can redistribute it and/or
|
||||||
|
* modify it under the terms of the GNU General Public License
|
||||||
|
* as published by the Free Software Foundation; either version 2
|
||||||
|
* of the License, or (at your option) any later version.
|
||||||
|
*
|
||||||
|
* This program is distributed in the hope that it will be useful,
|
||||||
|
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||||
|
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||||
|
* GNU General Public License for more details.
|
||||||
|
*
|
||||||
|
* You should have received a copy of the GNU General Public License
|
||||||
|
* along with this program; if not, write to the Free Software
|
||||||
|
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package com.metamx.druid.query.timeseries;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import com.metamx.druid.BaseQuery;
|
||||||
|
import com.metamx.druid.Query;
|
||||||
|
import com.metamx.druid.QueryGranularity;
|
||||||
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
|
import com.metamx.druid.aggregation.post.PostAggregator;
|
||||||
|
import com.metamx.druid.query.filter.DimFilter;
|
||||||
|
import com.metamx.druid.query.segment.QuerySegmentSpec;
|
||||||
|
import com.metamx.druid.result.Result;
|
||||||
|
import com.metamx.druid.result.TimeseriesResultValue;
|
||||||
|
import org.codehaus.jackson.annotate.JsonCreator;
|
||||||
|
import org.codehaus.jackson.annotate.JsonProperty;
|
||||||
|
import org.codehaus.jackson.annotate.JsonSubTypes;
|
||||||
|
import org.codehaus.jackson.annotate.JsonTypeName;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
@JsonTypeName("timeseries")
|
||||||
|
public class TimeseriesQuery extends BaseQuery<Result<TimeseriesResultValue>>
|
||||||
|
{
|
||||||
|
public static final String TIMESERIES = "timeseries";
|
||||||
|
|
||||||
|
private final DimFilter dimFilter;
|
||||||
|
private final QueryGranularity granularity;
|
||||||
|
private final List<AggregatorFactory> aggregatorSpecs;
|
||||||
|
private final List<PostAggregator> postAggregatorSpecs;
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public TimeseriesQuery(
|
||||||
|
@JsonProperty("dataSource") String dataSource,
|
||||||
|
@JsonProperty("intervals") QuerySegmentSpec querySegmentSpec,
|
||||||
|
@JsonProperty("filter") DimFilter dimFilter,
|
||||||
|
@JsonProperty("granularity") QueryGranularity granularity,
|
||||||
|
@JsonProperty("aggregations") List<AggregatorFactory> aggregatorSpecs,
|
||||||
|
@JsonProperty("postAggregations") List<PostAggregator> postAggregatorSpecs,
|
||||||
|
@JsonProperty("context") Map<String, String> context
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(dataSource, querySegmentSpec, context);
|
||||||
|
this.dimFilter = dimFilter;
|
||||||
|
this.granularity = granularity;
|
||||||
|
this.aggregatorSpecs = aggregatorSpecs;
|
||||||
|
this.postAggregatorSpecs = postAggregatorSpecs == null ? ImmutableList.<PostAggregator>of() : postAggregatorSpecs;
|
||||||
|
|
||||||
|
Preconditions.checkNotNull(aggregatorSpecs, "aggregations cannot be null");
|
||||||
|
Preconditions.checkArgument(aggregatorSpecs.size() > 0, "Must have at least one AggregatorFactory");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasFilters()
|
||||||
|
{
|
||||||
|
return dimFilter != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getType()
|
||||||
|
{
|
||||||
|
return TIMESERIES;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty("filter")
|
||||||
|
public DimFilter getDimensionsFilter()
|
||||||
|
{
|
||||||
|
return dimFilter;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public QueryGranularity getGranularity()
|
||||||
|
{
|
||||||
|
return granularity;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty("aggregations")
|
||||||
|
public List<AggregatorFactory> getAggregatorSpecs()
|
||||||
|
{
|
||||||
|
return aggregatorSpecs;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty("postAggregations")
|
||||||
|
public List<PostAggregator> getPostAggregatorSpecs()
|
||||||
|
{
|
||||||
|
return postAggregatorSpecs;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQuery withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec)
|
||||||
|
{
|
||||||
|
return new TimeseriesQuery(
|
||||||
|
getDataSource(),
|
||||||
|
querySegmentSpec,
|
||||||
|
dimFilter,
|
||||||
|
granularity,
|
||||||
|
aggregatorSpecs,
|
||||||
|
postAggregatorSpecs,
|
||||||
|
getContext()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesQuery withOverriddenContext(Map<String, String> contextOverrides)
|
||||||
|
{
|
||||||
|
return new TimeseriesQuery(
|
||||||
|
getDataSource(),
|
||||||
|
getQuerySegmentSpec(),
|
||||||
|
dimFilter,
|
||||||
|
granularity,
|
||||||
|
aggregatorSpecs,
|
||||||
|
postAggregatorSpecs,
|
||||||
|
computeOverridenContext(contextOverrides)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "TimeseriesQuery{" +
|
||||||
|
"dataSource='" + getDataSource() + '\'' +
|
||||||
|
", querySegmentSpec=" + getQuerySegmentSpec() +
|
||||||
|
", dimFilter=" + dimFilter +
|
||||||
|
", granularity='" + granularity + '\'' +
|
||||||
|
", aggregatorSpecs=" + aggregatorSpecs +
|
||||||
|
", postAggregatorSpecs=" + postAggregatorSpecs +
|
||||||
|
", context=" + getContext() +
|
||||||
|
'}';
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,262 @@
|
||||||
|
/*
|
||||||
|
* Druid - a distributed column store.
|
||||||
|
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||||
|
*
|
||||||
|
* This program is free software; you can redistribute it and/or
|
||||||
|
* modify it under the terms of the GNU General Public License
|
||||||
|
* as published by the Free Software Foundation; either version 2
|
||||||
|
* of the License, or (at your option) any later version.
|
||||||
|
*
|
||||||
|
* This program is distributed in the hope that it will be useful,
|
||||||
|
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||||
|
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||||
|
* GNU General Public License for more details.
|
||||||
|
*
|
||||||
|
* You should have received a copy of the GNU General Public License
|
||||||
|
* along with this program; if not, write to the Free Software
|
||||||
|
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package com.metamx.druid.query.timeseries;
|
||||||
|
|
||||||
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.base.Joiner;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
|
import com.google.common.collect.Ordering;
|
||||||
|
import com.metamx.common.guava.MergeSequence;
|
||||||
|
import com.metamx.common.guava.Sequence;
|
||||||
|
import com.metamx.common.guava.nary.BinaryFn;
|
||||||
|
import com.metamx.druid.Query;
|
||||||
|
import com.metamx.druid.ResultGranularTimestampComparator;
|
||||||
|
import com.metamx.druid.TimeseriesBinaryFn;
|
||||||
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
|
import com.metamx.druid.aggregation.post.PostAggregator;
|
||||||
|
import com.metamx.druid.collect.OrderedMergeSequence;
|
||||||
|
import com.metamx.druid.query.CacheStrategy;
|
||||||
|
import com.metamx.druid.query.IntervalChunkingQueryRunner;
|
||||||
|
import com.metamx.druid.query.MetricManipulationFn;
|
||||||
|
import com.metamx.druid.query.QueryCacheHelper;
|
||||||
|
import com.metamx.druid.query.QueryRunner;
|
||||||
|
import com.metamx.druid.query.QueryToolChest;
|
||||||
|
import com.metamx.druid.query.ResultMergeQueryRunner;
|
||||||
|
import com.metamx.druid.query.filter.DimFilter;
|
||||||
|
import com.metamx.druid.result.Result;
|
||||||
|
import com.metamx.druid.result.TimeseriesResultValue;
|
||||||
|
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||||
|
import org.codehaus.jackson.type.TypeReference;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
import org.joda.time.Minutes;
|
||||||
|
import org.joda.time.Period;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class TimeseriesQueryQueryToolChest implements QueryToolChest<Result<TimeseriesResultValue>, TimeseriesQuery>
|
||||||
|
{
|
||||||
|
private static final byte TIMESERIES_QUERY = 0x0;
|
||||||
|
|
||||||
|
private static final Joiner COMMA_JOIN = Joiner.on(",");
|
||||||
|
private static final TypeReference<Result<TimeseriesResultValue>> TYPE_REFERENCE = new TypeReference<Result<TimeseriesResultValue>>()
|
||||||
|
{
|
||||||
|
};
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public QueryRunner<Result<TimeseriesResultValue>> mergeResults(QueryRunner<Result<TimeseriesResultValue>> queryRunner)
|
||||||
|
{
|
||||||
|
return new ResultMergeQueryRunner<Result<TimeseriesResultValue>>(queryRunner)
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
protected Ordering<Result<TimeseriesResultValue>> makeOrdering(Query<Result<TimeseriesResultValue>> query)
|
||||||
|
{
|
||||||
|
return Ordering.from(
|
||||||
|
new ResultGranularTimestampComparator<TimeseriesResultValue>(
|
||||||
|
((TimeseriesQuery) query).getGranularity()
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected BinaryFn<Result<TimeseriesResultValue>, Result<TimeseriesResultValue>, Result<TimeseriesResultValue>> createMergeFn(
|
||||||
|
Query<Result<TimeseriesResultValue>> input
|
||||||
|
)
|
||||||
|
{
|
||||||
|
TimeseriesQuery query = (TimeseriesQuery) input;
|
||||||
|
return new TimeseriesBinaryFn(
|
||||||
|
query.getGranularity(),
|
||||||
|
query.getAggregatorSpecs(),
|
||||||
|
query.getPostAggregatorSpecs()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Sequence<Result<TimeseriesResultValue>> mergeSequences(Sequence<Sequence<Result<TimeseriesResultValue>>> seqOfSequences)
|
||||||
|
{
|
||||||
|
return new OrderedMergeSequence<Result<TimeseriesResultValue>>(
|
||||||
|
getOrdering(),
|
||||||
|
seqOfSequences
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ServiceMetricEvent.Builder makeMetricBuilder(TimeseriesQuery query)
|
||||||
|
{
|
||||||
|
int numMinutes = 0;
|
||||||
|
for (Interval interval : query.getIntervals()) {
|
||||||
|
numMinutes += Minutes.minutesIn(interval).getMinutes();
|
||||||
|
}
|
||||||
|
|
||||||
|
return new ServiceMetricEvent.Builder()
|
||||||
|
.setUser2(query.getDataSource())
|
||||||
|
.setUser4("timeseries")
|
||||||
|
.setUser5(COMMA_JOIN.join(query.getIntervals()))
|
||||||
|
.setUser6(String.valueOf(query.hasFilters()))
|
||||||
|
.setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size()))
|
||||||
|
.setUser9(Minutes.minutes(numMinutes).toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Function<Result<TimeseriesResultValue>, Result<TimeseriesResultValue>> makeMetricManipulatorFn(
|
||||||
|
final TimeseriesQuery query, final MetricManipulationFn fn
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return new Function<Result<TimeseriesResultValue>, Result<TimeseriesResultValue>>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Result<TimeseriesResultValue> apply(Result<TimeseriesResultValue> result)
|
||||||
|
{
|
||||||
|
final Map<String, Object> values = Maps.newHashMap();
|
||||||
|
final TimeseriesResultValue holder = result.getValue();
|
||||||
|
for (AggregatorFactory agg : query.getAggregatorSpecs()) {
|
||||||
|
values.put(agg.getName(), fn.manipulate(agg, holder.getMetric(agg.getName())));
|
||||||
|
}
|
||||||
|
for (PostAggregator postAgg : query.getPostAggregatorSpecs()) {
|
||||||
|
values.put(postAgg.getName(), holder.getMetric(postAgg.getName()));
|
||||||
|
}
|
||||||
|
return new Result<TimeseriesResultValue>(
|
||||||
|
result.getTimestamp(),
|
||||||
|
new TimeseriesResultValue(values)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TypeReference<Result<TimeseriesResultValue>> getResultTypeReference()
|
||||||
|
{
|
||||||
|
return TYPE_REFERENCE;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CacheStrategy<Result<TimeseriesResultValue>, TimeseriesQuery> getCacheStrategy(final TimeseriesQuery query)
|
||||||
|
{
|
||||||
|
return new CacheStrategy<Result<TimeseriesResultValue>, TimeseriesQuery>()
|
||||||
|
{
|
||||||
|
private final List<AggregatorFactory> aggs = query.getAggregatorSpecs();
|
||||||
|
private final List<PostAggregator> postAggs = query.getPostAggregatorSpecs();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] computeCacheKey(TimeseriesQuery query)
|
||||||
|
{
|
||||||
|
final DimFilter dimFilter = query.getDimensionsFilter();
|
||||||
|
final byte[] filterBytes = dimFilter == null ? new byte[]{} : dimFilter.getCacheKey();
|
||||||
|
final byte[] aggregatorBytes = QueryCacheHelper.computeAggregatorBytes(query.getAggregatorSpecs());
|
||||||
|
final byte[] granularityBytes = query.getGranularity().cacheKey();
|
||||||
|
|
||||||
|
return ByteBuffer
|
||||||
|
.allocate(1 + granularityBytes.length + filterBytes.length + aggregatorBytes.length)
|
||||||
|
.put(TIMESERIES_QUERY)
|
||||||
|
.put(granularityBytes)
|
||||||
|
.put(filterBytes)
|
||||||
|
.put(aggregatorBytes)
|
||||||
|
.array();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Function<Result<TimeseriesResultValue>, Object> prepareForCache()
|
||||||
|
{
|
||||||
|
return new Function<Result<TimeseriesResultValue>, Object>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Object apply(@Nullable final Result<TimeseriesResultValue> input)
|
||||||
|
{
|
||||||
|
TimeseriesResultValue results = input.getValue();
|
||||||
|
final List<Object> retVal = Lists.newArrayListWithCapacity(1 + aggs.size());
|
||||||
|
|
||||||
|
retVal.add(input.getTimestamp().getMillis());
|
||||||
|
for (AggregatorFactory agg : aggs) {
|
||||||
|
retVal.add(results.getMetric(agg.getName()));
|
||||||
|
}
|
||||||
|
|
||||||
|
return retVal;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Function<Object, Result<TimeseriesResultValue>> pullFromCache()
|
||||||
|
{
|
||||||
|
return new Function<Object, Result<TimeseriesResultValue>>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Result<TimeseriesResultValue> apply(@Nullable Object input)
|
||||||
|
{
|
||||||
|
List<Object> results = (List<Object>) input;
|
||||||
|
Map<String, Object> retVal = Maps.newLinkedHashMap();
|
||||||
|
|
||||||
|
Iterator<AggregatorFactory> aggsIter = aggs.iterator();
|
||||||
|
Iterator<Object> resultIter = results.iterator();
|
||||||
|
|
||||||
|
DateTime timestamp = new DateTime(resultIter.next());
|
||||||
|
while (aggsIter.hasNext() && resultIter.hasNext()) {
|
||||||
|
final AggregatorFactory factory = aggsIter.next();
|
||||||
|
retVal.put(factory.getName(), factory.deserialize(resultIter.next()));
|
||||||
|
}
|
||||||
|
|
||||||
|
for (PostAggregator postAgg : postAggs) {
|
||||||
|
retVal.put(postAgg.getName(), postAgg.compute(retVal));
|
||||||
|
}
|
||||||
|
|
||||||
|
return new Result<TimeseriesResultValue>(
|
||||||
|
timestamp,
|
||||||
|
new TimeseriesResultValue(retVal)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Sequence<Result<TimeseriesResultValue>> mergeSequences(Sequence<Sequence<Result<TimeseriesResultValue>>> seqOfSequences)
|
||||||
|
{
|
||||||
|
return new MergeSequence<Result<TimeseriesResultValue>>(getOrdering(), seqOfSequences);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public QueryRunner<Result<TimeseriesResultValue>> preMergeQueryDecoration(QueryRunner<Result<TimeseriesResultValue>> runner)
|
||||||
|
{
|
||||||
|
return new IntervalChunkingQueryRunner<Result<TimeseriesResultValue>>(runner, Period.months(1));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public QueryRunner<Result<TimeseriesResultValue>> postMergeQueryDecoration(QueryRunner<Result<TimeseriesResultValue>> runner)
|
||||||
|
{
|
||||||
|
return runner;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Ordering<Result<TimeseriesResultValue>> getOrdering()
|
||||||
|
{
|
||||||
|
return Ordering.natural();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,102 @@
|
||||||
|
/*
|
||||||
|
* Druid - a distributed column store.
|
||||||
|
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||||
|
*
|
||||||
|
* This program is free software; you can redistribute it and/or
|
||||||
|
* modify it under the terms of the GNU General Public License
|
||||||
|
* as published by the Free Software Foundation; either version 2
|
||||||
|
* of the License, or (at your option) any later version.
|
||||||
|
*
|
||||||
|
* This program is distributed in the hope that it will be useful,
|
||||||
|
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||||
|
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||||
|
* GNU General Public License for more details.
|
||||||
|
*
|
||||||
|
* You should have received a copy of the GNU General Public License
|
||||||
|
* along with this program; if not, write to the Free Software
|
||||||
|
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package com.metamx.druid.result;
|
||||||
|
|
||||||
|
import org.codehaus.jackson.annotate.JsonCreator;
|
||||||
|
import org.codehaus.jackson.annotate.JsonValue;
|
||||||
|
|
||||||
|
import java.lang.Double;import java.lang.Float;import java.lang.Long;import java.lang.Number;import java.lang.Object;import java.lang.Override;import java.lang.String;import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class MetricValueExtractor
|
||||||
|
{
|
||||||
|
private final Map<String, Object> value;
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public MetricValueExtractor(
|
||||||
|
Map<String, Object> value
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.value = value;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonValue
|
||||||
|
public Map<String, Object> getBaseObject()
|
||||||
|
{
|
||||||
|
return value;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Float getFloatMetric(String name)
|
||||||
|
{
|
||||||
|
final Object retVal = value.get(name);
|
||||||
|
return retVal == null ? null : ((Number) retVal).floatValue();
|
||||||
|
}
|
||||||
|
|
||||||
|
public Double getDoubleMetric(String name)
|
||||||
|
{
|
||||||
|
final Object retVal = value.get(name);
|
||||||
|
return retVal == null ? null : ((Number) retVal).doubleValue();
|
||||||
|
}
|
||||||
|
|
||||||
|
public Long getLongMetric(String name)
|
||||||
|
{
|
||||||
|
final Object retVal = value.get(name);
|
||||||
|
return retVal == null ? null : ((Number) retVal).longValue();
|
||||||
|
}
|
||||||
|
|
||||||
|
public Object getMetric(String name)
|
||||||
|
{
|
||||||
|
return value.get(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o)
|
||||||
|
{
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
MetricValueExtractor that = (MetricValueExtractor) o;
|
||||||
|
|
||||||
|
if (value != null ? !value.equals(that.value) : that.value != null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return value != null ? value.hashCode() : 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "MetricValueExtractor{" +
|
||||||
|
"value=" + value +
|
||||||
|
'}';
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,45 @@
|
||||||
|
/*
|
||||||
|
* Druid - a distributed column store.
|
||||||
|
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||||
|
*
|
||||||
|
* This program is free software; you can redistribute it and/or
|
||||||
|
* modify it under the terms of the GNU General Public License
|
||||||
|
* as published by the Free Software Foundation; either version 2
|
||||||
|
* of the License, or (at your option) any later version.
|
||||||
|
*
|
||||||
|
* This program is distributed in the hope that it will be useful,
|
||||||
|
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||||
|
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||||
|
* GNU General Public License for more details.
|
||||||
|
*
|
||||||
|
* You should have received a copy of the GNU General Public License
|
||||||
|
* along with this program; if not, write to the Free Software
|
||||||
|
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package com.metamx.druid.result;
|
||||||
|
|
||||||
|
import org.codehaus.jackson.annotate.JsonCreator;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class TimeseriesResultValue extends MetricValueExtractor
|
||||||
|
{
|
||||||
|
@JsonCreator
|
||||||
|
public TimeseriesResultValue(
|
||||||
|
Map<String, Object> value
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(value);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "TimeseriesResultValue{" +
|
||||||
|
"value=" + super.toString() +
|
||||||
|
'}';
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,65 @@
|
||||||
|
/*
|
||||||
|
* Druid - a distributed column store.
|
||||||
|
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||||
|
*
|
||||||
|
* This program is free software; you can redistribute it and/or
|
||||||
|
* modify it under the terms of the GNU General Public License
|
||||||
|
* as published by the Free Software Foundation; either version 2
|
||||||
|
* of the License, or (at your option) any later version.
|
||||||
|
*
|
||||||
|
* This program is distributed in the hope that it will be useful,
|
||||||
|
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||||
|
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||||
|
* GNU General Public License for more details.
|
||||||
|
*
|
||||||
|
* You should have received a copy of the GNU General Public License
|
||||||
|
* along with this program; if not, write to the Free Software
|
||||||
|
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package com.metamx.druid;
|
||||||
|
|
||||||
|
import com.metamx.druid.aggregation.Aggregator;
|
||||||
|
import com.metamx.druid.aggregation.post.PostAggregator;
|
||||||
|
import com.metamx.druid.result.Result;
|
||||||
|
import com.metamx.druid.result.TimeseriesResultValue;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class TimeseriesResultBuilder
|
||||||
|
{
|
||||||
|
private final DateTime timestamp;
|
||||||
|
|
||||||
|
private final Map<String, Object> metricValues = new HashMap<String, Object>();
|
||||||
|
|
||||||
|
public TimeseriesResultBuilder(
|
||||||
|
DateTime timestamp
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.timestamp = timestamp;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesResultBuilder addMetric(Aggregator aggregator)
|
||||||
|
{
|
||||||
|
metricValues.put(aggregator.getName(), aggregator.get());
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeseriesResultBuilder addMetric(PostAggregator postAggregator)
|
||||||
|
{
|
||||||
|
metricValues.put(postAggregator.getName(), postAggregator.compute(metricValues));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Result<TimeseriesResultValue> build()
|
||||||
|
{
|
||||||
|
return new Result<TimeseriesResultValue>(
|
||||||
|
timestamp,
|
||||||
|
new TimeseriesResultValue(metricValues)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -44,6 +44,8 @@ import com.metamx.druid.query.search.SearchQuery;
|
||||||
import com.metamx.druid.query.search.SearchQueryRunnerFactory;
|
import com.metamx.druid.query.search.SearchQueryRunnerFactory;
|
||||||
import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
|
import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
|
||||||
import com.metamx.druid.query.timeboundary.TimeBoundaryQueryRunnerFactory;
|
import com.metamx.druid.query.timeboundary.TimeBoundaryQueryRunnerFactory;
|
||||||
|
import com.metamx.druid.query.timeseries.TimeseriesQuery;
|
||||||
|
import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerFactory;
|
||||||
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
||||||
import org.skife.config.ConfigurationObjectFactory;
|
import org.skife.config.ConfigurationObjectFactory;
|
||||||
|
|
||||||
|
@ -94,6 +96,7 @@ public class ServerInit
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Map<Class<? extends Query>, QueryRunnerFactory> queryRunners = Maps.newLinkedHashMap();
|
Map<Class<? extends Query>, QueryRunnerFactory> queryRunners = Maps.newLinkedHashMap();
|
||||||
|
queryRunners.put(TimeseriesQuery.class, new TimeseriesQueryRunnerFactory());
|
||||||
queryRunners.put(GroupByQuery.class, new GroupByQueryRunnerFactory(new GroupByQueryEngine(configFactory.build(GroupByQueryEngineConfig.class), computationBufferPool)));
|
queryRunners.put(GroupByQuery.class, new GroupByQueryRunnerFactory(new GroupByQueryEngine(configFactory.build(GroupByQueryEngineConfig.class), computationBufferPool)));
|
||||||
queryRunners.put(SearchQuery.class, new SearchQueryRunnerFactory());
|
queryRunners.put(SearchQuery.class, new SearchQueryRunnerFactory());
|
||||||
queryRunners.put(TimeBoundaryQuery.class, new TimeBoundaryQueryRunnerFactory());
|
queryRunners.put(TimeBoundaryQuery.class, new TimeBoundaryQueryRunnerFactory());
|
||||||
|
|
|
@ -0,0 +1,63 @@
|
||||||
|
package com.metamx.druid.query;
|
||||||
|
|
||||||
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.metamx.common.guava.FunctionalIterable;
|
||||||
|
import com.metamx.common.logger.Logger;
|
||||||
|
import com.metamx.druid.QueryGranularity;
|
||||||
|
import com.metamx.druid.StorageAdapter;
|
||||||
|
import com.metamx.druid.aggregation.Aggregator;
|
||||||
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
|
import com.metamx.druid.index.brita.Filter;
|
||||||
|
import com.metamx.druid.index.v1.processing.Cursor;
|
||||||
|
import com.metamx.druid.result.Result;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class QueryRunnerFactories
|
||||||
|
{
|
||||||
|
private static final Logger log = new Logger(QueryRunnerFactories.class);
|
||||||
|
|
||||||
|
public static Aggregator[] makeAggregators(Cursor cursor, List<AggregatorFactory> aggregatorSpecs)
|
||||||
|
{
|
||||||
|
Aggregator[] aggregators = new Aggregator[aggregatorSpecs.size()];
|
||||||
|
int aggregatorIndex = 0;
|
||||||
|
for (AggregatorFactory spec : aggregatorSpecs) {
|
||||||
|
aggregators[aggregatorIndex] = spec.factorize(cursor);
|
||||||
|
++aggregatorIndex;
|
||||||
|
}
|
||||||
|
return aggregators;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static <T> Iterable<Result<T>> makeCursorBasedQuery(
|
||||||
|
final StorageAdapter adapter,
|
||||||
|
List<Interval> queryIntervals,
|
||||||
|
Filter filter,
|
||||||
|
QueryGranularity granularity,
|
||||||
|
Function<Cursor, Result<T>> mapFn
|
||||||
|
)
|
||||||
|
{
|
||||||
|
Preconditions.checkArgument(
|
||||||
|
queryIntervals.size() == 1, "Can only handle a single interval, got[%s]", queryIntervals
|
||||||
|
);
|
||||||
|
|
||||||
|
return FunctionalIterable
|
||||||
|
.create(adapter.makeCursors(filter, queryIntervals.get(0), granularity))
|
||||||
|
.transform(
|
||||||
|
new Function<Cursor, Cursor>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Cursor apply(@Nullable Cursor input)
|
||||||
|
{
|
||||||
|
log.debug("Running over cursor[%s]", adapter.getInterval(), input.getTime());
|
||||||
|
return input;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
)
|
||||||
|
.keep(mapFn);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,124 @@
|
||||||
|
package com.metamx.druid.query.timeseries;
|
||||||
|
|
||||||
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import com.metamx.common.ISE;
|
||||||
|
import com.metamx.common.guava.BaseSequence;
|
||||||
|
import com.metamx.common.guava.Sequence;
|
||||||
|
import com.metamx.druid.Query;
|
||||||
|
import com.metamx.druid.SearchResultBuilder;
|
||||||
|
import com.metamx.druid.StorageAdapter;
|
||||||
|
import com.metamx.druid.TimeseriesResultBuilder;
|
||||||
|
import com.metamx.druid.aggregation.Aggregator;
|
||||||
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
|
import com.metamx.druid.aggregation.post.PostAggregator;
|
||||||
|
import com.metamx.druid.index.brita.Filters;
|
||||||
|
import com.metamx.druid.index.v1.processing.Cursor;
|
||||||
|
import com.metamx.druid.query.ChainedExecutionQueryRunner;
|
||||||
|
import com.metamx.druid.query.QueryRunner;
|
||||||
|
import com.metamx.druid.query.QueryRunnerFactories;
|
||||||
|
import com.metamx.druid.query.QueryRunnerFactory;
|
||||||
|
import com.metamx.druid.query.QueryToolChest;
|
||||||
|
import com.metamx.druid.query.group.GroupByQuery;
|
||||||
|
import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
|
||||||
|
import com.metamx.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest;
|
||||||
|
import com.metamx.druid.result.Result;
|
||||||
|
import com.metamx.druid.result.TimeBoundaryResultValue;
|
||||||
|
import com.metamx.druid.result.TimeseriesResultValue;
|
||||||
|
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.concurrent.ExecutorService;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class TimeseriesQueryRunnerFactory
|
||||||
|
implements QueryRunnerFactory<Result<TimeseriesResultValue>, TimeseriesQuery>
|
||||||
|
{
|
||||||
|
private static final TimeseriesQueryQueryToolChest toolChest = new TimeseriesQueryQueryToolChest();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public QueryRunner<Result<TimeseriesResultValue>> createRunner(final StorageAdapter adapter)
|
||||||
|
{
|
||||||
|
return new QueryRunner<Result<TimeseriesResultValue>>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Sequence<Result<TimeseriesResultValue>> run(Query<Result<TimeseriesResultValue>> input)
|
||||||
|
{
|
||||||
|
if (!(input instanceof TimeseriesQuery)) {
|
||||||
|
throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
final TimeseriesQuery query = (TimeseriesQuery) input;
|
||||||
|
|
||||||
|
return new BaseSequence<Result<TimeseriesResultValue>, Iterator<Result<TimeseriesResultValue>>>(
|
||||||
|
new BaseSequence.IteratorMaker<Result<TimeseriesResultValue>, Iterator<Result<TimeseriesResultValue>>>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Iterator<Result<TimeseriesResultValue>> make()
|
||||||
|
{
|
||||||
|
return QueryRunnerFactories.makeCursorBasedQuery(
|
||||||
|
adapter,
|
||||||
|
query.getQuerySegmentSpec().getIntervals(),
|
||||||
|
Filters.convertDimensionFilters(query.getDimensionsFilter()),
|
||||||
|
query.getGranularity(),
|
||||||
|
new Function<Cursor, Result<TimeseriesResultValue>>()
|
||||||
|
{
|
||||||
|
private final List<AggregatorFactory> aggregatorSpecs = query.getAggregatorSpecs();
|
||||||
|
private final List<PostAggregator> postAggregatorSpecs = query.getPostAggregatorSpecs();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Result<TimeseriesResultValue> apply(Cursor cursor)
|
||||||
|
{
|
||||||
|
Aggregator[] aggregators = QueryRunnerFactories.makeAggregators(cursor, aggregatorSpecs);
|
||||||
|
|
||||||
|
while (!cursor.isDone()) {
|
||||||
|
for (Aggregator aggregator : aggregators) {
|
||||||
|
aggregator.aggregate();
|
||||||
|
}
|
||||||
|
cursor.advance();
|
||||||
|
}
|
||||||
|
|
||||||
|
TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime());
|
||||||
|
|
||||||
|
for (Aggregator aggregator : aggregators) {
|
||||||
|
bob.addMetric(aggregator);
|
||||||
|
}
|
||||||
|
|
||||||
|
for (PostAggregator postAgg : postAggregatorSpecs) {
|
||||||
|
bob.addMetric(postAgg);
|
||||||
|
}
|
||||||
|
|
||||||
|
return bob.build();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
).iterator();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void cleanup(Iterator<Result<TimeseriesResultValue>> toClean)
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public QueryRunner<Result<TimeseriesResultValue>> mergeRunners(
|
||||||
|
ExecutorService queryExecutor, Iterable<QueryRunner<Result<TimeseriesResultValue>>> queryRunners
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return new ChainedExecutionQueryRunner<Result<TimeseriesResultValue>>(
|
||||||
|
queryExecutor, toolChest.getOrdering(), queryRunners
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public QueryToolChest<Result<TimeseriesResultValue>, TimeseriesQuery> getToolchest()
|
||||||
|
{
|
||||||
|
return toolChest;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,403 @@
|
||||||
|
package com.metamx.druid.index.v1;
|
||||||
|
|
||||||
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.base.Throwables;
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
|
import com.google.common.io.CharStreams;
|
||||||
|
import com.google.common.io.Closeables;
|
||||||
|
import com.google.common.io.LineProcessor;
|
||||||
|
import com.google.common.primitives.Ints;
|
||||||
|
import com.metamx.common.ISE;
|
||||||
|
import com.metamx.common.logger.Logger;
|
||||||
|
import com.metamx.druid.QueryGranularity;
|
||||||
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
|
import com.metamx.druid.aggregation.DoubleSumAggregatorFactory;
|
||||||
|
import com.metamx.druid.client.RangeIterable;
|
||||||
|
import com.metamx.druid.guava.GuavaUtils;
|
||||||
|
import com.metamx.druid.index.v1.serde.ComplexMetricSerde;
|
||||||
|
import com.metamx.druid.index.v1.serde.ComplexMetrics;
|
||||||
|
import com.metamx.druid.input.InputRow;
|
||||||
|
import com.metamx.druid.kv.ArrayIndexed;
|
||||||
|
import com.metamx.druid.kv.Indexed;
|
||||||
|
import com.metamx.druid.kv.IndexedFloats;
|
||||||
|
import com.metamx.druid.kv.IndexedInts;
|
||||||
|
import com.metamx.druid.kv.IndexedLongs;
|
||||||
|
import com.metamx.druid.kv.Indexedids;
|
||||||
|
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.net.URL;
|
||||||
|
import java.nio.ByteOrder;
|
||||||
|
import java.nio.FloatBuffer;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class TestIndex
|
||||||
|
{
|
||||||
|
private static final Logger log = new Logger(TestIndex.class);
|
||||||
|
|
||||||
|
private static Index index = null;
|
||||||
|
private static Index unionIndexTop = null;
|
||||||
|
private static Index unionIndexBottom = null;
|
||||||
|
private static IncrementalIndex realtimeIndex = null;
|
||||||
|
private static MMappedIndex mmappedIndex = null;
|
||||||
|
private static MMappedIndex mergedRealtime = null;
|
||||||
|
|
||||||
|
public static final String[] DIMENSIONS = new String[]{"provider", "quality", "placement", "placementish"};
|
||||||
|
public static final String[] METRICS = new String[]{"index"};
|
||||||
|
public static final Map<String, Integer> dimIds = Maps.uniqueIndex(
|
||||||
|
new RangeIterable(4),
|
||||||
|
new Function<Integer, String>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public String apply(@Nullable Integer input)
|
||||||
|
{
|
||||||
|
return DIMENSIONS[input];
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
private static final Interval DATA_INTERVAL = new Interval("2011-01-12T00:00:00.000Z/2011-04-16T00:00:00.000Z");
|
||||||
|
private static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{
|
||||||
|
new DoubleSumAggregatorFactory(METRICS[0], METRICS[0])
|
||||||
|
};
|
||||||
|
|
||||||
|
public static Index convertMMapToIndex(MMappedIndex mmappedIndex)
|
||||||
|
{
|
||||||
|
Indexed<String> dimsIndexed = mmappedIndex.getAvailableDimensions();
|
||||||
|
String[] dimensions = new String[dimsIndexed.size()];
|
||||||
|
for (int i = 0; i < dimsIndexed.size(); ++i) {
|
||||||
|
dimensions[i] = dimsIndexed.get(i);
|
||||||
|
}
|
||||||
|
|
||||||
|
Indexed<String> metricsIndexed = mmappedIndex.getAvailableMetrics();
|
||||||
|
String[] metrics = new String[metricsIndexed.size()];
|
||||||
|
for (int i = 0; i < metricsIndexed.size(); ++i) {
|
||||||
|
metrics[i] = metricsIndexed.get(i);
|
||||||
|
}
|
||||||
|
|
||||||
|
IndexedLongs timeBuf = mmappedIndex.getReadOnlyTimestamps();
|
||||||
|
long[] timestamps = new long[timeBuf.size()];
|
||||||
|
timeBuf.fill(0, timestamps);
|
||||||
|
Closeables.closeQuietly(timeBuf);
|
||||||
|
|
||||||
|
Map<String, MetricHolder> metricVals = Maps.newLinkedHashMap();
|
||||||
|
for (String metric : metrics) {
|
||||||
|
MetricHolder holder = mmappedIndex.getMetricHolder(metric);
|
||||||
|
switch (holder.getType()) {
|
||||||
|
case FLOAT:
|
||||||
|
IndexedFloats mmappedFloats = holder.getFloatType();
|
||||||
|
float[] metricValsArray = new float[mmappedFloats.size()];
|
||||||
|
mmappedFloats.fill(0, metricValsArray);
|
||||||
|
Closeables.closeQuietly(mmappedFloats);
|
||||||
|
|
||||||
|
metricVals.put(
|
||||||
|
metric,
|
||||||
|
MetricHolder.floatMetric(
|
||||||
|
metric,
|
||||||
|
CompressedFloatsIndexedSupplier.fromFloatBuffer(
|
||||||
|
FloatBuffer.wrap(metricValsArray),
|
||||||
|
ByteOrder.nativeOrder()
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
break;
|
||||||
|
case COMPLEX:
|
||||||
|
Indexed complexObjects = holder.getComplexType();
|
||||||
|
Object[] vals = new Object[complexObjects.size()];
|
||||||
|
for (int i = 0; i < complexObjects.size(); ++i) {
|
||||||
|
vals[i] = complexObjects.get(i);
|
||||||
|
}
|
||||||
|
|
||||||
|
final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(holder.getTypeName());
|
||||||
|
if (serde == null) {
|
||||||
|
throw new ISE("Unknown type[%s]", holder.getTypeName());
|
||||||
|
}
|
||||||
|
|
||||||
|
metricVals.put(
|
||||||
|
metric,
|
||||||
|
MetricHolder.complexMetric(
|
||||||
|
metric,
|
||||||
|
holder.getTypeName(),
|
||||||
|
new ArrayIndexed(vals, serde.getObjectStrategy().getClazz())
|
||||||
|
)
|
||||||
|
);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Map<String, Map<String, Integer>> dimIdLookup = Maps.newHashMap();
|
||||||
|
Map<String, String[]> reverseDimLookup = Maps.newHashMap();
|
||||||
|
Map<String, ImmutableConciseSet[]> invertedIndexesMap = Maps.newHashMap();
|
||||||
|
Map<String, DimensionColumn> dimensionColumns = Maps.newHashMap();
|
||||||
|
|
||||||
|
for (String dimension : dimensions) {
|
||||||
|
final Indexed<String> dimValueLookup = mmappedIndex.getDimValueLookup(dimension);
|
||||||
|
String[] values = new String[dimValueLookup.size()];
|
||||||
|
for (int i = 0; i < dimValueLookup.size(); ++i) {
|
||||||
|
values[i] = dimValueLookup.get(i);
|
||||||
|
}
|
||||||
|
|
||||||
|
Map<String, Integer> lookupMap = Maps.newHashMapWithExpectedSize(dimValueLookup.size());
|
||||||
|
for (int i = 0; i < values.length; i++) {
|
||||||
|
lookupMap.put(values[i], i);
|
||||||
|
}
|
||||||
|
|
||||||
|
ImmutableConciseSet[] invertedIndexes = new ImmutableConciseSet[values.length];
|
||||||
|
final Indexed<String> dimValuesIndexed = mmappedIndex.getDimValueLookup(dimension);
|
||||||
|
for (int i = 0; i < dimValuesIndexed.size(); ++i) {
|
||||||
|
invertedIndexes[i] = mmappedIndex.getInvertedIndex(dimension, dimValuesIndexed.get(i));
|
||||||
|
}
|
||||||
|
|
||||||
|
int[] dimValues = new int[timestamps.length];
|
||||||
|
Map<List<Integer>, Integer> rowGroupings = Maps.newHashMap();
|
||||||
|
final Indexed<? extends IndexedInts> dimColumn = mmappedIndex.getDimColumn(dimension);
|
||||||
|
for (int i = 0; i < dimColumn.size(); ++i) {
|
||||||
|
int[] expansionValue = Indexedids.arrayFromIndexedInts(dimColumn.get(i));
|
||||||
|
Integer value = rowGroupings.get(Ints.asList(expansionValue));
|
||||||
|
if (value == null) {
|
||||||
|
value = rowGroupings.size();
|
||||||
|
rowGroupings.put(Ints.asList(expansionValue), value);
|
||||||
|
}
|
||||||
|
dimValues[i] = value;
|
||||||
|
}
|
||||||
|
|
||||||
|
int[][] expansionValues = new int[rowGroupings.size()][];
|
||||||
|
for (Map.Entry<List<Integer>, Integer> entry : rowGroupings.entrySet()) {
|
||||||
|
expansionValues[entry.getValue()] = Ints.toArray(entry.getKey());
|
||||||
|
}
|
||||||
|
|
||||||
|
dimIdLookup.put(dimension, lookupMap);
|
||||||
|
reverseDimLookup.put(dimension, values);
|
||||||
|
invertedIndexesMap.put(dimension, invertedIndexes);
|
||||||
|
dimensionColumns.put(dimension, new DimensionColumn(expansionValues, dimValues));
|
||||||
|
}
|
||||||
|
|
||||||
|
return new Index(
|
||||||
|
dimensions,
|
||||||
|
metrics,
|
||||||
|
mmappedIndex.getDataInterval(),
|
||||||
|
timestamps,
|
||||||
|
metricVals,
|
||||||
|
dimIdLookup,
|
||||||
|
reverseDimLookup,
|
||||||
|
invertedIndexesMap,
|
||||||
|
dimensionColumns
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Index getTestIndex() throws IOException
|
||||||
|
{
|
||||||
|
synchronized (log) {
|
||||||
|
if (index != null) {
|
||||||
|
return index;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return index = convertMMapToIndex(getMMappedTestIndex());
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Index getTestUnionIndexTop() throws IOException
|
||||||
|
{
|
||||||
|
synchronized (log) {
|
||||||
|
if (unionIndexTop != null) {
|
||||||
|
return unionIndexTop;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
IncrementalIndex incrementalIndex = makeRealtimeIndex("druid.sample.tsv.top");
|
||||||
|
MMappedIndex mmapped = persistRealtimeAndLoadMMapped(incrementalIndex);
|
||||||
|
|
||||||
|
return unionIndexTop = convertMMapToIndex(mmapped);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Index getTestUnionIndexBottom() throws IOException
|
||||||
|
{
|
||||||
|
synchronized (log) {
|
||||||
|
if (unionIndexBottom != null) {
|
||||||
|
return unionIndexBottom;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
IncrementalIndex incrementalIndex = makeRealtimeIndex("druid.sample.tsv.bottom");
|
||||||
|
MMappedIndex mmapped = persistRealtimeAndLoadMMapped(incrementalIndex);
|
||||||
|
|
||||||
|
return unionIndexBottom = convertMMapToIndex(mmapped);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static IncrementalIndex getIncrementalTestIndex()
|
||||||
|
{
|
||||||
|
synchronized (log) {
|
||||||
|
if (realtimeIndex != null) {
|
||||||
|
return realtimeIndex;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return realtimeIndex = makeRealtimeIndex("druid.sample.tsv");
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MMappedIndex getMMappedTestIndex()
|
||||||
|
{
|
||||||
|
synchronized (log) {
|
||||||
|
if (mmappedIndex != null) {
|
||||||
|
return mmappedIndex;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
IncrementalIndex incrementalIndex = getIncrementalTestIndex();
|
||||||
|
mmappedIndex = persistRealtimeAndLoadMMapped(incrementalIndex);
|
||||||
|
|
||||||
|
return mmappedIndex;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MMappedIndex mergedRealtimeIndex()
|
||||||
|
{
|
||||||
|
synchronized (log) {
|
||||||
|
if (mergedRealtime != null) {
|
||||||
|
return mergedRealtime;
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
IncrementalIndex top = makeRealtimeIndex("druid.sample.tsv.top");
|
||||||
|
IncrementalIndex bottom = makeRealtimeIndex("druid.sample.tsv.bottom");
|
||||||
|
|
||||||
|
File tmpFile = File.createTempFile("yay", "who");
|
||||||
|
tmpFile.delete();
|
||||||
|
|
||||||
|
File topFile = new File(tmpFile, "top");
|
||||||
|
File bottomFile = new File(tmpFile, "bottom");
|
||||||
|
File mergedFile = new File(tmpFile, "merged");
|
||||||
|
|
||||||
|
topFile.mkdirs();
|
||||||
|
topFile.deleteOnExit();
|
||||||
|
bottomFile.mkdirs();
|
||||||
|
bottomFile.deleteOnExit();
|
||||||
|
mergedFile.mkdirs();
|
||||||
|
mergedFile.deleteOnExit();
|
||||||
|
|
||||||
|
IndexMerger.persist(top, DATA_INTERVAL, topFile);
|
||||||
|
IndexMerger.persist(bottom, DATA_INTERVAL, bottomFile);
|
||||||
|
|
||||||
|
mergedRealtime = com.metamx.druid.index.v1.IndexIO.mapDir(
|
||||||
|
IndexMerger.mergeMMapped(
|
||||||
|
Arrays.asList(
|
||||||
|
com.metamx.druid.index.v1.IndexIO.mapDir(topFile),
|
||||||
|
com.metamx.druid.index.v1.IndexIO.mapDir(bottomFile)
|
||||||
|
), METRIC_AGGS, mergedFile
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
return mergedRealtime;
|
||||||
|
}
|
||||||
|
catch (IOException e) {
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static IncrementalIndex makeRealtimeIndex(final String resourceFilename)
|
||||||
|
{
|
||||||
|
URL resource = TestIndex.class.getClassLoader().getResource(resourceFilename);
|
||||||
|
String filename = resource.getFile();
|
||||||
|
log.info("Realtime loading index file[%s]", filename);
|
||||||
|
|
||||||
|
final IncrementalIndex retVal = new IncrementalIndex(
|
||||||
|
new DateTime("2011-01-12T00:00:00.000Z").getMillis(), QueryGranularity.NONE, METRIC_AGGS
|
||||||
|
);
|
||||||
|
|
||||||
|
final AtomicLong startTime = new AtomicLong();
|
||||||
|
int lineCount;
|
||||||
|
try {
|
||||||
|
lineCount = CharStreams.readLines(
|
||||||
|
GuavaUtils.joinFiles(new File(filename)),
|
||||||
|
new LineProcessor<Integer>()
|
||||||
|
{
|
||||||
|
boolean runOnce = false;
|
||||||
|
int lineCount = 0;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean processLine(String line) throws IOException
|
||||||
|
{
|
||||||
|
if (!runOnce) {
|
||||||
|
startTime.set(System.currentTimeMillis());
|
||||||
|
runOnce = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
final String[] splits = line.split("\t");
|
||||||
|
|
||||||
|
retVal.add(
|
||||||
|
new InputRow()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public long getTimestampFromEpoch()
|
||||||
|
{
|
||||||
|
return new DateTime(splits[0]).getMillis();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<String> getDimensions()
|
||||||
|
{
|
||||||
|
return Arrays.asList(DIMENSIONS);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<String> getDimension(String dimension)
|
||||||
|
{
|
||||||
|
return Arrays.asList(splits[dimIds.get(dimension) + 1].split("\u0001"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public float getFloatMetric(String metric)
|
||||||
|
{
|
||||||
|
Preconditions.checkArgument(METRICS[0].equals(metric), "WTF!?");
|
||||||
|
return Float.parseFloat(splits[5]);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
|
||||||
|
++lineCount;
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Integer getResult()
|
||||||
|
{
|
||||||
|
return lineCount;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
catch (IOException e) {
|
||||||
|
realtimeIndex = null;
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
|
|
||||||
|
log.info("Loaded %,d lines in %,d millis.", lineCount, System.currentTimeMillis() - startTime.get());
|
||||||
|
|
||||||
|
return retVal;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MMappedIndex persistRealtimeAndLoadMMapped(IncrementalIndex index)
|
||||||
|
{
|
||||||
|
try {
|
||||||
|
File someTmpFile = File.createTempFile("billy", "yay");
|
||||||
|
someTmpFile.delete();
|
||||||
|
someTmpFile.mkdirs();
|
||||||
|
someTmpFile.deleteOnExit();
|
||||||
|
|
||||||
|
IndexMerger.persist(index, someTmpFile);
|
||||||
|
return com.metamx.druid.index.v1.IndexIO.mapDir(someTmpFile);
|
||||||
|
}
|
||||||
|
catch (IOException e) {
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,127 @@
|
||||||
|
package com.metamx.druid.query;
|
||||||
|
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import com.metamx.druid.Query;
|
||||||
|
import com.metamx.druid.QueryGranularity;
|
||||||
|
import com.metamx.druid.StorageAdapter;
|
||||||
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
|
import com.metamx.druid.aggregation.CountAggregatorFactory;
|
||||||
|
import com.metamx.druid.aggregation.DoubleSumAggregatorFactory;
|
||||||
|
import com.metamx.druid.aggregation.LongSumAggregatorFactory;
|
||||||
|
import com.metamx.druid.aggregation.post.ArithmeticPostAggregator;
|
||||||
|
import com.metamx.druid.aggregation.post.ConstantPostAggregator;
|
||||||
|
import com.metamx.druid.aggregation.post.FieldAccessPostAggregator;
|
||||||
|
import com.metamx.druid.index.v1.IncrementalIndex;
|
||||||
|
import com.metamx.druid.index.v1.IncrementalIndexStorageAdapter;
|
||||||
|
import com.metamx.druid.index.v1.Index;
|
||||||
|
import com.metamx.druid.index.v1.IndexStorageAdapter;
|
||||||
|
import com.metamx.druid.index.v1.MMappedIndex;
|
||||||
|
import com.metamx.druid.index.v1.MMappedIndexStorageAdapter;
|
||||||
|
import com.metamx.druid.index.v1.TestIndex;
|
||||||
|
import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec;
|
||||||
|
import com.metamx.druid.query.segment.QuerySegmentSpec;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class QueryRunnerTestHelper
|
||||||
|
{
|
||||||
|
public static final String dataSource = "testing";
|
||||||
|
public static final QueryGranularity gran = QueryGranularity.DAY;
|
||||||
|
public static final QueryGranularity allGran = QueryGranularity.ALL;
|
||||||
|
public static final String providerDimension = "provider";
|
||||||
|
public static final String qualityDimension = "quality";
|
||||||
|
public static final String placementishDimension = "placementish";
|
||||||
|
public static final String indexMetric = "index";
|
||||||
|
public static final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows");
|
||||||
|
public static final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index");
|
||||||
|
public static final DoubleSumAggregatorFactory indexDoubleSum = new DoubleSumAggregatorFactory("index", "index");
|
||||||
|
public static final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L);
|
||||||
|
public static final FieldAccessPostAggregator rowsPostAgg = new FieldAccessPostAggregator("rows", "rows");
|
||||||
|
public static final FieldAccessPostAggregator indexPostAgg = new FieldAccessPostAggregator("index", "index");
|
||||||
|
public static final ArithmeticPostAggregator addRowsIndexConstant =
|
||||||
|
new ArithmeticPostAggregator(
|
||||||
|
"addRowsIndexConstant", "+", Lists.newArrayList(constant, rowsPostAgg, indexPostAgg)
|
||||||
|
);
|
||||||
|
public static final List<AggregatorFactory> commonAggregators = Arrays.asList(rowsCount, indexDoubleSum);
|
||||||
|
|
||||||
|
|
||||||
|
public static final String[] expectedFullOnIndexValues = new String[]{
|
||||||
|
"4500.0", "6077.949111938477", "4922.488838195801", "5726.140853881836", "4698.468170166016",
|
||||||
|
"4651.030891418457", "4398.145851135254", "4596.068244934082", "4434.630561828613", "0.0",
|
||||||
|
"6162.801361083984", "5590.292701721191", "4994.298484802246", "5179.679672241211", "6288.556800842285",
|
||||||
|
"6025.663551330566", "5772.855537414551", "5346.517524719238", "5497.331253051758", "5909.684387207031",
|
||||||
|
"5862.711364746094", "5958.373008728027", "5224.882194519043", "5456.789611816406", "5456.095397949219",
|
||||||
|
"4642.481948852539", "5023.572692871094", "5155.821723937988", "5350.3723220825195", "5236.997489929199",
|
||||||
|
"4910.097717285156", "4507.608840942383", "4659.80500793457", "5354.878845214844", "4945.796455383301",
|
||||||
|
"6459.080368041992", "4390.493583679199", "6545.758262634277", "6922.801231384277", "6023.452911376953",
|
||||||
|
"6812.107475280762", "6368.713348388672", "6381.748748779297", "5631.245086669922", "4976.192253112793",
|
||||||
|
"6541.463027954102", "5983.8513107299805", "5967.189498901367", "5567.139289855957", "4863.5944747924805",
|
||||||
|
"4681.164360046387", "6122.321441650391", "5410.308860778809", "4846.676376342773", "5333.872688293457",
|
||||||
|
"5013.053741455078", "4836.85563659668", "5264.486434936523", "4581.821243286133", "4680.233596801758",
|
||||||
|
"4771.363662719727", "5038.354717254639", "4816.808464050293", "4684.095504760742", "5023.663467407227",
|
||||||
|
"5889.72257232666", "4984.973915100098", "5664.220512390137", "5572.653915405273", "5537.123138427734",
|
||||||
|
"5980.422874450684", "6243.834693908691", "5372.147285461426", "5690.728981018066", "5827.796455383301",
|
||||||
|
"6141.0769119262695", "6082.3237228393555", "5678.771339416504", "6814.467971801758", "6626.151596069336",
|
||||||
|
"5833.2095947265625", "4679.222328186035", "5367.9403076171875", "5410.445640563965", "5689.197135925293",
|
||||||
|
"5240.5018310546875", "4790.912239074707", "4992.670921325684", "4796.888023376465", "5479.439590454102",
|
||||||
|
"5506.567192077637", "4743.144546508789", "4913.282669067383", "4723.869743347168"
|
||||||
|
};
|
||||||
|
|
||||||
|
public static final DateTime skippedDay = new DateTime("2011-01-21T00:00:00.000Z");
|
||||||
|
|
||||||
|
public static final QuerySegmentSpec firstToThird = new MultipleIntervalSegmentSpec(
|
||||||
|
Arrays.asList(new Interval("2011-04-01T00:00:00.000Z/2011-04-03T00:00:00.000Z"))
|
||||||
|
);
|
||||||
|
public static final QuerySegmentSpec fullOnInterval = new MultipleIntervalSegmentSpec(
|
||||||
|
Arrays.asList(new Interval("1970-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z"))
|
||||||
|
);
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public static Collection<?> makeQueryRunners(
|
||||||
|
QueryRunnerFactory factory
|
||||||
|
)
|
||||||
|
throws IOException
|
||||||
|
{
|
||||||
|
final Index testIndex = TestIndex.getTestIndex();
|
||||||
|
final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex();
|
||||||
|
final MMappedIndex persistedRTIndex = TestIndex.getMMappedTestIndex();
|
||||||
|
final MMappedIndex mergedRT = TestIndex.mergedRealtimeIndex();
|
||||||
|
return Arrays.asList(
|
||||||
|
new Object[][]{
|
||||||
|
{
|
||||||
|
makeQueryRunner(factory, new IndexStorageAdapter(testIndex))
|
||||||
|
},
|
||||||
|
{
|
||||||
|
makeQueryRunner(factory, new MMappedIndexStorageAdapter(MMappedIndex.fromIndex(testIndex)))
|
||||||
|
},
|
||||||
|
{
|
||||||
|
makeQueryRunner(factory, new IncrementalIndexStorageAdapter(rtIndex))
|
||||||
|
},
|
||||||
|
{
|
||||||
|
makeQueryRunner(factory, new MMappedIndexStorageAdapter(persistedRTIndex))
|
||||||
|
},
|
||||||
|
{
|
||||||
|
makeQueryRunner(factory, new MMappedIndexStorageAdapter(mergedRT))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static <T> QueryRunner<T> makeQueryRunner(
|
||||||
|
QueryRunnerFactory<T, Query<T>> factory,
|
||||||
|
StorageAdapter adapter
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return new FinalizeResultsQueryRunner<T>(
|
||||||
|
factory.createRunner(adapter),
|
||||||
|
factory.getToolchest()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,366 @@
|
||||||
|
package com.metamx.druid.query.search;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import com.google.common.collect.Sets;
|
||||||
|
import com.metamx.common.guava.Sequences;
|
||||||
|
import com.metamx.druid.Druids;
|
||||||
|
import com.metamx.druid.query.QueryRunner;
|
||||||
|
import com.metamx.druid.query.QueryRunnerTestHelper;
|
||||||
|
import com.metamx.druid.query.filter.DimFilter;
|
||||||
|
import com.metamx.druid.result.Result;
|
||||||
|
import com.metamx.druid.result.SearchResultValue;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
@RunWith(Parameterized.class)
|
||||||
|
public class SearchQueryRunnerTest
|
||||||
|
{
|
||||||
|
@Parameterized.Parameters
|
||||||
|
public static Collection<?> constructorFeeder() throws IOException
|
||||||
|
{
|
||||||
|
return QueryRunnerTestHelper.makeQueryRunners(
|
||||||
|
new SearchQueryRunnerFactory()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private final QueryRunner runner;
|
||||||
|
|
||||||
|
public SearchQueryRunnerTest(
|
||||||
|
QueryRunner runner
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.runner = runner;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSearch()
|
||||||
|
{
|
||||||
|
SearchQuery searchQuery = Druids.newSearchQueryBuilder()
|
||||||
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.granularity(QueryRunnerTestHelper.allGran)
|
||||||
|
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||||
|
.query("a")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
||||||
|
expectedResults.put(
|
||||||
|
QueryRunnerTestHelper.qualityDimension,
|
||||||
|
Sets.newHashSet("automotive", "mezzanine", "travel", "health", "entertainment")
|
||||||
|
);
|
||||||
|
expectedResults.put(QueryRunnerTestHelper.providerDimension, Sets.newHashSet("total_market"));
|
||||||
|
expectedResults.put(QueryRunnerTestHelper.placementishDimension, Sets.newHashSet("a"));
|
||||||
|
|
||||||
|
checkSearchQuery(searchQuery, expectedResults);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFragmentSearch()
|
||||||
|
{
|
||||||
|
SearchQuery searchQuery = Druids.newSearchQueryBuilder()
|
||||||
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.granularity(QueryRunnerTestHelper.allGran)
|
||||||
|
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||||
|
.query(new FragmentSearchQuerySpec(Arrays.asList("auto", "ve"), null))
|
||||||
|
.build();
|
||||||
|
|
||||||
|
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
||||||
|
expectedResults.put(QueryRunnerTestHelper.qualityDimension, Sets.newHashSet("automotive"));
|
||||||
|
|
||||||
|
checkSearchQuery(searchQuery, expectedResults);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSearchWithDimension1()
|
||||||
|
{
|
||||||
|
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
||||||
|
expectedResults.put(
|
||||||
|
QueryRunnerTestHelper.qualityDimension, new HashSet<String>(
|
||||||
|
Arrays.asList(
|
||||||
|
"automotive", "mezzanine", "travel", "health", "entertainment"
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
checkSearchQuery(
|
||||||
|
Druids.newSearchQueryBuilder()
|
||||||
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.granularity(QueryRunnerTestHelper.allGran)
|
||||||
|
.dimensions("quality")
|
||||||
|
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||||
|
.query("a")
|
||||||
|
.build(),
|
||||||
|
expectedResults
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSearchWithDimension2()
|
||||||
|
{
|
||||||
|
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
||||||
|
expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet<String>(Arrays.asList("total_market")));
|
||||||
|
|
||||||
|
checkSearchQuery(
|
||||||
|
Druids.newSearchQueryBuilder()
|
||||||
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.granularity(QueryRunnerTestHelper.allGran)
|
||||||
|
.dimensions("provider")
|
||||||
|
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||||
|
.query("a")
|
||||||
|
.build(),
|
||||||
|
expectedResults
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSearchWithDimensions1()
|
||||||
|
{
|
||||||
|
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
||||||
|
expectedResults.putAll(
|
||||||
|
ImmutableMap.<String, Set<String>>of(
|
||||||
|
QueryRunnerTestHelper.qualityDimension,
|
||||||
|
new HashSet<String>(
|
||||||
|
Arrays.asList(
|
||||||
|
"automotive", "mezzanine", "travel", "health", "entertainment"
|
||||||
|
)
|
||||||
|
),
|
||||||
|
QueryRunnerTestHelper.providerDimension,
|
||||||
|
new HashSet<String>(
|
||||||
|
Arrays.asList("total_market")
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
checkSearchQuery(
|
||||||
|
Druids.newSearchQueryBuilder()
|
||||||
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.granularity(QueryRunnerTestHelper.allGran)
|
||||||
|
.dimensions(Arrays.asList(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.providerDimension))
|
||||||
|
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||||
|
.query("a")
|
||||||
|
.build(),
|
||||||
|
expectedResults
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSearchWithDimensions2()
|
||||||
|
{
|
||||||
|
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
||||||
|
expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet<String>(Arrays.asList("total_market")));
|
||||||
|
|
||||||
|
checkSearchQuery(
|
||||||
|
Druids.newSearchQueryBuilder()
|
||||||
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.granularity(QueryRunnerTestHelper.allGran)
|
||||||
|
.dimensions(Arrays.asList(QueryRunnerTestHelper.placementishDimension, QueryRunnerTestHelper.providerDimension))
|
||||||
|
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||||
|
.query("mark")
|
||||||
|
.build(),
|
||||||
|
expectedResults
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSearchWithSingleFilter1()
|
||||||
|
{
|
||||||
|
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
||||||
|
expectedResults.put(
|
||||||
|
QueryRunnerTestHelper.qualityDimension, new HashSet<String>(Arrays.asList("automotive"))
|
||||||
|
);
|
||||||
|
|
||||||
|
checkSearchQuery(
|
||||||
|
Druids.newSearchQueryBuilder()
|
||||||
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.granularity(QueryRunnerTestHelper.allGran)
|
||||||
|
.filters(QueryRunnerTestHelper.qualityDimension, "automotive")
|
||||||
|
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||||
|
.dimensions(QueryRunnerTestHelper.qualityDimension)
|
||||||
|
.query("a")
|
||||||
|
.build(),
|
||||||
|
expectedResults
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSearchWithSingleFilter2()
|
||||||
|
{
|
||||||
|
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
||||||
|
expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet<String>(Arrays.asList("total_market")));
|
||||||
|
|
||||||
|
checkSearchQuery(
|
||||||
|
Druids.newSearchQueryBuilder()
|
||||||
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.granularity(QueryRunnerTestHelper.allGran)
|
||||||
|
.filters(QueryRunnerTestHelper.providerDimension, "total_market")
|
||||||
|
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||||
|
.dimensions(QueryRunnerTestHelper.providerDimension)
|
||||||
|
.query("a")
|
||||||
|
.build(),
|
||||||
|
expectedResults
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSearchMultiAndFilter()
|
||||||
|
{
|
||||||
|
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
||||||
|
expectedResults.put(QueryRunnerTestHelper.qualityDimension, new HashSet<String>(Arrays.asList("automotive")));
|
||||||
|
|
||||||
|
DimFilter filter = Druids.newAndDimFilterBuilder()
|
||||||
|
.fields(
|
||||||
|
Arrays.<DimFilter>asList(
|
||||||
|
Druids.newSelectorDimFilterBuilder()
|
||||||
|
.dimension(QueryRunnerTestHelper.providerDimension)
|
||||||
|
.value("spot")
|
||||||
|
.build(),
|
||||||
|
Druids.newSelectorDimFilterBuilder()
|
||||||
|
.dimension(QueryRunnerTestHelper.qualityDimension)
|
||||||
|
.value("automotive")
|
||||||
|
.build()
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
checkSearchQuery(
|
||||||
|
Druids.newSearchQueryBuilder()
|
||||||
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.granularity(QueryRunnerTestHelper.allGran)
|
||||||
|
.filters(filter)
|
||||||
|
.dimensions(QueryRunnerTestHelper.qualityDimension)
|
||||||
|
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||||
|
.query("a")
|
||||||
|
.build(),
|
||||||
|
expectedResults
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSearchWithMultiOrFilter()
|
||||||
|
{
|
||||||
|
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
||||||
|
expectedResults.put(QueryRunnerTestHelper.qualityDimension, new HashSet<String>(Arrays.asList("automotive")));
|
||||||
|
|
||||||
|
DimFilter filter = Druids.newOrDimFilterBuilder()
|
||||||
|
.fields(
|
||||||
|
Arrays.<DimFilter>asList(
|
||||||
|
Druids.newSelectorDimFilterBuilder()
|
||||||
|
.dimension(QueryRunnerTestHelper.qualityDimension)
|
||||||
|
.value("total_market")
|
||||||
|
.build(),
|
||||||
|
Druids.newSelectorDimFilterBuilder()
|
||||||
|
.dimension(QueryRunnerTestHelper.qualityDimension)
|
||||||
|
.value("automotive")
|
||||||
|
.build()
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
checkSearchQuery(
|
||||||
|
Druids.newSearchQueryBuilder()
|
||||||
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.granularity(QueryRunnerTestHelper.allGran)
|
||||||
|
.dimensions(QueryRunnerTestHelper.qualityDimension)
|
||||||
|
.filters(filter)
|
||||||
|
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||||
|
.query("a")
|
||||||
|
.build(),
|
||||||
|
expectedResults
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSearchWithEmptyResults()
|
||||||
|
{
|
||||||
|
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
||||||
|
|
||||||
|
checkSearchQuery(
|
||||||
|
Druids.newSearchQueryBuilder()
|
||||||
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.granularity(QueryRunnerTestHelper.allGran)
|
||||||
|
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||||
|
.query("abcd123")
|
||||||
|
.build(),
|
||||||
|
expectedResults
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSearchWithFilterEmptyResults()
|
||||||
|
{
|
||||||
|
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
||||||
|
|
||||||
|
DimFilter filter = Druids.newAndDimFilterBuilder()
|
||||||
|
.fields(
|
||||||
|
Arrays.<DimFilter>asList(
|
||||||
|
Druids.newSelectorDimFilterBuilder()
|
||||||
|
.dimension(QueryRunnerTestHelper.providerDimension)
|
||||||
|
.value("total_market")
|
||||||
|
.build(),
|
||||||
|
Druids.newSelectorDimFilterBuilder()
|
||||||
|
.dimension(QueryRunnerTestHelper.qualityDimension)
|
||||||
|
.value("automotive")
|
||||||
|
.build()
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
checkSearchQuery(
|
||||||
|
Druids.newSearchQueryBuilder()
|
||||||
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.granularity(QueryRunnerTestHelper.allGran)
|
||||||
|
.filters(filter)
|
||||||
|
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||||
|
.query("a")
|
||||||
|
.build(),
|
||||||
|
expectedResults
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void checkSearchQuery(SearchQuery searchQuery, Map<String, Set<String>> expectedResults)
|
||||||
|
{
|
||||||
|
Iterable<Result<SearchResultValue>> results = Sequences.toList(
|
||||||
|
runner.run(searchQuery),
|
||||||
|
Lists.<Result<SearchResultValue>>newArrayList()
|
||||||
|
);
|
||||||
|
|
||||||
|
for (Result<SearchResultValue> result : results) {
|
||||||
|
Assert.assertEquals(new DateTime("2011-01-12T00:00:00.000Z"), result.getTimestamp());
|
||||||
|
Assert.assertTrue(result.getValue() instanceof Iterable);
|
||||||
|
|
||||||
|
Iterable<SearchHit> resultValues = result.getValue();
|
||||||
|
for (SearchHit resultValue : resultValues) {
|
||||||
|
String dimension = resultValue.getDimension();
|
||||||
|
String theValue = resultValue.getValue();
|
||||||
|
Assert.assertTrue(expectedResults.containsKey(dimension));
|
||||||
|
|
||||||
|
Set<String> expectedSet = expectedResults.get(dimension);
|
||||||
|
Assert.assertTrue(
|
||||||
|
String.format("Couldn't remove dim[%s], value[%s]", dimension, theValue), expectedSet.remove(theValue)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
for (Map.Entry<String, Set<String>> entry : expectedResults.entrySet()) {
|
||||||
|
Assert.assertTrue(
|
||||||
|
String.format(
|
||||||
|
"Dimension %s should have had everything removed, still has[%s]", entry.getKey(), entry.getValue()
|
||||||
|
),
|
||||||
|
entry.getValue().isEmpty()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,60 @@
|
||||||
|
package com.metamx.druid.query.timeboundary;
|
||||||
|
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import com.metamx.common.guava.Sequences;
|
||||||
|
import com.metamx.druid.Druids;
|
||||||
|
import com.metamx.druid.query.QueryRunner;
|
||||||
|
import com.metamx.druid.query.QueryRunnerTestHelper;
|
||||||
|
import com.metamx.druid.result.Result;
|
||||||
|
import com.metamx.druid.result.TimeBoundaryResultValue;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collection;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
@RunWith(Parameterized.class)
|
||||||
|
public class TimeBoundaryQueryRunnerTest
|
||||||
|
{
|
||||||
|
@Parameterized.Parameters
|
||||||
|
public static Collection<?> constructorFeeder() throws IOException
|
||||||
|
{
|
||||||
|
return QueryRunnerTestHelper.makeQueryRunners(
|
||||||
|
new TimeBoundaryQueryRunnerFactory()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private final QueryRunner runner;
|
||||||
|
|
||||||
|
public TimeBoundaryQueryRunnerTest(
|
||||||
|
QueryRunner runner
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.runner = runner;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public void testTimeBoundary()
|
||||||
|
{
|
||||||
|
TimeBoundaryQuery timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder()
|
||||||
|
.dataSource("testing")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
|
||||||
|
runner.run(timeBoundaryQuery),
|
||||||
|
Lists.<Result<TimeBoundaryResultValue>>newArrayList()
|
||||||
|
);
|
||||||
|
TimeBoundaryResultValue val = results.iterator().next().getValue();
|
||||||
|
DateTime minTime = val.getMinTime();
|
||||||
|
DateTime maxTime = val.getMaxTime();
|
||||||
|
|
||||||
|
Assert.assertEquals(new DateTime("2011-01-12T00:00:00.000Z"), minTime);
|
||||||
|
Assert.assertEquals(new DateTime("2011-04-15T00:00:00.000Z"), maxTime);
|
||||||
|
}
|
||||||
|
}
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,732 @@
|
||||||
|
2011-01-12T00:00:00.000Z spot business preferred bpreferred 100.000000
|
||||||
|
2011-01-12T00:00:00.000Z spot health preferred hpreferred 100.000000
|
||||||
|
2011-01-12T00:00:00.000Z spot news preferred npreferred 100.000000
|
||||||
|
2011-01-12T00:00:00.000Z spot technology preferred tpreferred 100.000000
|
||||||
|
2011-01-12T00:00:00.000Z total_market mezzanine preferred mpreferred 1000.000000
|
||||||
|
2011-01-12T00:00:00.000Z upfront mezzanine preferred mpreferred 800.000000
|
||||||
|
2011-01-13T00:00:00.000Z spot automotive preferred apreferred 94.874713
|
||||||
|
2011-01-13T00:00:00.000Z spot entertainment preferred epreferred 110.087299
|
||||||
|
2011-01-13T00:00:00.000Z spot mezzanine preferred mpreferred 104.465767
|
||||||
|
2011-01-13T00:00:00.000Z spot premium preferred ppreferred 108.863011
|
||||||
|
2011-01-13T00:00:00.000Z spot travel preferred tpreferred 106.236928
|
||||||
|
2011-01-13T00:00:00.000Z total_market premium preferred ppreferred 1689.012875
|
||||||
|
2011-01-13T00:00:00.000Z upfront premium preferred ppreferred 1564.617729
|
||||||
|
2011-01-14T00:00:00.000Z spot business preferred bpreferred 102.670409
|
||||||
|
2011-01-14T00:00:00.000Z spot health preferred hpreferred 94.000432
|
||||||
|
2011-01-14T00:00:00.000Z spot news preferred npreferred 101.380760
|
||||||
|
2011-01-14T00:00:00.000Z spot technology preferred tpreferred 114.974216
|
||||||
|
2011-01-14T00:00:00.000Z total_market mezzanine preferred mpreferred 1049.141912
|
||||||
|
2011-01-14T00:00:00.000Z upfront mezzanine preferred mpreferred 1006.402111
|
||||||
|
2011-01-15T00:00:00.000Z spot automotive preferred apreferred 82.840417
|
||||||
|
2011-01-15T00:00:00.000Z spot entertainment preferred epreferred 100.551072
|
||||||
|
2011-01-15T00:00:00.000Z spot mezzanine preferred mpreferred 100.582654
|
||||||
|
2011-01-15T00:00:00.000Z spot premium preferred ppreferred 102.366798
|
||||||
|
2011-01-15T00:00:00.000Z spot travel preferred tpreferred 120.481420
|
||||||
|
2011-01-15T00:00:00.000Z total_market premium preferred ppreferred 1545.708865
|
||||||
|
2011-01-15T00:00:00.000Z upfront premium preferred ppreferred 1458.402661
|
||||||
|
2011-01-16T00:00:00.000Z spot business preferred bpreferred 100.591602
|
||||||
|
2011-01-16T00:00:00.000Z spot health preferred hpreferred 93.085943
|
||||||
|
2011-01-16T00:00:00.000Z spot news preferred npreferred 100.192013
|
||||||
|
2011-01-16T00:00:00.000Z spot technology preferred tpreferred 87.280816
|
||||||
|
2011-01-16T00:00:00.000Z total_market mezzanine preferred mpreferred 1016.965229
|
||||||
|
2011-01-16T00:00:00.000Z upfront mezzanine preferred mpreferred 852.437477
|
||||||
|
2011-01-17T00:00:00.000Z spot automotive preferred apreferred 105.442374
|
||||||
|
2011-01-17T00:00:00.000Z spot entertainment preferred epreferred 99.189052
|
||||||
|
2011-01-17T00:00:00.000Z spot mezzanine preferred mpreferred 103.692852
|
||||||
|
2011-01-17T00:00:00.000Z spot premium preferred ppreferred 106.696362
|
||||||
|
2011-01-17T00:00:00.000Z spot travel preferred tpreferred 134.415281
|
||||||
|
2011-01-17T00:00:00.000Z total_market premium preferred ppreferred 953.995422
|
||||||
|
2011-01-17T00:00:00.000Z upfront premium preferred ppreferred 712.774595
|
||||||
|
2011-01-18T00:00:00.000Z spot business preferred bpreferred 107.244699
|
||||||
|
2011-01-18T00:00:00.000Z spot health preferred hpreferred 103.018934
|
||||||
|
2011-01-18T00:00:00.000Z spot news preferred npreferred 104.724154
|
||||||
|
2011-01-18T00:00:00.000Z spot technology preferred tpreferred 88.764512
|
||||||
|
2011-01-18T00:00:00.000Z total_market mezzanine preferred mpreferred 1022.783330
|
||||||
|
2011-01-18T00:00:00.000Z upfront mezzanine preferred mpreferred 846.267516
|
||||||
|
2011-01-19T00:00:00.000Z spot automotive preferred apreferred 85.681683
|
||||||
|
2011-01-19T00:00:00.000Z spot entertainment preferred epreferred 92.314034
|
||||||
|
2011-01-19T00:00:00.000Z spot mezzanine preferred mpreferred 105.087466
|
||||||
|
2011-01-19T00:00:00.000Z spot premium preferred ppreferred 104.568464
|
||||||
|
2011-01-19T00:00:00.000Z spot travel preferred tpreferred 109.772202
|
||||||
|
2011-01-19T00:00:00.000Z total_market premium preferred ppreferred 849.877513
|
||||||
|
2011-01-19T00:00:00.000Z upfront premium preferred ppreferred 594.381703
|
||||||
|
2011-01-20T01:00:00.000Z spot business preferred bpreferred 106.367723
|
||||||
|
2011-01-20T01:00:00.000Z spot health preferred hpreferred 96.112901
|
||||||
|
2011-01-20T01:00:00.000Z spot news preferred npreferred 105.225158
|
||||||
|
2011-01-20T01:00:00.000Z spot technology preferred tpreferred 77.759854
|
||||||
|
2011-01-20T01:00:00.000Z total_market mezzanine preferred mpreferred 1066.208012
|
||||||
|
2011-01-20T01:00:00.000Z upfront mezzanine preferred mpreferred 870.115926
|
||||||
|
2011-01-22T00:00:00.000Z spot business preferred bpreferred 105.282866
|
||||||
|
2011-01-22T00:00:00.000Z spot health preferred hpreferred 103.527592
|
||||||
|
2011-01-22T00:00:00.000Z spot news preferred npreferred 103.399677
|
||||||
|
2011-01-22T00:00:00.000Z spot technology preferred tpreferred 107.627793
|
||||||
|
2011-01-22T00:00:00.000Z total_market mezzanine preferred mpreferred 1240.525484
|
||||||
|
2011-01-22T00:00:00.000Z upfront mezzanine preferred mpreferred 1410.278128
|
||||||
|
2011-01-23T00:00:00.000Z spot automotive preferred apreferred 100.432710
|
||||||
|
2011-01-23T00:00:00.000Z spot entertainment preferred epreferred 92.789692
|
||||||
|
2011-01-23T00:00:00.000Z spot mezzanine preferred mpreferred 103.730730
|
||||||
|
2011-01-23T00:00:00.000Z spot premium preferred ppreferred 101.573522
|
||||||
|
2011-01-23T00:00:00.000Z spot travel preferred tpreferred 128.699746
|
||||||
|
2011-01-23T00:00:00.000Z total_market premium preferred ppreferred 1349.254415
|
||||||
|
2011-01-23T00:00:00.000Z upfront premium preferred ppreferred 1224.501568
|
||||||
|
2011-01-24T00:00:00.000Z spot business preferred bpreferred 104.485760
|
||||||
|
2011-01-24T00:00:00.000Z spot health preferred hpreferred 189.385952
|
||||||
|
2011-01-24T00:00:00.000Z spot news preferred npreferred 104.167373
|
||||||
|
2011-01-24T00:00:00.000Z spot technology preferred tpreferred 105.345921
|
||||||
|
2011-01-24T00:00:00.000Z total_market mezzanine preferred mpreferred 1102.866656
|
||||||
|
2011-01-24T00:00:00.000Z upfront mezzanine preferred mpreferred 1215.589859
|
||||||
|
2011-01-25T00:00:00.000Z spot automotive preferred apreferred 90.111413
|
||||||
|
2011-01-25T00:00:00.000Z spot entertainment preferred epreferred 85.974579
|
||||||
|
2011-01-25T00:00:00.000Z spot mezzanine preferred mpreferred 96.594588
|
||||||
|
2011-01-25T00:00:00.000Z spot premium preferred ppreferred 102.512878
|
||||||
|
2011-01-25T00:00:00.000Z spot travel preferred tpreferred 122.077247
|
||||||
|
2011-01-25T00:00:00.000Z total_market premium preferred ppreferred 997.994544
|
||||||
|
2011-01-25T00:00:00.000Z upfront premium preferred ppreferred 786.363298
|
||||||
|
2011-01-26T00:00:00.000Z spot business preferred bpreferred 105.873942
|
||||||
|
2011-01-26T00:00:00.000Z spot health preferred hpreferred 159.988606
|
||||||
|
2011-01-26T00:00:00.000Z spot news preferred npreferred 105.266058
|
||||||
|
2011-01-26T00:00:00.000Z spot technology preferred tpreferred 105.617702
|
||||||
|
2011-01-26T00:00:00.000Z total_market mezzanine preferred mpreferred 1037.449471
|
||||||
|
2011-01-26T00:00:00.000Z upfront mezzanine preferred mpreferred 989.931541
|
||||||
|
2011-01-27T00:00:00.000Z spot automotive preferred apreferred 134.127106
|
||||||
|
2011-01-27T00:00:00.000Z spot entertainment preferred epreferred 90.806201
|
||||||
|
2011-01-27T00:00:00.000Z spot mezzanine preferred mpreferred 100.643435
|
||||||
|
2011-01-27T00:00:00.000Z spot premium preferred ppreferred 102.612747
|
||||||
|
2011-01-27T00:00:00.000Z spot travel preferred tpreferred 149.125271
|
||||||
|
2011-01-27T00:00:00.000Z total_market premium preferred ppreferred 1486.201299
|
||||||
|
2011-01-27T00:00:00.000Z upfront premium preferred ppreferred 1367.638074
|
||||||
|
2011-01-28T00:00:00.000Z spot business preferred bpreferred 111.641077
|
||||||
|
2011-01-28T00:00:00.000Z spot health preferred hpreferred 107.788998
|
||||||
|
2011-01-28T00:00:00.000Z spot news preferred npreferred 108.106449
|
||||||
|
2011-01-28T00:00:00.000Z spot technology preferred tpreferred 93.869236
|
||||||
|
2011-01-28T00:00:00.000Z total_market mezzanine preferred mpreferred 1300.302260
|
||||||
|
2011-01-28T00:00:00.000Z upfront mezzanine preferred mpreferred 1627.598064
|
||||||
|
2011-01-29T00:00:00.000Z spot automotive preferred apreferred 127.450345
|
||||||
|
2011-01-29T00:00:00.000Z spot entertainment preferred epreferred 103.345166
|
||||||
|
2011-01-29T00:00:00.000Z spot mezzanine preferred mpreferred 106.663538
|
||||||
|
2011-01-29T00:00:00.000Z spot premium preferred ppreferred 104.311418
|
||||||
|
2011-01-29T00:00:00.000Z spot travel preferred tpreferred 138.865014
|
||||||
|
2011-01-29T00:00:00.000Z total_market premium preferred ppreferred 792.326066
|
||||||
|
2011-01-29T00:00:00.000Z upfront premium preferred ppreferred 545.990623
|
||||||
|
2011-01-30T00:00:00.000Z spot business preferred bpreferred 106.064111
|
||||||
|
2011-01-30T00:00:00.000Z spot health preferred hpreferred 137.198397
|
||||||
|
2011-01-30T00:00:00.000Z spot news preferred npreferred 106.009926
|
||||||
|
2011-01-30T00:00:00.000Z spot technology preferred tpreferred 110.528451
|
||||||
|
2011-01-30T00:00:00.000Z total_market mezzanine preferred mpreferred 1233.448863
|
||||||
|
2011-01-30T00:00:00.000Z upfront mezzanine preferred mpreferred 1870.061029
|
||||||
|
2011-01-31T00:00:00.000Z spot automotive preferred apreferred 133.740047
|
||||||
|
2011-01-31T00:00:00.000Z spot entertainment preferred epreferred 104.548387
|
||||||
|
2011-01-31T00:00:00.000Z spot mezzanine preferred mpreferred 97.017604
|
||||||
|
2011-01-31T00:00:00.000Z spot premium preferred ppreferred 103.021032
|
||||||
|
2011-01-31T00:00:00.000Z spot travel preferred tpreferred 155.744951
|
||||||
|
2011-01-31T00:00:00.000Z total_market premium preferred ppreferred 1127.231000
|
||||||
|
2011-01-31T00:00:00.000Z upfront premium preferred ppreferred 943.497198
|
||||||
|
2011-02-01T00:00:00.000Z spot business preferred bpreferred 103.890175
|
||||||
|
2011-02-01T00:00:00.000Z spot health preferred hpreferred 113.896016
|
||||||
|
2011-02-01T00:00:00.000Z spot news preferred npreferred 104.383662
|
||||||
|
2011-02-01T00:00:00.000Z spot technology preferred tpreferred 83.931272
|
||||||
|
2011-02-01T00:00:00.000Z total_market mezzanine preferred mpreferred 1203.465595
|
||||||
|
2011-02-01T00:00:00.000Z upfront mezzanine preferred mpreferred 1667.497773
|
||||||
|
2011-02-02T00:00:00.000Z spot automotive preferred apreferred 113.492245
|
||||||
|
2011-02-02T00:00:00.000Z spot entertainment preferred epreferred 112.042996
|
||||||
|
2011-02-02T00:00:00.000Z spot mezzanine preferred mpreferred 97.711139
|
||||||
|
2011-02-02T00:00:00.000Z spot premium preferred ppreferred 102.152053
|
||||||
|
2011-02-02T00:00:00.000Z spot travel preferred tpreferred 123.170962
|
||||||
|
2011-02-02T00:00:00.000Z total_market premium preferred ppreferred 1410.792943
|
||||||
|
2011-02-02T00:00:00.000Z upfront premium preferred ppreferred 1273.707453
|
||||||
|
2011-02-03T00:00:00.000Z spot business preferred bpreferred 98.877952
|
||||||
|
2011-02-03T00:00:00.000Z spot health preferred hpreferred 85.069784
|
||||||
|
2011-02-03T00:00:00.000Z spot news preferred npreferred 100.976362
|
||||||
|
2011-02-03T00:00:00.000Z spot technology preferred tpreferred 92.326431
|
||||||
|
2011-02-03T00:00:00.000Z total_market mezzanine preferred mpreferred 1033.401241
|
||||||
|
2011-02-03T00:00:00.000Z upfront mezzanine preferred mpreferred 888.705280
|
||||||
|
2011-02-04T00:00:00.000Z spot automotive preferred apreferred 89.182906
|
||||||
|
2011-02-04T00:00:00.000Z spot entertainment preferred epreferred 112.471918
|
||||||
|
2011-02-04T00:00:00.000Z spot mezzanine preferred mpreferred 100.918373
|
||||||
|
2011-02-04T00:00:00.000Z spot premium preferred ppreferred 103.103922
|
||||||
|
2011-02-04T00:00:00.000Z spot travel preferred tpreferred 113.716758
|
||||||
|
2011-02-04T00:00:00.000Z total_market premium preferred ppreferred 1331.860983
|
||||||
|
2011-02-04T00:00:00.000Z upfront premium preferred ppreferred 1308.582051
|
||||||
|
2011-02-05T00:00:00.000Z spot business preferred bpreferred 111.394244
|
||||||
|
2011-02-05T00:00:00.000Z spot health preferred hpreferred 94.312960
|
||||||
|
2011-02-05T00:00:00.000Z spot news preferred npreferred 107.929804
|
||||||
|
2011-02-05T00:00:00.000Z spot technology preferred tpreferred 90.732978
|
||||||
|
2011-02-05T00:00:00.000Z total_market mezzanine preferred mpreferred 1039.500513
|
||||||
|
2011-02-05T00:00:00.000Z upfront mezzanine preferred mpreferred 785.078869
|
||||||
|
2011-02-06T00:00:00.000Z spot automotive preferred apreferred 134.462521
|
||||||
|
2011-02-06T00:00:00.000Z spot entertainment preferred epreferred 119.608310
|
||||||
|
2011-02-06T00:00:00.000Z spot mezzanine preferred mpreferred 101.847544
|
||||||
|
2011-02-06T00:00:00.000Z spot premium preferred ppreferred 102.770913
|
||||||
|
2011-02-06T00:00:00.000Z spot travel preferred tpreferred 121.270562
|
||||||
|
2011-02-06T00:00:00.000Z total_market premium preferred ppreferred 1029.995236
|
||||||
|
2011-02-06T00:00:00.000Z upfront premium preferred ppreferred 826.039207
|
||||||
|
2011-02-07T00:00:00.000Z spot business preferred bpreferred 98.815847
|
||||||
|
2011-02-07T00:00:00.000Z spot health preferred hpreferred 97.480779
|
||||||
|
2011-02-07T00:00:00.000Z spot news preferred npreferred 100.706057
|
||||||
|
2011-02-07T00:00:00.000Z spot technology preferred tpreferred 83.902353
|
||||||
|
2011-02-07T00:00:00.000Z total_market mezzanine preferred mpreferred 1047.212887
|
||||||
|
2011-02-07T00:00:00.000Z upfront mezzanine preferred mpreferred 1107.243787
|
||||||
|
2011-02-08T00:00:00.000Z spot automotive preferred apreferred 126.243536
|
||||||
|
2011-02-08T00:00:00.000Z spot entertainment preferred epreferred 106.969799
|
||||||
|
2011-02-08T00:00:00.000Z spot mezzanine preferred mpreferred 96.790543
|
||||||
|
2011-02-08T00:00:00.000Z spot premium preferred ppreferred 103.308255
|
||||||
|
2011-02-08T00:00:00.000Z spot travel preferred tpreferred 111.040150
|
||||||
|
2011-02-08T00:00:00.000Z total_market premium preferred ppreferred 1082.727640
|
||||||
|
2011-02-08T00:00:00.000Z upfront premium preferred ppreferred 911.956790
|
||||||
|
2011-02-09T00:00:00.000Z spot business preferred bpreferred 115.548444
|
||||||
|
2011-02-09T00:00:00.000Z spot health preferred hpreferred 97.071703
|
||||||
|
2011-02-09T00:00:00.000Z spot news preferred npreferred 113.554588
|
||||||
|
2011-02-09T00:00:00.000Z spot technology preferred tpreferred 83.742151
|
||||||
|
2011-02-09T00:00:00.000Z total_market mezzanine preferred mpreferred 971.050764
|
||||||
|
2011-02-09T00:00:00.000Z upfront mezzanine preferred mpreferred 794.098825
|
||||||
|
2011-02-10T00:00:00.000Z spot automotive preferred apreferred 115.461691
|
||||||
|
2011-02-10T00:00:00.000Z spot entertainment preferred epreferred 117.629065
|
||||||
|
2011-02-10T00:00:00.000Z spot mezzanine preferred mpreferred 95.698374
|
||||||
|
2011-02-10T00:00:00.000Z spot premium preferred ppreferred 105.708103
|
||||||
|
2011-02-10T00:00:00.000Z spot travel preferred tpreferred 105.557241
|
||||||
|
2011-02-10T00:00:00.000Z total_market premium preferred ppreferred 1089.647884
|
||||||
|
2011-02-10T00:00:00.000Z upfront premium preferred ppreferred 901.327272
|
||||||
|
2011-02-11T00:00:00.000Z spot business preferred bpreferred 114.637486
|
||||||
|
2011-02-11T00:00:00.000Z spot health preferred hpreferred 96.617339
|
||||||
|
2011-02-11T00:00:00.000Z spot news preferred npreferred 112.571724
|
||||||
|
2011-02-11T00:00:00.000Z spot technology preferred tpreferred 87.904114
|
||||||
|
2011-02-11T00:00:00.000Z total_market mezzanine preferred mpreferred 980.386611
|
||||||
|
2011-02-11T00:00:00.000Z upfront mezzanine preferred mpreferred 723.514254
|
||||||
|
2011-02-12T00:00:00.000Z spot automotive preferred apreferred 130.104979
|
||||||
|
2011-02-12T00:00:00.000Z spot entertainment preferred epreferred 115.225386
|
||||||
|
2011-02-12T00:00:00.000Z spot mezzanine preferred mpreferred 101.515571
|
||||||
|
2011-02-12T00:00:00.000Z spot premium preferred ppreferred 108.637522
|
||||||
|
2011-02-12T00:00:00.000Z spot travel preferred tpreferred 102.850696
|
||||||
|
2011-02-12T00:00:00.000Z total_market premium preferred ppreferred 1092.416967
|
||||||
|
2011-02-12T00:00:00.000Z upfront premium preferred ppreferred 879.406101
|
||||||
|
2011-02-13T00:00:00.000Z spot business preferred bpreferred 118.841176
|
||||||
|
2011-02-13T00:00:00.000Z spot health preferred hpreferred 98.607490
|
||||||
|
2011-02-13T00:00:00.000Z spot news preferred npreferred 117.965974
|
||||||
|
2011-02-13T00:00:00.000Z spot technology preferred tpreferred 87.820236
|
||||||
|
2011-02-13T00:00:00.000Z total_market mezzanine preferred mpreferred 987.067381
|
||||||
|
2011-02-13T00:00:00.000Z upfront mezzanine preferred mpreferred 724.262526
|
||||||
|
2011-02-14T00:00:00.000Z spot automotive preferred apreferred 119.323168
|
||||||
|
2011-02-14T00:00:00.000Z spot entertainment preferred epreferred 123.098262
|
||||||
|
2011-02-14T00:00:00.000Z spot mezzanine preferred mpreferred 101.645725
|
||||||
|
2011-02-14T00:00:00.000Z spot premium preferred ppreferred 121.060464
|
||||||
|
2011-02-14T00:00:00.000Z spot travel preferred tpreferred 107.663239
|
||||||
|
2011-02-14T00:00:00.000Z total_market premium preferred ppreferred 1199.607472
|
||||||
|
2011-02-14T00:00:00.000Z upfront premium preferred ppreferred 948.657939
|
||||||
|
2011-02-15T00:00:00.000Z spot business preferred bpreferred 115.527003
|
||||||
|
2011-02-15T00:00:00.000Z spot health preferred hpreferred 103.519393
|
||||||
|
2011-02-15T00:00:00.000Z spot news preferred npreferred 117.138956
|
||||||
|
2011-02-15T00:00:00.000Z spot technology preferred tpreferred 79.700998
|
||||||
|
2011-02-15T00:00:00.000Z total_market mezzanine preferred mpreferred 1044.384300
|
||||||
|
2011-02-15T00:00:00.000Z upfront mezzanine preferred mpreferred 807.601674
|
||||||
|
2011-02-16T00:00:00.000Z spot automotive preferred apreferred 133.726576
|
||||||
|
2011-02-16T00:00:00.000Z spot entertainment preferred epreferred 130.717934
|
||||||
|
2011-02-16T00:00:00.000Z spot mezzanine preferred mpreferred 108.338531
|
||||||
|
2011-02-16T00:00:00.000Z spot premium preferred ppreferred 123.270869
|
||||||
|
2011-02-16T00:00:00.000Z spot travel preferred tpreferred 113.643571
|
||||||
|
2011-02-16T00:00:00.000Z total_market premium preferred ppreferred 1360.032423
|
||||||
|
2011-02-16T00:00:00.000Z upfront premium preferred ppreferred 1208.456692
|
||||||
|
2011-02-17T00:00:00.000Z spot business preferred bpreferred 125.032692
|
||||||
|
2011-02-17T00:00:00.000Z spot health preferred hpreferred 101.092779
|
||||||
|
2011-02-17T00:00:00.000Z spot news preferred npreferred 121.872569
|
||||||
|
2011-02-17T00:00:00.000Z spot technology preferred tpreferred 91.969195
|
||||||
|
2011-02-17T00:00:00.000Z total_market mezzanine preferred mpreferred 993.591221
|
||||||
|
2011-02-17T00:00:00.000Z upfront mezzanine preferred mpreferred 679.619354
|
||||||
|
2011-02-18T00:00:00.000Z spot automotive preferred apreferred 151.053296
|
||||||
|
2011-02-18T00:00:00.000Z spot entertainment preferred epreferred 128.063524
|
||||||
|
2011-02-18T00:00:00.000Z spot mezzanine preferred mpreferred 101.960196
|
||||||
|
2011-02-18T00:00:00.000Z spot premium preferred ppreferred 122.546253
|
||||||
|
2011-02-18T00:00:00.000Z spot travel preferred tpreferred 120.151389
|
||||||
|
2011-02-18T00:00:00.000Z total_market premium preferred ppreferred 1601.829436
|
||||||
|
2011-02-18T00:00:00.000Z upfront premium preferred ppreferred 1649.533329
|
||||||
|
2011-02-19T00:00:00.000Z spot business preferred bpreferred 123.447481
|
||||||
|
2011-02-19T00:00:00.000Z spot health preferred hpreferred 108.822138
|
||||||
|
2011-02-19T00:00:00.000Z spot news preferred npreferred 119.371511
|
||||||
|
2011-02-19T00:00:00.000Z spot technology preferred tpreferred 91.547944
|
||||||
|
2011-02-19T00:00:00.000Z total_market mezzanine preferred mpreferred 1200.527201
|
||||||
|
2011-02-19T00:00:00.000Z upfront mezzanine preferred mpreferred 1433.398801
|
||||||
|
2011-02-20T00:00:00.000Z spot automotive preferred apreferred 157.483005
|
||||||
|
2011-02-20T00:00:00.000Z spot entertainment preferred epreferred 176.323916
|
||||||
|
2011-02-20T00:00:00.000Z spot mezzanine preferred mpreferred 127.819268
|
||||||
|
2011-02-20T00:00:00.000Z spot premium preferred ppreferred 132.719065
|
||||||
|
2011-02-20T00:00:00.000Z spot travel preferred tpreferred 124.637709
|
||||||
|
2011-02-20T00:00:00.000Z total_market premium preferred ppreferred 1304.326111
|
||||||
|
2011-02-20T00:00:00.000Z upfront premium preferred ppreferred 1022.854576
|
||||||
|
2011-02-21T00:00:00.000Z spot business preferred bpreferred 132.231346
|
||||||
|
2011-02-21T00:00:00.000Z spot health preferred hpreferred 111.386745
|
||||||
|
2011-02-21T00:00:00.000Z spot news preferred npreferred 126.995117
|
||||||
|
2011-02-21T00:00:00.000Z spot technology preferred tpreferred 119.850150
|
||||||
|
2011-02-21T00:00:00.000Z total_market mezzanine preferred mpreferred 1544.108134
|
||||||
|
2011-02-21T00:00:00.000Z upfront mezzanine preferred mpreferred 1228.502469
|
||||||
|
2011-02-22T00:00:00.000Z spot automotive preferred apreferred 149.171056
|
||||||
|
2011-02-22T00:00:00.000Z spot entertainment preferred epreferred 139.557139
|
||||||
|
2011-02-22T00:00:00.000Z spot mezzanine preferred mpreferred 111.394542
|
||||||
|
2011-02-22T00:00:00.000Z spot premium preferred ppreferred 126.596847
|
||||||
|
2011-02-22T00:00:00.000Z spot travel preferred tpreferred 115.915849
|
||||||
|
2011-02-22T00:00:00.000Z total_market premium preferred ppreferred 1421.648704
|
||||||
|
2011-02-22T00:00:00.000Z upfront premium preferred ppreferred 1291.897942
|
||||||
|
2011-02-23T00:00:00.000Z spot business preferred bpreferred 127.199815
|
||||||
|
2011-02-23T00:00:00.000Z spot health preferred hpreferred 107.896489
|
||||||
|
2011-02-23T00:00:00.000Z spot news preferred npreferred 122.404192
|
||||||
|
2011-02-23T00:00:00.000Z spot technology preferred tpreferred 91.191071
|
||||||
|
2011-02-23T00:00:00.000Z total_market mezzanine preferred mpreferred 1251.906228
|
||||||
|
2011-02-23T00:00:00.000Z upfront mezzanine preferred mpreferred 1306.495696
|
||||||
|
2011-02-24T00:00:00.000Z spot automotive preferred apreferred 168.988478
|
||||||
|
2011-02-24T00:00:00.000Z spot entertainment preferred epreferred 139.978575
|
||||||
|
2011-02-24T00:00:00.000Z spot mezzanine preferred mpreferred 113.621184
|
||||||
|
2011-02-24T00:00:00.000Z spot premium preferred ppreferred 119.772575
|
||||||
|
2011-02-24T00:00:00.000Z spot travel preferred tpreferred 118.270052
|
||||||
|
2011-02-24T00:00:00.000Z total_market premium preferred ppreferred 970.728273
|
||||||
|
2011-02-24T00:00:00.000Z upfront premium preferred ppreferred 578.795979
|
||||||
|
2011-02-25T00:00:00.000Z spot business preferred bpreferred 107.539869
|
||||||
|
2011-02-25T00:00:00.000Z spot health preferred hpreferred 99.698015
|
||||||
|
2011-02-25T00:00:00.000Z spot news preferred npreferred 106.827630
|
||||||
|
2011-02-25T00:00:00.000Z spot technology preferred tpreferred 82.350556
|
||||||
|
2011-02-25T00:00:00.000Z total_market mezzanine preferred mpreferred 1057.427269
|
||||||
|
2011-02-25T00:00:00.000Z upfront mezzanine preferred mpreferred 1017.573185
|
||||||
|
2011-02-26T00:00:00.000Z spot automotive preferred apreferred 225.243186
|
||||||
|
2011-02-26T00:00:00.000Z spot entertainment preferred epreferred 138.924835
|
||||||
|
2011-02-26T00:00:00.000Z spot mezzanine preferred mpreferred 98.314744
|
||||||
|
2011-02-26T00:00:00.000Z spot premium preferred ppreferred 118.896845
|
||||||
|
2011-02-26T00:00:00.000Z spot travel preferred tpreferred 131.310541
|
||||||
|
2011-02-26T00:00:00.000Z total_market premium preferred ppreferred 1743.921750
|
||||||
|
2011-02-26T00:00:00.000Z upfront premium preferred ppreferred 1862.737933
|
||||||
|
2011-02-27T00:00:00.000Z spot business preferred bpreferred 96.864384
|
||||||
|
2011-02-27T00:00:00.000Z spot health preferred hpreferred 114.634278
|
||||||
|
2011-02-27T00:00:00.000Z spot news preferred npreferred 99.158839
|
||||||
|
2011-02-27T00:00:00.000Z spot technology preferred tpreferred 78.727950
|
||||||
|
2011-02-27T00:00:00.000Z total_market mezzanine preferred mpreferred 1044.562903
|
||||||
|
2011-02-27T00:00:00.000Z upfront mezzanine preferred mpreferred 873.306547
|
||||||
|
2011-02-28T00:00:00.000Z spot automotive preferred apreferred 122.258195
|
||||||
|
2011-02-28T00:00:00.000Z spot entertainment preferred epreferred 141.261324
|
||||||
|
2011-02-28T00:00:00.000Z spot mezzanine preferred mpreferred 102.185098
|
||||||
|
2011-02-28T00:00:00.000Z spot premium preferred ppreferred 121.786785
|
||||||
|
2011-02-28T00:00:00.000Z spot travel preferred tpreferred 114.284569
|
||||||
|
2011-02-28T00:00:00.000Z total_market premium preferred ppreferred 1292.542896
|
||||||
|
2011-02-28T00:00:00.000Z upfront premium preferred ppreferred 1101.918270
|
||||||
|
2011-03-01T00:00:00.000Z spot business preferred bpreferred 99.070796
|
||||||
|
2011-03-01T00:00:00.000Z spot health preferred hpreferred 114.700932
|
||||||
|
2011-03-01T00:00:00.000Z spot news preferred npreferred 99.772347
|
||||||
|
2011-03-01T00:00:00.000Z spot technology preferred tpreferred 72.792970
|
||||||
|
2011-03-01T00:00:00.000Z total_market mezzanine preferred mpreferred 1124.201419
|
||||||
|
2011-03-01T00:00:00.000Z upfront mezzanine preferred mpreferred 1166.141121
|
||||||
|
2011-03-02T00:00:00.000Z spot automotive preferred apreferred 174.890520
|
||||||
|
2011-03-02T00:00:00.000Z spot entertainment preferred epreferred 147.117434
|
||||||
|
2011-03-02T00:00:00.000Z spot mezzanine preferred mpreferred 109.239196
|
||||||
|
2011-03-02T00:00:00.000Z spot premium preferred ppreferred 120.212473
|
||||||
|
2011-03-02T00:00:00.000Z spot travel preferred tpreferred 116.460744
|
||||||
|
2011-03-02T00:00:00.000Z total_market premium preferred ppreferred 1143.078414
|
||||||
|
2011-03-02T00:00:00.000Z upfront premium preferred ppreferred 865.777900
|
||||||
|
2011-03-03T00:00:00.000Z spot business preferred bpreferred 92.537499
|
||||||
|
2011-03-03T00:00:00.000Z spot health preferred hpreferred 108.914095
|
||||||
|
2011-03-03T00:00:00.000Z spot news preferred npreferred 96.031371
|
||||||
|
2011-03-03T00:00:00.000Z spot technology preferred tpreferred 75.977564
|
||||||
|
2011-03-03T00:00:00.000Z total_market mezzanine preferred mpreferred 1097.490771
|
||||||
|
2011-03-03T00:00:00.000Z upfront mezzanine preferred mpreferred 901.307577
|
||||||
|
2011-03-04T00:00:00.000Z spot automotive preferred apreferred 119.851231
|
||||||
|
2011-03-04T00:00:00.000Z spot entertainment preferred epreferred 132.832331
|
||||||
|
2011-03-04T00:00:00.000Z spot mezzanine preferred mpreferred 107.285615
|
||||||
|
2011-03-04T00:00:00.000Z spot premium preferred ppreferred 113.883056
|
||||||
|
2011-03-04T00:00:00.000Z spot travel preferred tpreferred 109.607245
|
||||||
|
2011-03-04T00:00:00.000Z total_market premium preferred ppreferred 1179.803776
|
||||||
|
2011-03-04T00:00:00.000Z upfront premium preferred ppreferred 975.577927
|
||||||
|
2011-03-05T00:00:00.000Z spot business preferred bpreferred 97.942645
|
||||||
|
2011-03-05T00:00:00.000Z spot health preferred hpreferred 108.394611
|
||||||
|
2011-03-05T00:00:00.000Z spot news preferred npreferred 102.486832
|
||||||
|
2011-03-05T00:00:00.000Z spot technology preferred tpreferred 81.105110
|
||||||
|
2011-03-05T00:00:00.000Z total_market mezzanine preferred mpreferred 1281.601175
|
||||||
|
2011-03-05T00:00:00.000Z upfront mezzanine preferred mpreferred 1360.694785
|
||||||
|
2011-03-06T00:00:00.000Z spot automotive preferred apreferred 129.531062
|
||||||
|
2011-03-06T00:00:00.000Z spot entertainment preferred epreferred 144.925734
|
||||||
|
2011-03-06T00:00:00.000Z spot mezzanine preferred mpreferred 113.035167
|
||||||
|
2011-03-06T00:00:00.000Z spot premium preferred ppreferred 115.956859
|
||||||
|
2011-03-06T00:00:00.000Z spot travel preferred tpreferred 120.953163
|
||||||
|
2011-03-06T00:00:00.000Z total_market premium preferred ppreferred 1103.239788
|
||||||
|
2011-03-06T00:00:00.000Z upfront premium preferred ppreferred 869.308360
|
||||||
|
2011-03-07T00:00:00.000Z spot business preferred bpreferred 105.575929
|
||||||
|
2011-03-07T00:00:00.000Z spot health preferred hpreferred 118.024245
|
||||||
|
2011-03-07T00:00:00.000Z spot news preferred npreferred 107.220218
|
||||||
|
2011-03-07T00:00:00.000Z spot technology preferred tpreferred 68.699125
|
||||||
|
2011-03-07T00:00:00.000Z total_market mezzanine preferred mpreferred 1177.858403
|
||||||
|
2011-03-07T00:00:00.000Z upfront mezzanine preferred mpreferred 1095.637520
|
||||||
|
2011-03-08T00:00:00.000Z spot automotive preferred apreferred 109.764955
|
||||||
|
2011-03-08T00:00:00.000Z spot entertainment preferred epreferred 143.214331
|
||||||
|
2011-03-08T00:00:00.000Z spot mezzanine preferred mpreferred 107.465492
|
||||||
|
2011-03-08T00:00:00.000Z spot premium preferred ppreferred 122.692722
|
||||||
|
2011-03-08T00:00:00.000Z spot travel preferred tpreferred 111.704071
|
||||||
|
2011-03-08T00:00:00.000Z total_market premium preferred ppreferred 1084.332554
|
||||||
|
2011-03-08T00:00:00.000Z upfront premium preferred ppreferred 835.762631
|
||||||
|
2011-03-09T00:00:00.000Z spot business preferred bpreferred 110.873407
|
||||||
|
2011-03-09T00:00:00.000Z spot health preferred hpreferred 115.013313
|
||||||
|
2011-03-09T00:00:00.000Z spot news preferred npreferred 112.407868
|
||||||
|
2011-03-09T00:00:00.000Z spot technology preferred tpreferred 82.426362
|
||||||
|
2011-03-09T00:00:00.000Z total_market mezzanine preferred mpreferred 1029.802500
|
||||||
|
2011-03-09T00:00:00.000Z upfront mezzanine preferred mpreferred 776.702940
|
||||||
|
2011-03-10T00:00:00.000Z spot automotive preferred apreferred 148.809150
|
||||||
|
2011-03-10T00:00:00.000Z spot entertainment preferred epreferred 134.212714
|
||||||
|
2011-03-10T00:00:00.000Z spot mezzanine preferred mpreferred 119.613508
|
||||||
|
2011-03-10T00:00:00.000Z spot premium preferred ppreferred 118.864028
|
||||||
|
2011-03-10T00:00:00.000Z spot travel preferred tpreferred 107.706257
|
||||||
|
2011-03-10T00:00:00.000Z total_market premium preferred ppreferred 1077.279402
|
||||||
|
2011-03-10T00:00:00.000Z upfront premium preferred ppreferred 835.461226
|
||||||
|
2011-03-11T00:00:00.000Z spot business preferred bpreferred 106.898536
|
||||||
|
2011-03-11T00:00:00.000Z spot health preferred hpreferred 112.856230
|
||||||
|
2011-03-11T00:00:00.000Z spot news preferred npreferred 108.135811
|
||||||
|
2011-03-11T00:00:00.000Z spot technology preferred tpreferred 67.731170
|
||||||
|
2011-03-11T00:00:00.000Z total_market mezzanine preferred mpreferred 1098.543170
|
||||||
|
2011-03-11T00:00:00.000Z upfront mezzanine preferred mpreferred 715.516125
|
||||||
|
2011-03-12T00:00:00.000Z spot automotive preferred apreferred 155.728048
|
||||||
|
2011-03-12T00:00:00.000Z spot entertainment preferred epreferred 132.687079
|
||||||
|
2011-03-12T00:00:00.000Z spot mezzanine preferred mpreferred 114.681603
|
||||||
|
2011-03-12T00:00:00.000Z spot premium preferred ppreferred 118.574721
|
||||||
|
2011-03-12T00:00:00.000Z spot travel preferred tpreferred 109.384493
|
||||||
|
2011-03-12T00:00:00.000Z total_market premium preferred ppreferred 1008.745525
|
||||||
|
2011-03-12T00:00:00.000Z upfront premium preferred ppreferred 764.508070
|
||||||
|
2011-03-13T00:00:00.000Z spot business preferred bpreferred 113.760384
|
||||||
|
2011-03-13T00:00:00.000Z spot health preferred hpreferred 120.760130
|
||||||
|
2011-03-13T00:00:00.000Z spot news preferred npreferred 114.814070
|
||||||
|
2011-03-13T00:00:00.000Z spot technology preferred tpreferred 70.126017
|
||||||
|
2011-03-13T00:00:00.000Z total_market mezzanine preferred mpreferred 998.753955
|
||||||
|
2011-03-13T00:00:00.000Z upfront mezzanine preferred mpreferred 736.409261
|
||||||
|
2011-03-14T00:00:00.000Z spot automotive preferred apreferred 153.191744
|
||||||
|
2011-03-14T00:00:00.000Z spot entertainment preferred epreferred 123.248581
|
||||||
|
2011-03-14T00:00:00.000Z spot mezzanine preferred mpreferred 104.716583
|
||||||
|
2011-03-14T00:00:00.000Z spot premium preferred ppreferred 122.275869
|
||||||
|
2011-03-14T00:00:00.000Z spot travel preferred tpreferred 104.724023
|
||||||
|
2011-03-14T00:00:00.000Z total_market premium preferred ppreferred 1141.588400
|
||||||
|
2011-03-14T00:00:00.000Z upfront premium preferred ppreferred 893.985017
|
||||||
|
2011-03-15T00:00:00.000Z spot business preferred bpreferred 109.382273
|
||||||
|
2011-03-15T00:00:00.000Z spot health preferred hpreferred 119.250945
|
||||||
|
2011-03-15T00:00:00.000Z spot news preferred npreferred 112.354294
|
||||||
|
2011-03-15T00:00:00.000Z spot technology preferred tpreferred 59.266595
|
||||||
|
2011-03-15T00:00:00.000Z total_market mezzanine preferred mpreferred 1037.381049
|
||||||
|
2011-03-15T00:00:00.000Z upfront mezzanine preferred mpreferred 832.874861
|
||||||
|
2011-03-16T00:00:00.000Z spot automotive preferred apreferred 147.471464
|
||||||
|
2011-03-16T00:00:00.000Z spot entertainment preferred epreferred 110.070846
|
||||||
|
2011-03-16T00:00:00.000Z spot mezzanine preferred mpreferred 99.137980
|
||||||
|
2011-03-16T00:00:00.000Z spot premium preferred ppreferred 120.455865
|
||||||
|
2011-03-16T00:00:00.000Z spot travel preferred tpreferred 107.903885
|
||||||
|
2011-03-16T00:00:00.000Z total_market premium preferred ppreferred 1092.942008
|
||||||
|
2011-03-16T00:00:00.000Z upfront premium preferred ppreferred 848.339888
|
||||||
|
2011-03-17T00:00:00.000Z spot business preferred bpreferred 113.501786
|
||||||
|
2011-03-17T00:00:00.000Z spot health preferred hpreferred 114.540037
|
||||||
|
2011-03-17T00:00:00.000Z spot news preferred npreferred 116.816958
|
||||||
|
2011-03-17T00:00:00.000Z spot technology preferred tpreferred 69.258523
|
||||||
|
2011-03-17T00:00:00.000Z total_market mezzanine preferred mpreferred 1072.239320
|
||||||
|
2011-03-17T00:00:00.000Z upfront mezzanine preferred mpreferred 878.683776
|
||||||
|
2011-03-18T00:00:00.000Z spot automotive preferred apreferred 180.343171
|
||||||
|
2011-03-18T00:00:00.000Z spot entertainment preferred epreferred 130.260926
|
||||||
|
2011-03-18T00:00:00.000Z spot mezzanine preferred mpreferred 111.540639
|
||||||
|
2011-03-18T00:00:00.000Z spot premium preferred ppreferred 119.629977
|
||||||
|
2011-03-18T00:00:00.000Z spot travel preferred tpreferred 114.564808
|
||||||
|
2011-03-18T00:00:00.000Z total_market premium preferred ppreferred 1176.605164
|
||||||
|
2011-03-18T00:00:00.000Z upfront premium preferred ppreferred 936.429632
|
||||||
|
2011-03-19T00:00:00.000Z spot business preferred bpreferred 109.788875
|
||||||
|
2011-03-19T00:00:00.000Z spot health preferred hpreferred 117.197085
|
||||||
|
2011-03-19T00:00:00.000Z spot news preferred npreferred 112.236468
|
||||||
|
2011-03-19T00:00:00.000Z spot technology preferred tpreferred 72.369471
|
||||||
|
2011-03-19T00:00:00.000Z total_market mezzanine preferred mpreferred 1107.220174
|
||||||
|
2011-03-19T00:00:00.000Z upfront mezzanine preferred mpreferred 846.288386
|
||||||
|
2011-03-20T00:00:00.000Z spot automotive preferred apreferred 178.454262
|
||||||
|
2011-03-20T00:00:00.000Z spot entertainment preferred epreferred 157.749330
|
||||||
|
2011-03-20T00:00:00.000Z spot mezzanine preferred mpreferred 128.274705
|
||||||
|
2011-03-20T00:00:00.000Z spot premium preferred ppreferred 129.519442
|
||||||
|
2011-03-20T00:00:00.000Z spot travel preferred tpreferred 128.705337
|
||||||
|
2011-03-20T00:00:00.000Z total_market premium preferred ppreferred 1217.547439
|
||||||
|
2011-03-20T00:00:00.000Z upfront premium preferred ppreferred 954.754185
|
||||||
|
2011-03-21T00:00:00.000Z spot business preferred bpreferred 124.411632
|
||||||
|
2011-03-21T00:00:00.000Z spot health preferred hpreferred 122.462424
|
||||||
|
2011-03-21T00:00:00.000Z spot news preferred npreferred 125.243882
|
||||||
|
2011-03-21T00:00:00.000Z spot technology preferred tpreferred 75.936640
|
||||||
|
2011-03-21T00:00:00.000Z total_market mezzanine preferred mpreferred 1283.957016
|
||||||
|
2011-03-21T00:00:00.000Z upfront mezzanine preferred mpreferred 1031.990042
|
||||||
|
2011-03-22T00:00:00.000Z spot automotive preferred apreferred 177.460613
|
||||||
|
2011-03-22T00:00:00.000Z spot entertainment preferred epreferred 151.407583
|
||||||
|
2011-03-22T00:00:00.000Z spot mezzanine preferred mpreferred 126.415884
|
||||||
|
2011-03-22T00:00:00.000Z spot premium preferred ppreferred 133.124963
|
||||||
|
2011-03-22T00:00:00.000Z spot travel preferred tpreferred 122.357549
|
||||||
|
2011-03-22T00:00:00.000Z total_market premium preferred ppreferred 1119.247202
|
||||||
|
2011-03-22T00:00:00.000Z upfront premium preferred ppreferred 843.952139
|
||||||
|
2011-03-23T00:00:00.000Z spot business preferred bpreferred 126.764513
|
||||||
|
2011-03-23T00:00:00.000Z spot health preferred hpreferred 118.972310
|
||||||
|
2011-03-23T00:00:00.000Z spot news preferred npreferred 129.864384
|
||||||
|
2011-03-23T00:00:00.000Z spot technology preferred tpreferred 102.603152
|
||||||
|
2011-03-23T00:00:00.000Z total_market mezzanine preferred mpreferred 1403.338838
|
||||||
|
2011-03-23T00:00:00.000Z upfront mezzanine preferred mpreferred 1362.650586
|
||||||
|
2011-03-24T00:00:00.000Z spot automotive preferred apreferred 135.569784
|
||||||
|
2011-03-24T00:00:00.000Z spot entertainment preferred epreferred 156.650862
|
||||||
|
2011-03-24T00:00:00.000Z spot mezzanine preferred mpreferred 115.945757
|
||||||
|
2011-03-24T00:00:00.000Z spot premium preferred ppreferred 129.023978
|
||||||
|
2011-03-24T00:00:00.000Z spot travel preferred tpreferred 124.062061
|
||||||
|
2011-03-24T00:00:00.000Z total_market premium preferred ppreferred 1137.842315
|
||||||
|
2011-03-24T00:00:00.000Z upfront premium preferred ppreferred 873.652030
|
||||||
|
2011-03-25T00:00:00.000Z spot business preferred bpreferred 125.766952
|
||||||
|
2011-03-25T00:00:00.000Z spot health preferred hpreferred 119.400783
|
||||||
|
2011-03-25T00:00:00.000Z spot news preferred npreferred 126.057960
|
||||||
|
2011-03-25T00:00:00.000Z spot technology preferred tpreferred 89.408906
|
||||||
|
2011-03-25T00:00:00.000Z total_market mezzanine preferred mpreferred 1124.935193
|
||||||
|
2011-03-25T00:00:00.000Z upfront mezzanine preferred mpreferred 855.717712
|
||||||
|
2011-03-26T00:00:00.000Z spot automotive preferred apreferred 148.957194
|
||||||
|
2011-03-26T00:00:00.000Z spot entertainment preferred epreferred 162.815450
|
||||||
|
2011-03-26T00:00:00.000Z spot mezzanine preferred mpreferred 117.449116
|
||||||
|
2011-03-26T00:00:00.000Z spot premium preferred ppreferred 139.214665
|
||||||
|
2011-03-26T00:00:00.000Z spot travel preferred tpreferred 138.663182
|
||||||
|
2011-03-26T00:00:00.000Z total_market premium preferred ppreferred 1247.890809
|
||||||
|
2011-03-26T00:00:00.000Z upfront premium preferred ppreferred 962.235801
|
||||||
|
2011-03-27T00:00:00.000Z spot business preferred bpreferred 135.183271
|
||||||
|
2011-03-27T00:00:00.000Z spot health preferred hpreferred 130.599006
|
||||||
|
2011-03-27T00:00:00.000Z spot mezzanine preferred mpreferred 114.952545
|
||||||
|
2011-03-27T00:00:00.000Z spot news preferred npreferred 139.294248
|
||||||
|
2011-03-27T00:00:00.000Z spot premium preferred ppreferred 142.430177
|
||||||
|
2011-03-27T00:00:00.000Z spot technology preferred tpreferred 108.489598
|
||||||
|
2011-03-27T00:00:00.000Z spot travel preferred tpreferred 141.820068
|
||||||
|
2011-03-27T00:00:00.000Z total_market mezzanine preferred mpreferred 1185.709973
|
||||||
|
2011-03-27T00:00:00.000Z total_market premium preferred ppreferred 1345.781728
|
||||||
|
2011-03-27T00:00:00.000Z upfront mezzanine preferred mpreferred 1019.898509
|
||||||
|
2011-03-27T00:00:00.000Z upfront premium preferred ppreferred 1056.419292
|
||||||
|
2011-03-28T00:00:00.000Z spot automotive preferred apreferred 156.155294
|
||||||
|
2011-03-28T00:00:00.000Z spot business preferred bpreferred 124.336139
|
||||||
|
2011-03-28T00:00:00.000Z spot entertainment preferred epreferred 163.100154
|
||||||
|
2011-03-28T00:00:00.000Z spot health preferred hpreferred 131.191818
|
||||||
|
2011-03-28T00:00:00.000Z spot mezzanine preferred mpreferred 115.709767
|
||||||
|
2011-03-28T00:00:00.000Z spot news preferred npreferred 127.403089
|
||||||
|
2011-03-28T00:00:00.000Z spot premium preferred ppreferred 140.941296
|
||||||
|
2011-03-28T00:00:00.000Z spot technology preferred tpreferred 103.578536
|
||||||
|
2011-03-28T00:00:00.000Z spot travel preferred tpreferred 130.880788
|
||||||
|
2011-03-28T00:00:00.000Z total_market mezzanine preferred mpreferred 1250.166788
|
||||||
|
2011-03-28T00:00:00.000Z total_market premium preferred ppreferred 1390.754050
|
||||||
|
2011-03-28T00:00:00.000Z upfront mezzanine preferred mpreferred 1198.723103
|
||||||
|
2011-03-28T00:00:00.000Z upfront premium preferred ppreferred 1108.136072
|
||||||
|
2011-03-29T00:00:00.000Z spot automotive preferred apreferred 134.084672
|
||||||
|
2011-03-29T00:00:00.000Z spot business preferred bpreferred 117.626804
|
||||||
|
2011-03-29T00:00:00.000Z spot entertainment preferred epreferred 158.073319
|
||||||
|
2011-03-29T00:00:00.000Z spot health preferred hpreferred 128.074393
|
||||||
|
2011-03-29T00:00:00.000Z spot mezzanine preferred mpreferred 109.678515
|
||||||
|
2011-03-29T00:00:00.000Z spot news preferred npreferred 122.620188
|
||||||
|
2011-03-29T00:00:00.000Z spot premium preferred ppreferred 144.446039
|
||||||
|
2011-03-29T00:00:00.000Z spot technology preferred tpreferred 91.604463
|
||||||
|
2011-03-29T00:00:00.000Z spot travel preferred tpreferred 132.477651
|
||||||
|
2011-03-29T00:00:00.000Z total_market mezzanine preferred mpreferred 1224.116225
|
||||||
|
2011-03-29T00:00:00.000Z total_market premium preferred ppreferred 1361.080245
|
||||||
|
2011-03-29T00:00:00.000Z upfront mezzanine preferred mpreferred 1289.009485
|
||||||
|
2011-03-29T00:00:00.000Z upfront premium preferred ppreferred 1069.431801
|
||||||
|
2011-03-30T00:00:00.000Z spot automotive preferred apreferred 135.942820
|
||||||
|
2011-03-30T00:00:00.000Z spot business preferred bpreferred 120.283054
|
||||||
|
2011-03-30T00:00:00.000Z spot entertainment preferred epreferred 167.960620
|
||||||
|
2011-03-30T00:00:00.000Z spot health preferred hpreferred 121.215839
|
||||||
|
2011-03-30T00:00:00.000Z spot mezzanine preferred mpreferred 116.587746
|
||||||
|
2011-03-30T00:00:00.000Z spot news preferred npreferred 122.612114
|
||||||
|
2011-03-30T00:00:00.000Z spot premium preferred ppreferred 138.827311
|
||||||
|
2011-03-30T00:00:00.000Z spot technology preferred tpreferred 93.331887
|
||||||
|
2011-03-30T00:00:00.000Z spot travel preferred tpreferred 128.645571
|
||||||
|
2011-03-30T00:00:00.000Z total_market mezzanine preferred mpreferred 1190.933753
|
||||||
|
2011-03-30T00:00:00.000Z total_market premium preferred ppreferred 1310.797070
|
||||||
|
2011-03-30T00:00:00.000Z upfront mezzanine preferred mpreferred 1001.134025
|
||||||
|
2011-03-30T00:00:00.000Z upfront premium preferred ppreferred 1030.499562
|
||||||
|
2011-03-31T00:00:00.000Z spot automotive preferred apreferred 151.752485
|
||||||
|
2011-03-31T00:00:00.000Z spot business preferred bpreferred 124.414321
|
||||||
|
2011-03-31T00:00:00.000Z spot entertainment preferred epreferred 175.778647
|
||||||
|
2011-03-31T00:00:00.000Z spot health preferred hpreferred 120.607382
|
||||||
|
2011-03-31T00:00:00.000Z spot mezzanine preferred mpreferred 117.060598
|
||||||
|
2011-03-31T00:00:00.000Z spot news preferred npreferred 125.243245
|
||||||
|
2011-03-31T00:00:00.000Z spot premium preferred ppreferred 150.247713
|
||||||
|
2011-03-31T00:00:00.000Z spot technology preferred tpreferred 93.390841
|
||||||
|
2011-03-31T00:00:00.000Z spot travel preferred tpreferred 125.839686
|
||||||
|
2011-03-31T00:00:00.000Z total_market mezzanine preferred mpreferred 1466.209327
|
||||||
|
2011-03-31T00:00:00.000Z total_market premium preferred ppreferred 1366.447617
|
||||||
|
2011-03-31T00:00:00.000Z upfront mezzanine preferred mpreferred 1734.274909
|
||||||
|
2011-03-31T00:00:00.000Z upfront premium preferred ppreferred 1063.201156
|
||||||
|
2011-04-01T00:00:00.000Z spot automotive preferred apreferred 135.885094
|
||||||
|
2011-04-01T00:00:00.000Z spot business preferred bpreferred 118.570340
|
||||||
|
2011-04-01T00:00:00.000Z spot entertainment preferred epreferred 158.747224
|
||||||
|
2011-04-01T00:00:00.000Z spot health preferred hpreferred 120.134704
|
||||||
|
2011-04-01T00:00:00.000Z spot mezzanine preferred mpreferred 109.705815
|
||||||
|
2011-04-01T00:00:00.000Z spot news preferred npreferred 121.583581
|
||||||
|
2011-04-01T00:00:00.000Z spot premium preferred ppreferred 144.507368
|
||||||
|
2011-04-01T00:00:00.000Z spot technology preferred tpreferred 78.622547
|
||||||
|
2011-04-01T00:00:00.000Z spot travel preferred tpreferred 119.922742
|
||||||
|
2011-04-01T00:00:00.000Z total_market mezzanine preferred mpreferred 1314.839715
|
||||||
|
2011-04-01T00:00:00.000Z total_market premium preferred ppreferred 1522.043733
|
||||||
|
2011-04-01T00:00:00.000Z upfront mezzanine preferred mpreferred 1447.341160
|
||||||
|
2011-04-01T00:00:00.000Z upfront premium preferred ppreferred 1234.247546
|
||||||
|
2011-04-02T00:00:00.000Z spot automotive preferred apreferred 147.425935
|
||||||
|
2011-04-02T00:00:00.000Z spot business preferred bpreferred 112.987027
|
||||||
|
2011-04-02T00:00:00.000Z spot entertainment preferred epreferred 166.016049
|
||||||
|
2011-04-02T00:00:00.000Z spot health preferred hpreferred 113.446008
|
||||||
|
2011-04-02T00:00:00.000Z spot mezzanine preferred mpreferred 110.931934
|
||||||
|
2011-04-02T00:00:00.000Z spot news preferred npreferred 114.290141
|
||||||
|
2011-04-02T00:00:00.000Z spot premium preferred ppreferred 135.301506
|
||||||
|
2011-04-02T00:00:00.000Z spot technology preferred tpreferred 97.387433
|
||||||
|
2011-04-02T00:00:00.000Z spot travel preferred tpreferred 126.411364
|
||||||
|
2011-04-02T00:00:00.000Z total_market mezzanine preferred mpreferred 1193.556278
|
||||||
|
2011-04-02T00:00:00.000Z total_market premium preferred ppreferred 1321.375057
|
||||||
|
2011-04-02T00:00:00.000Z upfront mezzanine preferred mpreferred 1144.342401
|
||||||
|
2011-04-02T00:00:00.000Z upfront premium preferred ppreferred 1049.738585
|
||||||
|
2011-04-03T00:00:00.000Z spot automotive preferred apreferred 122.971856
|
||||||
|
2011-04-03T00:00:00.000Z spot business preferred bpreferred 105.735462
|
||||||
|
2011-04-03T00:00:00.000Z spot entertainment preferred epreferred 153.927965
|
||||||
|
2011-04-03T00:00:00.000Z spot health preferred hpreferred 103.532351
|
||||||
|
2011-04-03T00:00:00.000Z spot mezzanine preferred mpreferred 107.047773
|
||||||
|
2011-04-03T00:00:00.000Z spot news preferred npreferred 107.919674
|
||||||
|
2011-04-03T00:00:00.000Z spot premium preferred ppreferred 122.141707
|
||||||
|
2011-04-03T00:00:00.000Z spot technology preferred tpreferred 80.861743
|
||||||
|
2011-04-03T00:00:00.000Z spot travel preferred tpreferred 117.247070
|
||||||
|
2011-04-03T00:00:00.000Z total_market mezzanine preferred mpreferred 1055.783661
|
||||||
|
2011-04-03T00:00:00.000Z total_market premium preferred ppreferred 1021.638673
|
||||||
|
2011-04-03T00:00:00.000Z upfront mezzanine preferred mpreferred 811.991286
|
||||||
|
2011-04-03T00:00:00.000Z upfront premium preferred ppreferred 768.423077
|
||||||
|
2011-04-04T00:00:00.000Z spot automotive preferred apreferred 110.919829
|
||||||
|
2011-04-04T00:00:00.000Z spot business preferred bpreferred 107.613577
|
||||||
|
2011-04-04T00:00:00.000Z spot entertainment preferred epreferred 146.729242
|
||||||
|
2011-04-04T00:00:00.000Z spot health preferred hpreferred 105.375351
|
||||||
|
2011-04-04T00:00:00.000Z spot mezzanine preferred mpreferred 110.573670
|
||||||
|
2011-04-04T00:00:00.000Z spot news preferred npreferred 114.382255
|
||||||
|
2011-04-04T00:00:00.000Z spot premium preferred ppreferred 125.285894
|
||||||
|
2011-04-04T00:00:00.000Z spot technology preferred tpreferred 72.668430
|
||||||
|
2011-04-04T00:00:00.000Z spot travel preferred tpreferred 117.703023
|
||||||
|
2011-04-04T00:00:00.000Z total_market mezzanine preferred mpreferred 1197.008423
|
||||||
|
2011-04-04T00:00:00.000Z total_market premium preferred ppreferred 1131.531986
|
||||||
|
2011-04-04T00:00:00.000Z upfront mezzanine preferred mpreferred 1151.069173
|
||||||
|
2011-04-04T00:00:00.000Z upfront premium preferred ppreferred 877.079396
|
||||||
|
2011-04-05T00:00:00.000Z spot automotive preferred apreferred 113.318712
|
||||||
|
2011-04-05T00:00:00.000Z spot business preferred bpreferred 105.615563
|
||||||
|
2011-04-05T00:00:00.000Z spot entertainment preferred epreferred 141.713507
|
||||||
|
2011-04-05T00:00:00.000Z spot health preferred hpreferred 106.207931
|
||||||
|
2011-04-05T00:00:00.000Z spot mezzanine preferred mpreferred 109.890586
|
||||||
|
2011-04-05T00:00:00.000Z spot news preferred npreferred 110.012987
|
||||||
|
2011-04-05T00:00:00.000Z spot premium preferred ppreferred 124.478408
|
||||||
|
2011-04-05T00:00:00.000Z spot technology preferred tpreferred 86.683603
|
||||||
|
2011-04-05T00:00:00.000Z spot travel preferred tpreferred 117.051694
|
||||||
|
2011-04-05T00:00:00.000Z total_market mezzanine preferred mpreferred 1192.144303
|
||||||
|
2011-04-05T00:00:00.000Z total_market premium preferred ppreferred 1154.289559
|
||||||
|
2011-04-05T00:00:00.000Z upfront mezzanine preferred mpreferred 1146.423036
|
||||||
|
2011-04-05T00:00:00.000Z upfront premium preferred ppreferred 902.615706
|
||||||
|
2011-04-06T00:00:00.000Z spot automotive preferred apreferred 115.334018
|
||||||
|
2011-04-06T00:00:00.000Z spot business preferred bpreferred 109.700256
|
||||||
|
2011-04-06T00:00:00.000Z spot entertainment preferred epreferred 147.553562
|
||||||
|
2011-04-06T00:00:00.000Z spot health preferred hpreferred 100.775597
|
||||||
|
2011-04-06T00:00:00.000Z spot mezzanine preferred mpreferred 108.659345
|
||||||
|
2011-04-06T00:00:00.000Z spot news preferred npreferred 113.408308
|
||||||
|
2011-04-06T00:00:00.000Z spot premium preferred ppreferred 121.079585
|
||||||
|
2011-04-06T00:00:00.000Z spot technology preferred tpreferred 92.336403
|
||||||
|
2011-04-06T00:00:00.000Z spot travel preferred tpreferred 112.921482
|
||||||
|
2011-04-06T00:00:00.000Z total_market mezzanine preferred mpreferred 1272.677122
|
||||||
|
2011-04-06T00:00:00.000Z total_market premium preferred ppreferred 1141.514652
|
||||||
|
2011-04-06T00:00:00.000Z upfront mezzanine preferred mpreferred 1355.843374
|
||||||
|
2011-04-06T00:00:00.000Z upfront premium preferred ppreferred 897.393445
|
||||||
|
2011-04-07T00:00:00.000Z spot automotive preferred apreferred 117.508062
|
||||||
|
2011-04-07T00:00:00.000Z spot business preferred bpreferred 115.418054
|
||||||
|
2011-04-07T00:00:00.000Z spot entertainment preferred epreferred 141.565031
|
||||||
|
2011-04-07T00:00:00.000Z spot health preferred hpreferred 95.562446
|
||||||
|
2011-04-07T00:00:00.000Z spot mezzanine preferred mpreferred 102.933171
|
||||||
|
2011-04-07T00:00:00.000Z spot news preferred npreferred 122.696621
|
||||||
|
2011-04-07T00:00:00.000Z spot premium preferred ppreferred 119.763135
|
||||||
|
2011-04-07T00:00:00.000Z spot technology preferred tpreferred 84.357701
|
||||||
|
2011-04-07T00:00:00.000Z spot travel preferred tpreferred 117.432760
|
||||||
|
2011-04-07T00:00:00.000Z total_market mezzanine preferred mpreferred 1190.896088
|
||||||
|
2011-04-07T00:00:00.000Z total_market premium preferred ppreferred 1009.363132
|
||||||
|
2011-04-07T00:00:00.000Z upfront mezzanine preferred mpreferred 1260.143027
|
||||||
|
2011-04-07T00:00:00.000Z upfront premium preferred ppreferred 762.862488
|
||||||
|
2011-04-08T00:00:00.000Z spot automotive preferred apreferred 120.973860
|
||||||
|
2011-04-08T00:00:00.000Z spot business preferred bpreferred 116.248860
|
||||||
|
2011-04-08T00:00:00.000Z spot entertainment preferred epreferred 146.830618
|
||||||
|
2011-04-08T00:00:00.000Z spot health preferred hpreferred 96.226609
|
||||||
|
2011-04-08T00:00:00.000Z spot mezzanine preferred mpreferred 95.447888
|
||||||
|
2011-04-08T00:00:00.000Z spot news preferred npreferred 120.709912
|
||||||
|
2011-04-08T00:00:00.000Z spot premium preferred ppreferred 120.309688
|
||||||
|
2011-04-08T00:00:00.000Z spot technology preferred tpreferred 94.631354
|
||||||
|
2011-04-08T00:00:00.000Z spot travel preferred tpreferred 122.743898
|
||||||
|
2011-04-08T00:00:00.000Z total_market mezzanine preferred mpreferred 1006.913816
|
||||||
|
2011-04-08T00:00:00.000Z total_market premium preferred ppreferred 1032.599837
|
||||||
|
2011-04-08T00:00:00.000Z upfront mezzanine preferred mpreferred 935.168026
|
||||||
|
2011-04-08T00:00:00.000Z upfront premium preferred ppreferred 782.107861
|
||||||
|
2011-04-09T00:00:00.000Z spot automotive preferred apreferred 116.080323
|
||||||
|
2011-04-09T00:00:00.000Z spot business preferred bpreferred 116.060759
|
||||||
|
2011-04-09T00:00:00.000Z spot entertainment preferred epreferred 158.682525
|
||||||
|
2011-04-09T00:00:00.000Z spot health preferred hpreferred 95.509796
|
||||||
|
2011-04-09T00:00:00.000Z spot mezzanine preferred mpreferred 110.007248
|
||||||
|
2011-04-09T00:00:00.000Z spot news preferred npreferred 121.905685
|
||||||
|
2011-04-09T00:00:00.000Z spot premium preferred ppreferred 126.369367
|
||||||
|
2011-04-09T00:00:00.000Z spot technology preferred tpreferred 92.905070
|
||||||
|
2011-04-09T00:00:00.000Z spot travel preferred tpreferred 123.791320
|
||||||
|
2011-04-09T00:00:00.000Z total_market mezzanine preferred mpreferred 1137.385764
|
||||||
|
2011-04-09T00:00:00.000Z total_market premium preferred ppreferred 1030.075553
|
||||||
|
2011-04-09T00:00:00.000Z upfront mezzanine preferred mpreferred 996.205369
|
||||||
|
2011-04-09T00:00:00.000Z upfront premium preferred ppreferred 767.692135
|
||||||
|
2011-04-10T00:00:00.000Z spot automotive preferred apreferred 113.221448
|
||||||
|
2011-04-10T00:00:00.000Z spot business preferred bpreferred 95.570457
|
||||||
|
2011-04-10T00:00:00.000Z spot entertainment preferred epreferred 131.766616
|
||||||
|
2011-04-10T00:00:00.000Z spot health preferred hpreferred 99.950855
|
||||||
|
2011-04-10T00:00:00.000Z spot mezzanine preferred mpreferred 91.470524
|
||||||
|
2011-04-10T00:00:00.000Z spot news preferred npreferred 99.393076
|
||||||
|
2011-04-10T00:00:00.000Z spot premium preferred ppreferred 123.207579
|
||||||
|
2011-04-10T00:00:00.000Z spot technology preferred tpreferred 84.898691
|
||||||
|
2011-04-10T00:00:00.000Z spot travel preferred tpreferred 114.353962
|
||||||
|
2011-04-10T00:00:00.000Z total_market mezzanine preferred mpreferred 1005.253077
|
||||||
|
2011-04-10T00:00:00.000Z total_market premium preferred ppreferred 1030.094757
|
||||||
|
2011-04-10T00:00:00.000Z upfront mezzanine preferred mpreferred 1031.741509
|
||||||
|
2011-04-10T00:00:00.000Z upfront premium preferred ppreferred 775.965555
|
||||||
|
2011-04-11T00:00:00.000Z spot automotive preferred apreferred 130.165796
|
||||||
|
2011-04-11T00:00:00.000Z spot business preferred bpreferred 107.765101
|
||||||
|
2011-04-11T00:00:00.000Z spot entertainment preferred epreferred 142.751726
|
||||||
|
2011-04-11T00:00:00.000Z spot health preferred hpreferred 104.847285
|
||||||
|
2011-04-11T00:00:00.000Z spot mezzanine preferred mpreferred 95.272956
|
||||||
|
2011-04-11T00:00:00.000Z spot news preferred npreferred 106.229286
|
||||||
|
2011-04-11T00:00:00.000Z spot premium preferred ppreferred 126.823859
|
||||||
|
2011-04-11T00:00:00.000Z spot technology preferred tpreferred 89.250155
|
||||||
|
2011-04-11T00:00:00.000Z spot travel preferred tpreferred 122.049678
|
||||||
|
2011-04-11T00:00:00.000Z total_market mezzanine preferred mpreferred 1112.794811
|
||||||
|
2011-04-11T00:00:00.000Z total_market premium preferred ppreferred 1113.357530
|
||||||
|
2011-04-11T00:00:00.000Z upfront mezzanine preferred mpreferred 1374.968412
|
||||||
|
2011-04-11T00:00:00.000Z upfront premium preferred ppreferred 853.163039
|
||||||
|
2011-04-12T00:00:00.000Z spot automotive preferred apreferred 122.386348
|
||||||
|
2011-04-12T00:00:00.000Z spot business preferred bpreferred 106.380995
|
||||||
|
2011-04-12T00:00:00.000Z spot entertainment preferred epreferred 141.932300
|
||||||
|
2011-04-12T00:00:00.000Z spot health preferred hpreferred 103.142372
|
||||||
|
2011-04-12T00:00:00.000Z spot mezzanine preferred mpreferred 97.340631
|
||||||
|
2011-04-12T00:00:00.000Z spot news preferred npreferred 105.381244
|
||||||
|
2011-04-12T00:00:00.000Z spot premium preferred ppreferred 125.189098
|
||||||
|
2011-04-12T00:00:00.000Z spot technology preferred tpreferred 90.533391
|
||||||
|
2011-04-12T00:00:00.000Z spot travel preferred tpreferred 122.128172
|
||||||
|
2011-04-12T00:00:00.000Z total_market mezzanine preferred mpreferred 1153.974725
|
||||||
|
2011-04-12T00:00:00.000Z total_market premium preferred ppreferred 1069.640880
|
||||||
|
2011-04-12T00:00:00.000Z upfront mezzanine preferred mpreferred 1456.611830
|
||||||
|
2011-04-12T00:00:00.000Z upfront premium preferred ppreferred 811.925240
|
||||||
|
2011-04-13T00:00:00.000Z spot automotive preferred apreferred 122.688340
|
||||||
|
2011-04-13T00:00:00.000Z spot business preferred bpreferred 105.739623
|
||||||
|
2011-04-13T00:00:00.000Z spot entertainment preferred epreferred 136.983407
|
||||||
|
2011-04-13T00:00:00.000Z spot health preferred hpreferred 100.860813
|
||||||
|
2011-04-13T00:00:00.000Z spot mezzanine preferred mpreferred 94.839191
|
||||||
|
2011-04-13T00:00:00.000Z spot news preferred npreferred 105.261296
|
||||||
|
2011-04-13T00:00:00.000Z spot premium preferred ppreferred 119.836611
|
||||||
|
2011-04-13T00:00:00.000Z spot technology preferred tpreferred 91.972558
|
||||||
|
2011-04-13T00:00:00.000Z spot travel preferred tpreferred 120.145572
|
||||||
|
2011-04-13T00:00:00.000Z total_market mezzanine preferred mpreferred 1016.137449
|
||||||
|
2011-04-13T00:00:00.000Z total_market premium preferred ppreferred 994.902292
|
||||||
|
2011-04-13T00:00:00.000Z upfront mezzanine preferred mpreferred 989.032799
|
||||||
|
2011-04-13T00:00:00.000Z upfront premium preferred ppreferred 744.744657
|
||||||
|
2011-04-14T00:00:00.000Z spot automotive preferred apreferred 111.179339
|
||||||
|
2011-04-14T00:00:00.000Z spot business preferred bpreferred 101.984377
|
||||||
|
2011-04-14T00:00:00.000Z spot entertainment preferred epreferred 133.606430
|
||||||
|
2011-04-14T00:00:00.000Z spot health preferred hpreferred 99.738319
|
||||||
|
2011-04-14T00:00:00.000Z spot mezzanine preferred mpreferred 91.270553
|
||||||
|
2011-04-14T00:00:00.000Z spot news preferred npreferred 101.251756
|
||||||
|
2011-04-14T00:00:00.000Z spot premium preferred ppreferred 118.285128
|
||||||
|
2011-04-14T00:00:00.000Z spot technology preferred tpreferred 84.951300
|
||||||
|
2011-04-14T00:00:00.000Z spot travel preferred tpreferred 119.768525
|
||||||
|
2011-04-14T00:00:00.000Z total_market mezzanine preferred mpreferred 1032.154263
|
||||||
|
2011-04-14T00:00:00.000Z total_market premium preferred ppreferred 999.586450
|
||||||
|
2011-04-14T00:00:00.000Z upfront mezzanine preferred mpreferred 1166.401205
|
||||||
|
2011-04-14T00:00:00.000Z upfront premium preferred ppreferred 753.104985
|
||||||
|
2011-04-15T00:00:00.000Z spot automotive preferred apreferred 106.793700
|
||||||
|
2011-04-15T00:00:00.000Z spot business preferred bpreferred 94.469747
|
||||||
|
2011-04-15T00:00:00.000Z spot entertainment preferred epreferred 135.109191
|
||||||
|
2011-04-15T00:00:00.000Z spot health preferred hpreferred 99.596909
|
||||||
|
2011-04-15T00:00:00.000Z spot mezzanine preferred mpreferred 92.782760
|
||||||
|
2011-04-15T00:00:00.000Z spot news preferred npreferred 97.859766
|
||||||
|
2011-04-15T00:00:00.000Z spot premium preferred ppreferred 120.508160
|
||||||
|
2011-04-15T00:00:00.000Z spot technology preferred tpreferred 89.646236
|
||||||
|
2011-04-15T00:00:00.000Z spot travel preferred tpreferred 120.290348
|
||||||
|
2011-04-15T00:00:00.000Z total_market mezzanine preferred mpreferred 994.752744
|
||||||
|
2011-04-15T00:00:00.000Z total_market premium preferred ppreferred 1029.056992
|
||||||
|
2011-04-15T00:00:00.000Z upfront mezzanine preferred mpreferred 962.731172
|
||||||
|
2011-04-15T00:00:00.000Z upfront premium preferred ppreferred 780.271977
|
|
@ -0,0 +1,477 @@
|
||||||
|
2011-01-12T00:00:00.000Z spot automotive preferred apreferred 100.000000
|
||||||
|
2011-01-12T00:00:00.000Z spot entertainment preferred epreferred 100.000000
|
||||||
|
2011-01-12T00:00:00.000Z spot mezzanine preferred mpreferred 100.000000
|
||||||
|
2011-01-12T00:00:00.000Z spot premium preferred ppreferred 100.000000
|
||||||
|
2011-01-12T00:00:00.000Z spot travel preferred tpreferred 100.000000
|
||||||
|
2011-01-12T00:00:00.000Z total_market premium preferred ppreferred 1000.000000
|
||||||
|
2011-01-12T00:00:00.000Z upfront premium preferred ppreferred 800.000000
|
||||||
|
2011-01-13T00:00:00.000Z spot business preferred bpreferred 103.629399
|
||||||
|
2011-01-13T00:00:00.000Z spot health preferred hpreferred 114.947403
|
||||||
|
2011-01-13T00:00:00.000Z spot news preferred npreferred 102.851683
|
||||||
|
2011-01-13T00:00:00.000Z spot technology preferred tpreferred 111.356672
|
||||||
|
2011-01-13T00:00:00.000Z total_market mezzanine preferred mpreferred 1040.945505
|
||||||
|
2011-01-13T00:00:00.000Z upfront mezzanine preferred mpreferred 826.060182
|
||||||
|
2011-01-14T00:00:00.000Z spot automotive preferred apreferred 86.450372
|
||||||
|
2011-01-14T00:00:00.000Z spot entertainment preferred epreferred 109.573474
|
||||||
|
2011-01-14T00:00:00.000Z spot mezzanine preferred mpreferred 97.903068
|
||||||
|
2011-01-14T00:00:00.000Z spot premium preferred ppreferred 104.611784
|
||||||
|
2011-01-14T00:00:00.000Z spot travel preferred tpreferred 112.259958
|
||||||
|
2011-01-14T00:00:00.000Z total_market premium preferred ppreferred 1073.476545
|
||||||
|
2011-01-14T00:00:00.000Z upfront premium preferred ppreferred 869.643722
|
||||||
|
2011-01-15T00:00:00.000Z spot business preferred bpreferred 99.781645
|
||||||
|
2011-01-15T00:00:00.000Z spot health preferred hpreferred 87.954346
|
||||||
|
2011-01-15T00:00:00.000Z spot news preferred npreferred 99.383407
|
||||||
|
2011-01-15T00:00:00.000Z spot technology preferred tpreferred 111.680229
|
||||||
|
2011-01-15T00:00:00.000Z total_market mezzanine preferred mpreferred 1007.365510
|
||||||
|
2011-01-15T00:00:00.000Z upfront mezzanine preferred mpreferred 809.041763
|
||||||
|
2011-01-16T00:00:00.000Z spot automotive preferred apreferred 71.315931
|
||||||
|
2011-01-16T00:00:00.000Z spot entertainment preferred epreferred 99.007588
|
||||||
|
2011-01-16T00:00:00.000Z spot mezzanine preferred mpreferred 99.863171
|
||||||
|
2011-01-16T00:00:00.000Z spot premium preferred ppreferred 103.348007
|
||||||
|
2011-01-16T00:00:00.000Z spot travel preferred tpreferred 116.779610
|
||||||
|
2011-01-16T00:00:00.000Z total_market premium preferred ppreferred 1077.612663
|
||||||
|
2011-01-16T00:00:00.000Z upfront premium preferred ppreferred 879.988099
|
||||||
|
2011-01-17T00:00:00.000Z spot business preferred bpreferred 105.914315
|
||||||
|
2011-01-17T00:00:00.000Z spot health preferred hpreferred 110.157325
|
||||||
|
2011-01-17T00:00:00.000Z spot news preferred npreferred 103.615039
|
||||||
|
2011-01-17T00:00:00.000Z spot technology preferred tpreferred 89.901887
|
||||||
|
2011-01-17T00:00:00.000Z total_market mezzanine preferred mpreferred 1075.089574
|
||||||
|
2011-01-17T00:00:00.000Z upfront mezzanine preferred mpreferred 950.146770
|
||||||
|
2011-01-18T00:00:00.000Z spot automotive preferred apreferred 87.195139
|
||||||
|
2011-01-18T00:00:00.000Z spot entertainment preferred epreferred 94.452739
|
||||||
|
2011-01-18T00:00:00.000Z spot mezzanine preferred mpreferred 101.087367
|
||||||
|
2011-01-18T00:00:00.000Z spot premium preferred ppreferred 108.979936
|
||||||
|
2011-01-18T00:00:00.000Z spot travel preferred tpreferred 113.680094
|
||||||
|
2011-01-18T00:00:00.000Z total_market premium preferred ppreferred 937.061939
|
||||||
|
2011-01-18T00:00:00.000Z upfront premium preferred ppreferred 682.885525
|
||||||
|
2011-01-19T00:00:00.000Z spot business preferred bpreferred 106.700550
|
||||||
|
2011-01-19T00:00:00.000Z spot health preferred hpreferred 97.620315
|
||||||
|
2011-01-19T00:00:00.000Z spot news preferred npreferred 106.127830
|
||||||
|
2011-01-19T00:00:00.000Z spot technology preferred tpreferred 77.316731
|
||||||
|
2011-01-19T00:00:00.000Z total_market mezzanine preferred mpreferred 1156.744712
|
||||||
|
2011-01-19T00:00:00.000Z upfront mezzanine preferred mpreferred 1109.874950
|
||||||
|
2011-01-20T01:00:00.000Z spot automotive preferred apreferred 93.396274
|
||||||
|
2011-01-20T01:00:00.000Z spot entertainment preferred epreferred 90.439020
|
||||||
|
2011-01-20T01:00:00.000Z spot mezzanine preferred mpreferred 105.669498
|
||||||
|
2011-01-20T01:00:00.000Z spot premium preferred ppreferred 101.305541
|
||||||
|
2011-01-20T01:00:00.000Z spot travel preferred tpreferred 140.179069
|
||||||
|
2011-01-20T01:00:00.000Z total_market premium preferred ppreferred 904.340636
|
||||||
|
2011-01-20T01:00:00.000Z upfront premium preferred ppreferred 677.510973
|
||||||
|
2011-01-22T00:00:00.000Z spot automotive preferred apreferred 95.235266
|
||||||
|
2011-01-22T00:00:00.000Z spot entertainment preferred epreferred 93.681096
|
||||||
|
2011-01-22T00:00:00.000Z spot mezzanine preferred mpreferred 104.184494
|
||||||
|
2011-01-22T00:00:00.000Z spot premium preferred ppreferred 99.284525
|
||||||
|
2011-01-22T00:00:00.000Z spot travel preferred tpreferred 137.109783
|
||||||
|
2011-01-22T00:00:00.000Z total_market premium preferred ppreferred 1343.232494
|
||||||
|
2011-01-22T00:00:00.000Z upfront premium preferred ppreferred 1219.432170
|
||||||
|
2011-01-23T00:00:00.000Z spot business preferred bpreferred 107.348157
|
||||||
|
2011-01-23T00:00:00.000Z spot health preferred hpreferred 96.826443
|
||||||
|
2011-01-23T00:00:00.000Z spot news preferred npreferred 106.418686
|
||||||
|
2011-01-23T00:00:00.000Z spot technology preferred tpreferred 110.467875
|
||||||
|
2011-01-23T00:00:00.000Z total_market mezzanine preferred mpreferred 1088.943083
|
||||||
|
2011-01-23T00:00:00.000Z upfront mezzanine preferred mpreferred 979.306038
|
||||||
|
2011-01-24T00:00:00.000Z spot automotive preferred apreferred 96.671647
|
||||||
|
2011-01-24T00:00:00.000Z spot entertainment preferred epreferred 88.748460
|
||||||
|
2011-01-24T00:00:00.000Z spot mezzanine preferred mpreferred 97.906256
|
||||||
|
2011-01-24T00:00:00.000Z spot premium preferred ppreferred 101.581339
|
||||||
|
2011-01-24T00:00:00.000Z spot travel preferred tpreferred 131.695956
|
||||||
|
2011-01-24T00:00:00.000Z total_market premium preferred ppreferred 939.244103
|
||||||
|
2011-01-24T00:00:00.000Z upfront premium preferred ppreferred 716.609179
|
||||||
|
2011-01-25T00:00:00.000Z spot business preferred bpreferred 101.624789
|
||||||
|
2011-01-25T00:00:00.000Z spot health preferred hpreferred 180.575246
|
||||||
|
2011-01-25T00:00:00.000Z spot news preferred npreferred 102.907866
|
||||||
|
2011-01-25T00:00:00.000Z spot technology preferred tpreferred 102.044542
|
||||||
|
2011-01-25T00:00:00.000Z total_market mezzanine preferred mpreferred 1109.875413
|
||||||
|
2011-01-25T00:00:00.000Z upfront mezzanine preferred mpreferred 1301.023342
|
||||||
|
2011-01-26T00:00:00.000Z spot automotive preferred apreferred 84.906466
|
||||||
|
2011-01-26T00:00:00.000Z spot entertainment preferred epreferred 84.710523
|
||||||
|
2011-01-26T00:00:00.000Z spot mezzanine preferred mpreferred 96.046584
|
||||||
|
2011-01-26T00:00:00.000Z spot premium preferred ppreferred 101.088903
|
||||||
|
2011-01-26T00:00:00.000Z spot travel preferred tpreferred 122.160681
|
||||||
|
2011-01-26T00:00:00.000Z total_market premium preferred ppreferred 1686.419659
|
||||||
|
2011-01-26T00:00:00.000Z upfront premium preferred ppreferred 1609.096706
|
||||||
|
2011-01-27T00:00:00.000Z spot business preferred bpreferred 104.882908
|
||||||
|
2011-01-27T00:00:00.000Z spot health preferred hpreferred 170.735853
|
||||||
|
2011-01-27T00:00:00.000Z spot news preferred npreferred 104.609483
|
||||||
|
2011-01-27T00:00:00.000Z spot technology preferred tpreferred 116.979005
|
||||||
|
2011-01-27T00:00:00.000Z total_market mezzanine preferred mpreferred 1074.006938
|
||||||
|
2011-01-27T00:00:00.000Z upfront mezzanine preferred mpreferred 1023.295213
|
||||||
|
2011-01-28T00:00:00.000Z spot automotive preferred apreferred 123.006128
|
||||||
|
2011-01-28T00:00:00.000Z spot entertainment preferred epreferred 99.681629
|
||||||
|
2011-01-28T00:00:00.000Z spot mezzanine preferred mpreferred 106.075672
|
||||||
|
2011-01-28T00:00:00.000Z spot premium preferred ppreferred 103.822842
|
||||||
|
2011-01-28T00:00:00.000Z spot travel preferred tpreferred 158.739359
|
||||||
|
2011-01-28T00:00:00.000Z total_market premium preferred ppreferred 1021.334487
|
||||||
|
2011-01-28T00:00:00.000Z upfront premium preferred ppreferred 810.889422
|
||||||
|
2011-01-29T00:00:00.000Z spot business preferred bpreferred 100.992147
|
||||||
|
2011-01-29T00:00:00.000Z spot health preferred hpreferred 114.905745
|
||||||
|
2011-01-29T00:00:00.000Z spot news preferred npreferred 101.998823
|
||||||
|
2011-01-29T00:00:00.000Z spot technology preferred tpreferred 109.549035
|
||||||
|
2011-01-29T00:00:00.000Z total_market mezzanine preferred mpreferred 1314.619452
|
||||||
|
2011-01-29T00:00:00.000Z upfront mezzanine preferred mpreferred 1685.500085
|
||||||
|
2011-01-30T00:00:00.000Z spot automotive preferred apreferred 124.943293
|
||||||
|
2011-01-30T00:00:00.000Z spot entertainment preferred epreferred 108.415967
|
||||||
|
2011-01-30T00:00:00.000Z spot mezzanine preferred mpreferred 97.023907
|
||||||
|
2011-01-30T00:00:00.000Z spot premium preferred ppreferred 104.298490
|
||||||
|
2011-01-30T00:00:00.000Z spot travel preferred tpreferred 137.932693
|
||||||
|
2011-01-30T00:00:00.000Z total_market premium preferred ppreferred 805.930143
|
||||||
|
2011-01-30T00:00:00.000Z upfront premium preferred ppreferred 555.476028
|
||||||
|
2011-01-31T00:00:00.000Z spot business preferred bpreferred 103.492964
|
||||||
|
2011-01-31T00:00:00.000Z spot health preferred hpreferred 124.171944
|
||||||
|
2011-01-31T00:00:00.000Z spot news preferred npreferred 103.832040
|
||||||
|
2011-01-31T00:00:00.000Z spot technology preferred tpreferred 85.125795
|
||||||
|
2011-01-31T00:00:00.000Z total_market mezzanine preferred mpreferred 1184.920651
|
||||||
|
2011-01-31T00:00:00.000Z upfront mezzanine preferred mpreferred 1643.340851
|
||||||
|
2011-02-01T00:00:00.000Z spot automotive preferred apreferred 132.123776
|
||||||
|
2011-02-01T00:00:00.000Z spot entertainment preferred epreferred 103.652865
|
||||||
|
2011-02-01T00:00:00.000Z spot mezzanine preferred mpreferred 98.909356
|
||||||
|
2011-02-01T00:00:00.000Z spot premium preferred ppreferred 102.480377
|
||||||
|
2011-02-01T00:00:00.000Z spot travel preferred tpreferred 134.014606
|
||||||
|
2011-02-01T00:00:00.000Z total_market premium preferred ppreferred 1100.904846
|
||||||
|
2011-02-01T00:00:00.000Z upfront premium preferred ppreferred 913.561076
|
||||||
|
2011-02-02T00:00:00.000Z spot business preferred bpreferred 104.963233
|
||||||
|
2011-02-02T00:00:00.000Z spot health preferred hpreferred 102.281859
|
||||||
|
2011-02-02T00:00:00.000Z spot news preferred npreferred 105.578807
|
||||||
|
2011-02-02T00:00:00.000Z spot technology preferred tpreferred 96.706279
|
||||||
|
2011-02-02T00:00:00.000Z total_market mezzanine preferred mpreferred 1097.211164
|
||||||
|
2011-02-02T00:00:00.000Z upfront mezzanine preferred mpreferred 1218.561908
|
||||||
|
2011-02-03T00:00:00.000Z spot automotive preferred apreferred 85.770241
|
||||||
|
2011-02-03T00:00:00.000Z spot entertainment preferred epreferred 106.425780
|
||||||
|
2011-02-03T00:00:00.000Z spot mezzanine preferred mpreferred 100.559287
|
||||||
|
2011-02-03T00:00:00.000Z spot premium preferred ppreferred 102.349315
|
||||||
|
2011-02-03T00:00:00.000Z spot travel preferred tpreferred 134.140377
|
||||||
|
2011-02-03T00:00:00.000Z total_market premium preferred ppreferred 1283.166055
|
||||||
|
2011-02-03T00:00:00.000Z upfront premium preferred ppreferred 1113.114125
|
||||||
|
2011-02-04T00:00:00.000Z spot business preferred bpreferred 106.888769
|
||||||
|
2011-02-04T00:00:00.000Z spot health preferred hpreferred 99.837572
|
||||||
|
2011-02-04T00:00:00.000Z spot news preferred npreferred 106.050728
|
||||||
|
2011-02-04T00:00:00.000Z spot technology preferred tpreferred 93.973465
|
||||||
|
2011-02-04T00:00:00.000Z total_market mezzanine preferred mpreferred 1025.633340
|
||||||
|
2011-02-04T00:00:00.000Z upfront mezzanine preferred mpreferred 864.568891
|
||||||
|
2011-02-05T00:00:00.000Z spot automotive preferred apreferred 93.001571
|
||||||
|
2011-02-05T00:00:00.000Z spot entertainment preferred epreferred 117.030289
|
||||||
|
2011-02-05T00:00:00.000Z spot mezzanine preferred mpreferred 105.660538
|
||||||
|
2011-02-05T00:00:00.000Z spot premium preferred ppreferred 100.646747
|
||||||
|
2011-02-05T00:00:00.000Z spot travel preferred tpreferred 114.723682
|
||||||
|
2011-02-05T00:00:00.000Z total_market premium preferred ppreferred 1332.468373
|
||||||
|
2011-02-05T00:00:00.000Z upfront premium preferred ppreferred 1363.614929
|
||||||
|
2011-02-06T00:00:00.000Z spot business preferred bpreferred 110.897359
|
||||||
|
2011-02-06T00:00:00.000Z spot health preferred hpreferred 93.585758
|
||||||
|
2011-02-06T00:00:00.000Z spot news preferred npreferred 110.053071
|
||||||
|
2011-02-06T00:00:00.000Z spot technology preferred tpreferred 93.620739
|
||||||
|
2011-02-06T00:00:00.000Z total_market mezzanine preferred mpreferred 1011.205470
|
||||||
|
2011-02-06T00:00:00.000Z upfront mezzanine preferred mpreferred 787.125330
|
||||||
|
2011-02-07T00:00:00.000Z spot automotive preferred apreferred 130.194219
|
||||||
|
2011-02-07T00:00:00.000Z spot entertainment preferred epreferred 112.924874
|
||||||
|
2011-02-07T00:00:00.000Z spot mezzanine preferred mpreferred 97.434318
|
||||||
|
2011-02-07T00:00:00.000Z spot premium preferred ppreferred 102.705243
|
||||||
|
2011-02-07T00:00:00.000Z spot travel preferred tpreferred 115.246714
|
||||||
|
2011-02-07T00:00:00.000Z total_market premium preferred ppreferred 1057.079944
|
||||||
|
2011-02-07T00:00:00.000Z upfront premium preferred ppreferred 872.625669
|
||||||
|
2011-02-08T00:00:00.000Z spot business preferred bpreferred 93.190129
|
||||||
|
2011-02-08T00:00:00.000Z spot health preferred hpreferred 97.432302
|
||||||
|
2011-02-08T00:00:00.000Z spot news preferred npreferred 97.085047
|
||||||
|
2011-02-08T00:00:00.000Z spot technology preferred tpreferred 75.735586
|
||||||
|
2011-02-08T00:00:00.000Z total_market mezzanine preferred mpreferred 1064.972638
|
||||||
|
2011-02-08T00:00:00.000Z upfront mezzanine preferred mpreferred 1188.369265
|
||||||
|
2011-02-09T00:00:00.000Z spot automotive preferred apreferred 129.221792
|
||||||
|
2011-02-09T00:00:00.000Z spot entertainment preferred epreferred 111.729360
|
||||||
|
2011-02-09T00:00:00.000Z spot mezzanine preferred mpreferred 96.478571
|
||||||
|
2011-02-09T00:00:00.000Z spot premium preferred ppreferred 105.498315
|
||||||
|
2011-02-09T00:00:00.000Z spot travel preferred tpreferred 112.646238
|
||||||
|
2011-02-09T00:00:00.000Z total_market premium preferred ppreferred 1320.638308
|
||||||
|
2011-02-09T00:00:00.000Z upfront premium preferred ppreferred 1299.093262
|
||||||
|
2011-02-10T00:00:00.000Z spot business preferred bpreferred 118.062165
|
||||||
|
2011-02-10T00:00:00.000Z spot health preferred hpreferred 97.235999
|
||||||
|
2011-02-10T00:00:00.000Z spot news preferred npreferred 115.824976
|
||||||
|
2011-02-10T00:00:00.000Z spot technology preferred tpreferred 91.750911
|
||||||
|
2011-02-10T00:00:00.000Z total_market mezzanine preferred mpreferred 1070.165582
|
||||||
|
2011-02-10T00:00:00.000Z upfront mezzanine preferred mpreferred 1212.928303
|
||||||
|
2011-02-11T00:00:00.000Z spot automotive preferred apreferred 129.187009
|
||||||
|
2011-02-11T00:00:00.000Z spot entertainment preferred epreferred 114.960877
|
||||||
|
2011-02-11T00:00:00.000Z spot mezzanine preferred mpreferred 100.111873
|
||||||
|
2011-02-11T00:00:00.000Z spot premium preferred ppreferred 105.672256
|
||||||
|
2011-02-11T00:00:00.000Z spot travel preferred tpreferred 102.864842
|
||||||
|
2011-02-11T00:00:00.000Z total_market premium preferred ppreferred 1179.695901
|
||||||
|
2011-02-11T00:00:00.000Z upfront premium preferred ppreferred 1061.973330
|
||||||
|
2011-02-12T00:00:00.000Z spot business preferred bpreferred 115.758445
|
||||||
|
2011-02-12T00:00:00.000Z spot health preferred hpreferred 96.457082
|
||||||
|
2011-02-12T00:00:00.000Z spot news preferred npreferred 114.877503
|
||||||
|
2011-02-12T00:00:00.000Z spot technology preferred tpreferred 88.142774
|
||||||
|
2011-02-12T00:00:00.000Z total_market mezzanine preferred mpreferred 959.236186
|
||||||
|
2011-02-12T00:00:00.000Z upfront mezzanine preferred mpreferred 602.979544
|
||||||
|
2011-02-13T00:00:00.000Z spot automotive preferred apreferred 119.490316
|
||||||
|
2011-02-13T00:00:00.000Z spot entertainment preferred epreferred 119.907266
|
||||||
|
2011-02-13T00:00:00.000Z spot mezzanine preferred mpreferred 100.905238
|
||||||
|
2011-02-13T00:00:00.000Z spot premium preferred ppreferred 112.514409
|
||||||
|
2011-02-13T00:00:00.000Z spot travel preferred tpreferred 106.033416
|
||||||
|
2011-02-13T00:00:00.000Z total_market premium preferred ppreferred 1103.458199
|
||||||
|
2011-02-13T00:00:00.000Z upfront premium preferred ppreferred 862.931321
|
||||||
|
2011-02-14T00:00:00.000Z spot business preferred bpreferred 115.628202
|
||||||
|
2011-02-14T00:00:00.000Z spot health preferred hpreferred 103.008650
|
||||||
|
2011-02-14T00:00:00.000Z spot news preferred npreferred 117.110451
|
||||||
|
2011-02-14T00:00:00.000Z spot technology preferred tpreferred 73.717033
|
||||||
|
2011-02-14T00:00:00.000Z total_market mezzanine preferred mpreferred 1091.223197
|
||||||
|
2011-02-14T00:00:00.000Z upfront mezzanine preferred mpreferred 1133.135123
|
||||||
|
2011-02-15T00:00:00.000Z spot automotive preferred apreferred 123.485071
|
||||||
|
2011-02-15T00:00:00.000Z spot entertainment preferred epreferred 121.563912
|
||||||
|
2011-02-15T00:00:00.000Z spot mezzanine preferred mpreferred 105.269599
|
||||||
|
2011-02-15T00:00:00.000Z spot premium preferred ppreferred 121.411398
|
||||||
|
2011-02-15T00:00:00.000Z spot travel preferred tpreferred 108.428302
|
||||||
|
2011-02-15T00:00:00.000Z total_market premium preferred ppreferred 1183.240825
|
||||||
|
2011-02-15T00:00:00.000Z upfront premium preferred ppreferred 914.525048
|
||||||
|
2011-02-16T00:00:00.000Z spot business preferred bpreferred 116.432276
|
||||||
|
2011-02-16T00:00:00.000Z spot health preferred hpreferred 105.762627
|
||||||
|
2011-02-16T00:00:00.000Z spot news preferred npreferred 117.334381
|
||||||
|
2011-02-16T00:00:00.000Z spot technology preferred tpreferred 98.918664
|
||||||
|
2011-02-16T00:00:00.000Z total_market mezzanine preferred mpreferred 1289.097304
|
||||||
|
2011-02-16T00:00:00.000Z upfront mezzanine preferred mpreferred 1553.348548
|
||||||
|
2011-02-17T00:00:00.000Z spot automotive preferred apreferred 147.942017
|
||||||
|
2011-02-17T00:00:00.000Z spot entertainment preferred epreferred 126.982673
|
||||||
|
2011-02-17T00:00:00.000Z spot mezzanine preferred mpreferred 103.940963
|
||||||
|
2011-02-17T00:00:00.000Z spot premium preferred ppreferred 120.050545
|
||||||
|
2011-02-17T00:00:00.000Z spot travel preferred tpreferred 112.150745
|
||||||
|
2011-02-17T00:00:00.000Z total_market premium preferred ppreferred 1021.071173
|
||||||
|
2011-02-17T00:00:00.000Z upfront premium preferred ppreferred 645.177645
|
||||||
|
2011-02-18T00:00:00.000Z spot business preferred bpreferred 127.611947
|
||||||
|
2011-02-18T00:00:00.000Z spot health preferred hpreferred 100.849247
|
||||||
|
2011-02-18T00:00:00.000Z spot news preferred npreferred 124.513018
|
||||||
|
2011-02-18T00:00:00.000Z spot technology preferred tpreferred 92.174432
|
||||||
|
2011-02-18T00:00:00.000Z total_market mezzanine preferred mpreferred 1105.383465
|
||||||
|
2011-02-18T00:00:00.000Z upfront mezzanine preferred mpreferred 1120.088751
|
||||||
|
2011-02-19T00:00:00.000Z spot automotive preferred apreferred 163.351690
|
||||||
|
2011-02-19T00:00:00.000Z spot entertainment preferred epreferred 133.726878
|
||||||
|
2011-02-19T00:00:00.000Z spot mezzanine preferred mpreferred 102.919452
|
||||||
|
2011-02-19T00:00:00.000Z spot premium preferred ppreferred 125.052129
|
||||||
|
2011-02-19T00:00:00.000Z spot travel preferred tpreferred 121.733400
|
||||||
|
2011-02-19T00:00:00.000Z total_market premium preferred ppreferred 1600.723226
|
||||||
|
2011-02-19T00:00:00.000Z upfront premium preferred ppreferred 1598.179271
|
||||||
|
2011-02-20T00:00:00.000Z spot business preferred bpreferred 129.409606
|
||||||
|
2011-02-20T00:00:00.000Z spot health preferred hpreferred 109.790712
|
||||||
|
2011-02-20T00:00:00.000Z spot news preferred npreferred 122.082375
|
||||||
|
2011-02-20T00:00:00.000Z spot technology preferred tpreferred 105.985049
|
||||||
|
2011-02-20T00:00:00.000Z total_market mezzanine preferred mpreferred 1317.458323
|
||||||
|
2011-02-20T00:00:00.000Z upfront mezzanine preferred mpreferred 1192.563067
|
||||||
|
2011-02-21T00:00:00.000Z spot automotive preferred apreferred 155.632898
|
||||||
|
2011-02-21T00:00:00.000Z spot entertainment preferred epreferred 193.787574
|
||||||
|
2011-02-21T00:00:00.000Z spot mezzanine preferred mpreferred 154.627912
|
||||||
|
2011-02-21T00:00:00.000Z spot premium preferred ppreferred 138.092468
|
||||||
|
2011-02-21T00:00:00.000Z spot travel preferred tpreferred 119.739112
|
||||||
|
2011-02-21T00:00:00.000Z total_market premium preferred ppreferred 1488.737765
|
||||||
|
2011-02-21T00:00:00.000Z upfront premium preferred ppreferred 1298.415763
|
||||||
|
2011-02-22T00:00:00.000Z spot business preferred bpreferred 124.524992
|
||||||
|
2011-02-22T00:00:00.000Z spot health preferred hpreferred 108.370907
|
||||||
|
2011-02-22T00:00:00.000Z spot news preferred npreferred 122.510640
|
||||||
|
2011-02-22T00:00:00.000Z spot technology preferred tpreferred 86.333235
|
||||||
|
2011-02-22T00:00:00.000Z total_market mezzanine preferred mpreferred 1224.827108
|
||||||
|
2011-02-22T00:00:00.000Z upfront mezzanine preferred mpreferred 1345.964309
|
||||||
|
2011-02-23T00:00:00.000Z spot automotive preferred apreferred 165.273009
|
||||||
|
2011-02-23T00:00:00.000Z spot entertainment preferred epreferred 145.588115
|
||||||
|
2011-02-23T00:00:00.000Z spot mezzanine preferred mpreferred 113.141185
|
||||||
|
2011-02-23T00:00:00.000Z spot premium preferred ppreferred 124.305608
|
||||||
|
2011-02-23T00:00:00.000Z spot travel preferred tpreferred 123.961542
|
||||||
|
2011-02-23T00:00:00.000Z total_market premium preferred ppreferred 1414.619004
|
||||||
|
2011-02-23T00:00:00.000Z upfront premium preferred ppreferred 1287.766687
|
||||||
|
2011-02-24T00:00:00.000Z spot business preferred bpreferred 123.553981
|
||||||
|
2011-02-24T00:00:00.000Z spot health preferred hpreferred 104.951315
|
||||||
|
2011-02-24T00:00:00.000Z spot news preferred npreferred 118.862342
|
||||||
|
2011-02-24T00:00:00.000Z spot technology preferred tpreferred 91.962584
|
||||||
|
2011-02-24T00:00:00.000Z total_market mezzanine preferred mpreferred 1350.175381
|
||||||
|
2011-02-24T00:00:00.000Z upfront mezzanine preferred mpreferred 1631.584352
|
||||||
|
2011-02-25T00:00:00.000Z spot automotive preferred apreferred 172.335540
|
||||||
|
2011-02-25T00:00:00.000Z spot entertainment preferred epreferred 140.941317
|
||||||
|
2011-02-25T00:00:00.000Z spot mezzanine preferred mpreferred 99.460461
|
||||||
|
2011-02-25T00:00:00.000Z spot premium preferred ppreferred 115.932803
|
||||||
|
2011-02-25T00:00:00.000Z spot travel preferred tpreferred 120.124862
|
||||||
|
2011-02-25T00:00:00.000Z total_market premium preferred ppreferred 1073.967314
|
||||||
|
2011-02-25T00:00:00.000Z upfront premium preferred ppreferred 782.013486
|
||||||
|
2011-02-26T00:00:00.000Z spot business preferred bpreferred 103.643952
|
||||||
|
2011-02-26T00:00:00.000Z spot health preferred hpreferred 103.667031
|
||||||
|
2011-02-26T00:00:00.000Z spot news preferred npreferred 105.352891
|
||||||
|
2011-02-26T00:00:00.000Z spot technology preferred tpreferred 83.099365
|
||||||
|
2011-02-26T00:00:00.000Z total_market mezzanine preferred mpreferred 996.433708
|
||||||
|
2011-02-26T00:00:00.000Z upfront mezzanine preferred mpreferred 829.916235
|
||||||
|
2011-02-27T00:00:00.000Z spot automotive preferred apreferred 277.273533
|
||||||
|
2011-02-27T00:00:00.000Z spot entertainment preferred epreferred 136.394846
|
||||||
|
2011-02-27T00:00:00.000Z spot mezzanine preferred mpreferred 103.226967
|
||||||
|
2011-02-27T00:00:00.000Z spot premium preferred ppreferred 121.929932
|
||||||
|
2011-02-27T00:00:00.000Z spot travel preferred tpreferred 136.163414
|
||||||
|
2011-02-27T00:00:00.000Z total_market premium preferred ppreferred 1474.591017
|
||||||
|
2011-02-27T00:00:00.000Z upfront premium preferred ppreferred 1427.016724
|
||||||
|
2011-02-28T00:00:00.000Z spot business preferred bpreferred 97.218943
|
||||||
|
2011-02-28T00:00:00.000Z spot health preferred hpreferred 112.528286
|
||||||
|
2011-02-28T00:00:00.000Z spot news preferred npreferred 99.505465
|
||||||
|
2011-02-28T00:00:00.000Z spot technology preferred tpreferred 72.163651
|
||||||
|
2011-02-28T00:00:00.000Z total_market mezzanine preferred mpreferred 1159.278766
|
||||||
|
2011-02-28T00:00:00.000Z upfront mezzanine preferred mpreferred 1430.257348
|
||||||
|
2011-03-01T00:00:00.000Z spot automotive preferred apreferred 153.059937
|
||||||
|
2011-03-01T00:00:00.000Z spot entertainment preferred epreferred 143.424672
|
||||||
|
2011-03-01T00:00:00.000Z spot mezzanine preferred mpreferred 105.453024
|
||||||
|
2011-03-01T00:00:00.000Z spot premium preferred ppreferred 123.251814
|
||||||
|
2011-03-01T00:00:00.000Z spot travel preferred tpreferred 116.975408
|
||||||
|
2011-03-01T00:00:00.000Z total_market premium preferred ppreferred 1243.354010
|
||||||
|
2011-03-01T00:00:00.000Z upfront premium preferred ppreferred 1004.940887
|
||||||
|
2011-03-02T00:00:00.000Z spot business preferred bpreferred 98.432014
|
||||||
|
2011-03-02T00:00:00.000Z spot health preferred hpreferred 112.968782
|
||||||
|
2011-03-02T00:00:00.000Z spot news preferred npreferred 100.600391
|
||||||
|
2011-03-02T00:00:00.000Z spot technology preferred tpreferred 82.823988
|
||||||
|
2011-03-02T00:00:00.000Z total_market mezzanine preferred mpreferred 1051.808940
|
||||||
|
2011-03-02T00:00:00.000Z upfront mezzanine preferred mpreferred 740.183720
|
||||||
|
2011-03-03T00:00:00.000Z spot automotive preferred apreferred 127.994476
|
||||||
|
2011-03-03T00:00:00.000Z spot entertainment preferred epreferred 140.215411
|
||||||
|
2011-03-03T00:00:00.000Z spot mezzanine preferred mpreferred 108.784646
|
||||||
|
2011-03-03T00:00:00.000Z spot premium preferred ppreferred 115.393493
|
||||||
|
2011-03-03T00:00:00.000Z spot travel preferred tpreferred 114.188310
|
||||||
|
2011-03-03T00:00:00.000Z total_market premium preferred ppreferred 1010.370296
|
||||||
|
2011-03-03T00:00:00.000Z upfront premium preferred ppreferred 691.958920
|
||||||
|
2011-03-04T00:00:00.000Z spot business preferred bpreferred 93.634505
|
||||||
|
2011-03-04T00:00:00.000Z spot health preferred hpreferred 110.018472
|
||||||
|
2011-03-04T00:00:00.000Z spot news preferred npreferred 97.535226
|
||||||
|
2011-03-04T00:00:00.000Z spot technology preferred tpreferred 81.131208
|
||||||
|
2011-03-04T00:00:00.000Z total_market mezzanine preferred mpreferred 1326.829155
|
||||||
|
2011-03-04T00:00:00.000Z upfront mezzanine preferred mpreferred 1674.331703
|
||||||
|
2011-03-05T00:00:00.000Z spot automotive preferred apreferred 136.941770
|
||||||
|
2011-03-05T00:00:00.000Z spot entertainment preferred epreferred 145.393016
|
||||||
|
2011-03-05T00:00:00.000Z spot mezzanine preferred mpreferred 112.522435
|
||||||
|
2011-03-05T00:00:00.000Z spot premium preferred ppreferred 114.691277
|
||||||
|
2011-03-05T00:00:00.000Z spot travel preferred tpreferred 117.904527
|
||||||
|
2011-03-05T00:00:00.000Z total_market premium preferred ppreferred 994.731237
|
||||||
|
2011-03-05T00:00:00.000Z upfront premium preferred ppreferred 755.899363
|
||||||
|
2011-03-06T00:00:00.000Z spot business preferred bpreferred 99.508679
|
||||||
|
2011-03-06T00:00:00.000Z spot health preferred hpreferred 113.069662
|
||||||
|
2011-03-06T00:00:00.000Z spot news preferred npreferred 102.536839
|
||||||
|
2011-03-06T00:00:00.000Z spot technology preferred tpreferred 81.612269
|
||||||
|
2011-03-06T00:00:00.000Z total_market mezzanine preferred mpreferred 1081.650406
|
||||||
|
2011-03-06T00:00:00.000Z upfront mezzanine preferred mpreferred 771.348460
|
||||||
|
2011-03-07T00:00:00.000Z spot automotive preferred apreferred 111.909348
|
||||||
|
2011-03-07T00:00:00.000Z spot entertainment preferred epreferred 150.452695
|
||||||
|
2011-03-07T00:00:00.000Z spot mezzanine preferred mpreferred 111.106693
|
||||||
|
2011-03-07T00:00:00.000Z spot premium preferred ppreferred 121.582721
|
||||||
|
2011-03-07T00:00:00.000Z spot travel preferred tpreferred 106.884238
|
||||||
|
2011-03-07T00:00:00.000Z total_market premium preferred ppreferred 1152.547767
|
||||||
|
2011-03-07T00:00:00.000Z upfront premium preferred ppreferred 906.373797
|
||||||
|
2011-03-08T00:00:00.000Z spot business preferred bpreferred 98.972716
|
||||||
|
2011-03-08T00:00:00.000Z spot health preferred hpreferred 119.777621
|
||||||
|
2011-03-08T00:00:00.000Z spot news preferred npreferred 101.652185
|
||||||
|
2011-03-08T00:00:00.000Z spot technology preferred tpreferred 70.866726
|
||||||
|
2011-03-08T00:00:00.000Z total_market mezzanine preferred mpreferred 1117.953961
|
||||||
|
2011-03-08T00:00:00.000Z upfront mezzanine preferred mpreferred 988.893782
|
||||||
|
2011-03-09T00:00:00.000Z spot automotive preferred apreferred 139.260950
|
||||||
|
2011-03-09T00:00:00.000Z spot entertainment preferred epreferred 138.466933
|
||||||
|
2011-03-09T00:00:00.000Z spot mezzanine preferred mpreferred 105.613469
|
||||||
|
2011-03-09T00:00:00.000Z spot premium preferred ppreferred 121.220772
|
||||||
|
2011-03-09T00:00:00.000Z spot travel preferred tpreferred 107.998334
|
||||||
|
2011-03-09T00:00:00.000Z total_market premium preferred ppreferred 1121.385333
|
||||||
|
2011-03-09T00:00:00.000Z upfront premium preferred ppreferred 875.683406
|
||||||
|
2011-03-10T00:00:00.000Z spot business preferred bpreferred 105.214709
|
||||||
|
2011-03-10T00:00:00.000Z spot health preferred hpreferred 114.717338
|
||||||
|
2011-03-10T00:00:00.000Z spot news preferred npreferred 107.127962
|
||||||
|
2011-03-10T00:00:00.000Z spot technology preferred tpreferred 79.793836
|
||||||
|
2011-03-10T00:00:00.000Z total_market mezzanine preferred mpreferred 1244.849915
|
||||||
|
2011-03-10T00:00:00.000Z upfront mezzanine preferred mpreferred 1070.836247
|
||||||
|
2011-03-11T00:00:00.000Z spot automotive preferred apreferred 135.820968
|
||||||
|
2011-03-11T00:00:00.000Z spot entertainment preferred epreferred 135.038992
|
||||||
|
2011-03-11T00:00:00.000Z spot mezzanine preferred mpreferred 120.497687
|
||||||
|
2011-03-11T00:00:00.000Z spot premium preferred ppreferred 118.298350
|
||||||
|
2011-03-11T00:00:00.000Z spot travel preferred tpreferred 108.186877
|
||||||
|
2011-03-11T00:00:00.000Z total_market premium preferred ppreferred 998.650727
|
||||||
|
2011-03-11T00:00:00.000Z upfront premium preferred ppreferred 755.646538
|
||||||
|
2011-03-12T00:00:00.000Z spot business preferred bpreferred 113.493460
|
||||||
|
2011-03-12T00:00:00.000Z spot health preferred hpreferred 112.554597
|
||||||
|
2011-03-12T00:00:00.000Z spot news preferred npreferred 115.572940
|
||||||
|
2011-03-12T00:00:00.000Z spot technology preferred tpreferred 74.394926
|
||||||
|
2011-03-12T00:00:00.000Z total_market mezzanine preferred mpreferred 1088.807596
|
||||||
|
2011-03-12T00:00:00.000Z upfront mezzanine preferred mpreferred 771.100508
|
||||||
|
2011-03-13T00:00:00.000Z spot automotive preferred apreferred 149.637715
|
||||||
|
2011-03-13T00:00:00.000Z spot entertainment preferred epreferred 120.113921
|
||||||
|
2011-03-13T00:00:00.000Z spot mezzanine preferred mpreferred 103.227522
|
||||||
|
2011-03-13T00:00:00.000Z spot premium preferred ppreferred 120.620862
|
||||||
|
2011-03-13T00:00:00.000Z spot travel preferred tpreferred 108.579283
|
||||||
|
2011-03-13T00:00:00.000Z total_market premium preferred ppreferred 1129.723252
|
||||||
|
2011-03-13T00:00:00.000Z upfront premium preferred ppreferred 884.837267
|
||||||
|
2011-03-14T00:00:00.000Z spot business preferred bpreferred 109.461442
|
||||||
|
2011-03-14T00:00:00.000Z spot health preferred hpreferred 120.487244
|
||||||
|
2011-03-14T00:00:00.000Z spot news preferred npreferred 111.688901
|
||||||
|
2011-03-14T00:00:00.000Z spot technology preferred tpreferred 59.021022
|
||||||
|
2011-03-14T00:00:00.000Z total_market mezzanine preferred mpreferred 1075.243024
|
||||||
|
2011-03-14T00:00:00.000Z upfront mezzanine preferred mpreferred 918.722840
|
||||||
|
2011-03-15T00:00:00.000Z spot automotive preferred apreferred 145.963558
|
||||||
|
2011-03-15T00:00:00.000Z spot entertainment preferred epreferred 121.386341
|
||||||
|
2011-03-15T00:00:00.000Z spot mezzanine preferred mpreferred 103.583295
|
||||||
|
2011-03-15T00:00:00.000Z spot premium preferred ppreferred 122.038585
|
||||||
|
2011-03-15T00:00:00.000Z spot travel preferred tpreferred 103.134338
|
||||||
|
2011-03-15T00:00:00.000Z total_market premium preferred ppreferred 1099.197263
|
||||||
|
2011-03-15T00:00:00.000Z upfront premium preferred ppreferred 850.995007
|
||||||
|
2011-03-16T00:00:00.000Z spot business preferred bpreferred 110.565004
|
||||||
|
2011-03-16T00:00:00.000Z spot health preferred hpreferred 115.750963
|
||||||
|
2011-03-16T00:00:00.000Z spot news preferred npreferred 112.577264
|
||||||
|
2011-03-16T00:00:00.000Z spot technology preferred tpreferred 69.329723
|
||||||
|
2011-03-16T00:00:00.000Z total_market mezzanine preferred mpreferred 981.577244
|
||||||
|
2011-03-16T00:00:00.000Z upfront mezzanine preferred mpreferred 767.973326
|
||||||
|
2011-03-17T00:00:00.000Z spot automotive preferred apreferred 148.905410
|
||||||
|
2011-03-17T00:00:00.000Z spot entertainment preferred epreferred 109.666402
|
||||||
|
2011-03-17T00:00:00.000Z spot mezzanine preferred mpreferred 105.996125
|
||||||
|
2011-03-17T00:00:00.000Z spot premium preferred ppreferred 122.740143
|
||||||
|
2011-03-17T00:00:00.000Z spot travel preferred tpreferred 110.797348
|
||||||
|
2011-03-17T00:00:00.000Z total_market premium preferred ppreferred 1154.415689
|
||||||
|
2011-03-17T00:00:00.000Z upfront premium preferred ppreferred 906.101957
|
||||||
|
2011-03-18T00:00:00.000Z spot business preferred bpreferred 110.037579
|
||||||
|
2011-03-18T00:00:00.000Z spot health preferred hpreferred 113.490115
|
||||||
|
2011-03-18T00:00:00.000Z spot news preferred npreferred 113.238556
|
||||||
|
2011-03-18T00:00:00.000Z spot technology preferred tpreferred 68.573162
|
||||||
|
2011-03-18T00:00:00.000Z total_market mezzanine preferred mpreferred 1311.178603
|
||||||
|
2011-03-18T00:00:00.000Z upfront mezzanine preferred mpreferred 1403.830217
|
||||||
|
2011-03-19T00:00:00.000Z spot automotive preferred apreferred 177.514270
|
||||||
|
2011-03-19T00:00:00.000Z spot entertainment preferred epreferred 134.147573
|
||||||
|
2011-03-19T00:00:00.000Z spot mezzanine preferred mpreferred 112.999693
|
||||||
|
2011-03-19T00:00:00.000Z spot premium preferred ppreferred 120.638001
|
||||||
|
2011-03-19T00:00:00.000Z spot travel preferred tpreferred 115.384807
|
||||||
|
2011-03-19T00:00:00.000Z total_market premium preferred ppreferred 1102.698977
|
||||||
|
2011-03-19T00:00:00.000Z upfront premium preferred ppreferred 856.490089
|
||||||
|
2011-03-20T00:00:00.000Z spot business preferred bpreferred 123.507497
|
||||||
|
2011-03-20T00:00:00.000Z spot health preferred hpreferred 117.851058
|
||||||
|
2011-03-20T00:00:00.000Z spot news preferred npreferred 125.496367
|
||||||
|
2011-03-20T00:00:00.000Z spot technology preferred tpreferred 85.013155
|
||||||
|
2011-03-20T00:00:00.000Z total_market mezzanine preferred mpreferred 1285.090048
|
||||||
|
2011-03-20T00:00:00.000Z upfront mezzanine preferred mpreferred 1032.257527
|
||||||
|
2011-03-21T00:00:00.000Z spot automotive preferred apreferred 182.035296
|
||||||
|
2011-03-21T00:00:00.000Z spot entertainment preferred epreferred 157.153730
|
||||||
|
2011-03-21T00:00:00.000Z spot mezzanine preferred mpreferred 128.149976
|
||||||
|
2011-03-21T00:00:00.000Z spot premium preferred ppreferred 131.807919
|
||||||
|
2011-03-21T00:00:00.000Z spot travel preferred tpreferred 123.653645
|
||||||
|
2011-03-21T00:00:00.000Z total_market premium preferred ppreferred 1178.830164
|
||||||
|
2011-03-21T00:00:00.000Z upfront premium preferred ppreferred 907.021565
|
||||||
|
2011-03-22T00:00:00.000Z spot business preferred bpreferred 121.270611
|
||||||
|
2011-03-22T00:00:00.000Z spot health preferred hpreferred 124.400780
|
||||||
|
2011-03-22T00:00:00.000Z spot news preferred npreferred 124.970533
|
||||||
|
2011-03-22T00:00:00.000Z spot technology preferred tpreferred 79.948248
|
||||||
|
2011-03-22T00:00:00.000Z total_market mezzanine preferred mpreferred 1301.778098
|
||||||
|
2011-03-22T00:00:00.000Z upfront mezzanine preferred mpreferred 1110.788895
|
||||||
|
2011-03-23T00:00:00.000Z spot automotive preferred apreferred 154.019632
|
||||||
|
2011-03-23T00:00:00.000Z spot entertainment preferred epreferred 158.592715
|
||||||
|
2011-03-23T00:00:00.000Z spot mezzanine preferred mpreferred 126.672392
|
||||||
|
2011-03-23T00:00:00.000Z spot premium preferred ppreferred 131.786598
|
||||||
|
2011-03-23T00:00:00.000Z spot travel preferred tpreferred 123.754240
|
||||||
|
2011-03-23T00:00:00.000Z total_market premium preferred ppreferred 1156.601892
|
||||||
|
2011-03-23T00:00:00.000Z upfront premium preferred ppreferred 884.801502
|
||||||
|
2011-03-24T00:00:00.000Z spot business preferred bpreferred 123.895027
|
||||||
|
2011-03-24T00:00:00.000Z spot health preferred hpreferred 119.777998
|
||||||
|
2011-03-24T00:00:00.000Z spot news preferred npreferred 125.044877
|
||||||
|
2011-03-24T00:00:00.000Z spot technology preferred tpreferred 88.521042
|
||||||
|
2011-03-24T00:00:00.000Z total_market mezzanine preferred mpreferred 1429.580257
|
||||||
|
2011-03-24T00:00:00.000Z upfront mezzanine preferred mpreferred 1684.268799
|
||||||
|
2011-03-25T00:00:00.000Z spot automotive preferred apreferred 140.577121
|
||||||
|
2011-03-25T00:00:00.000Z spot entertainment preferred epreferred 157.432108
|
||||||
|
2011-03-25T00:00:00.000Z spot mezzanine preferred mpreferred 114.706960
|
||||||
|
2011-03-25T00:00:00.000Z spot premium preferred ppreferred 128.943094
|
||||||
|
2011-03-25T00:00:00.000Z spot travel preferred tpreferred 136.136598
|
||||||
|
2011-03-25T00:00:00.000Z total_market premium preferred ppreferred 1256.499779
|
||||||
|
2011-03-25T00:00:00.000Z upfront premium preferred ppreferred 996.564152
|
||||||
|
2011-03-26T00:00:00.000Z spot business preferred bpreferred 130.824022
|
||||||
|
2011-03-26T00:00:00.000Z spot health preferred hpreferred 118.463523
|
||||||
|
2011-03-26T00:00:00.000Z spot news preferred npreferred 142.972964
|
||||||
|
2011-03-26T00:00:00.000Z spot technology preferred tpreferred 101.686196
|
||||||
|
2011-03-26T00:00:00.000Z total_market mezzanine preferred mpreferred 1217.877395
|
||||||
|
2011-03-26T00:00:00.000Z upfront mezzanine preferred mpreferred 1061.678577
|
||||||
|
2011-03-27T00:00:00.000Z spot automotive preferred apreferred 144.056669
|
||||||
|
2011-03-27T00:00:00.000Z spot entertainment preferred epreferred 163.161361
|
Loading…
Reference in New Issue