adding QueryGranularity to segment metadata and optionally expose same from segmentMetadata query (#2873)

This commit is contained in:
Himanshu 2016-05-03 13:31:10 -05:00 committed by Fangjin Yang
parent 40e595c7a0
commit 8e2742b7e8
16 changed files with 258 additions and 25 deletions

View File

@ -49,6 +49,9 @@ The format of the result is:
"aggregators" : {
"metric1" : { "type" : "longSum", "name" : "metric1", "fieldName" : "metric1" }
},
"queryGranularity" : {
"type": "none"
},
"size" : 300000,
"numRows" : 5000000
} ]
@ -104,7 +107,7 @@ This is a list of properties that determines the amount of information returned
By default, the "cardinality", "size", "interval", and "minmax" types will be used. If a property is not needed, omitting it from this list will result in a more efficient query.
There are five types of column analyses:
Types of column analyses are described below:
#### cardinality
@ -123,6 +126,10 @@ dimension columns.
* `intervals` in the result will contain the list of intervals associated with the queried segments.
#### queryGranularity
* `queryGranularity` in the result will contain query granularity of data stored in segments. this can be null if query granularity of segments was unknown or unmergeable (if merging is enabled).
#### aggregators
* `aggregators` in the result will contain the list of aggregators usable for querying metric columns. This may be

View File

@ -62,7 +62,8 @@
},
"size": 34881,
"numRows": 1014,
"aggregators": null
"aggregators": null,
"queryGranularity": null
}
]
},

View File

@ -607,7 +607,8 @@
},
"size": 747056474,
"numRows": 3702583,
"aggregators": null
"aggregators": null,
"queryGranularity": null
},
{
"id": "twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9",
@ -625,7 +626,8 @@
},
"size": 755796690,
"numRows": 3743002,
"aggregators": null
"aggregators": null,
"queryGranularity": null
},
{
"id": "twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9",
@ -643,7 +645,8 @@
},
"size": 706893542,
"numRows":3502959,
"aggregators": null
"aggregators": null,
"queryGranularity": null
}
]
},

View File

@ -1066,7 +1066,8 @@
},
"size": 902457341,
"numRows": 4462111,
"aggregators": null
"aggregators": null,
"queryGranularity": null
}
]
}

View File

@ -26,7 +26,9 @@ import org.joda.time.DateTime;
import org.joda.time.Period;
import org.joda.time.ReadableDuration;
import java.util.List;
import java.util.Map;
import java.util.Objects;
public abstract class QueryGranularity
{
@ -108,4 +110,22 @@ public abstract class QueryGranularity
}
throw new IAE("Cannot convert [%s] to QueryGranularity", o.getClass());
}
//simple merge strategy on query granularity that checks if all are equal or else
//returns null. this can be improved in future but is good enough for most use-cases.
public static QueryGranularity mergeQueryGranularities(List<QueryGranularity> toMerge)
{
if (toMerge == null || toMerge.size() == 0) {
return null;
}
QueryGranularity result = toMerge.get(0);
for (int i = 1; i < toMerge.size(); i++) {
if (!Objects.equals(result, toMerge.get(i))) {
return null;
}
}
return result;
}
}

View File

@ -37,6 +37,7 @@ import com.metamx.common.guava.nary.BinaryFn;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.common.guava.CombiningSequence;
import io.druid.common.utils.JodaUtils;
import io.druid.granularity.QueryGranularity;
import io.druid.query.CacheStrategy;
import io.druid.query.DruidMetrics;
import io.druid.query.Query;
@ -329,6 +330,13 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
}
}
final QueryGranularity queryGranularity = QueryGranularity.mergeQueryGranularities(
Lists.newArrayList(
arg1.getQueryGranularity(),
arg2.getQueryGranularity()
)
);
final String mergedId;
if (arg1.getId() != null && arg2.getId() != null && arg1.getId().equals(arg2.getId())) {
@ -343,7 +351,8 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
columns,
arg1.getSize() + arg2.getSize(),
arg1.getNumRows() + arg2.getNumRows(),
aggregators.isEmpty() ? null : aggregators
aggregators.isEmpty() ? null : aggregators,
queryGranularity
);
}
@ -356,7 +365,8 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
analysis.getColumns(),
analysis.getSize(),
analysis.getNumRows(),
analysis.getAggregators()
analysis.getAggregators(),
analysis.getQueryGranularity()
);
}
}

View File

@ -29,6 +29,7 @@ import com.google.inject.Inject;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import com.metamx.common.logger.Logger;
import io.druid.granularity.QueryGranularity;
import io.druid.query.AbstractPrioritizedCallable;
import io.druid.query.BaseQuery;
import io.druid.query.ConcatQueryRunner;
@ -111,8 +112,9 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
List<Interval> retIntervals = query.analyzingInterval() ? Arrays.asList(segment.getDataInterval()) : null;
final Map<String, AggregatorFactory> aggregators;
Metadata metadata = null;
if (query.hasAggregators()) {
final Metadata metadata = segment.asStorageAdapter().getMetadata();
metadata = segment.asStorageAdapter().getMetadata();
if (metadata != null && metadata.getAggregators() != null) {
aggregators = Maps.newHashMap();
for (AggregatorFactory aggregator : metadata.getAggregators()) {
@ -125,6 +127,16 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
aggregators = null;
}
final QueryGranularity queryGranularity;
if (query.hasQueryGranularity()) {
if (metadata == null) {
metadata = segment.asStorageAdapter().getMetadata();
}
queryGranularity = metadata.getQueryGranularity();
} else {
queryGranularity = null;
}
return Sequences.simple(
Arrays.asList(
new SegmentAnalysis(
@ -133,7 +145,8 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
columns,
totalSize,
numRows,
aggregators
aggregators,
queryGranularity
)
)
);

View File

@ -21,6 +21,7 @@ package io.druid.query.metadata.metadata;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.granularity.QueryGranularity;
import io.druid.query.aggregation.AggregatorFactory;
import org.joda.time.Interval;
@ -36,6 +37,7 @@ public class SegmentAnalysis implements Comparable<SegmentAnalysis>
private final long size;
private final long numRows;
private final Map<String, AggregatorFactory> aggregators;
private final QueryGranularity queryGranularity;
@JsonCreator
public SegmentAnalysis(
@ -44,7 +46,8 @@ public class SegmentAnalysis implements Comparable<SegmentAnalysis>
@JsonProperty("columns") Map<String, ColumnAnalysis> columns,
@JsonProperty("size") long size,
@JsonProperty("numRows") long numRows,
@JsonProperty("aggregators") Map<String, AggregatorFactory> aggregators
@JsonProperty("aggregators") Map<String, AggregatorFactory> aggregators,
@JsonProperty("queryGranularity") QueryGranularity queryGranularity
)
{
this.id = id;
@ -53,6 +56,7 @@ public class SegmentAnalysis implements Comparable<SegmentAnalysis>
this.size = size;
this.numRows = numRows;
this.aggregators = aggregators;
this.queryGranularity = queryGranularity;
}
@JsonProperty
@ -85,6 +89,12 @@ public class SegmentAnalysis implements Comparable<SegmentAnalysis>
return numRows;
}
@JsonProperty
public QueryGranularity getQueryGranularity()
{
return queryGranularity;
}
@JsonProperty
public Map<String, AggregatorFactory> getAggregators()
{
@ -101,6 +111,7 @@ public class SegmentAnalysis implements Comparable<SegmentAnalysis>
", size=" + size +
", numRows=" + numRows +
", aggregators=" + aggregators +
", queryGranularity=" + queryGranularity +
'}';
}
@ -122,7 +133,8 @@ public class SegmentAnalysis implements Comparable<SegmentAnalysis>
Objects.equals(id, that.id) &&
Objects.equals(interval, that.interval) &&
Objects.equals(columns, that.columns) &&
Objects.equals(aggregators, that.aggregators);
Objects.equals(aggregators, that.aggregators) &&
Objects.equals(queryGranularity, that.queryGranularity);
}
/**
@ -132,7 +144,7 @@ public class SegmentAnalysis implements Comparable<SegmentAnalysis>
@Override
public int hashCode()
{
return Objects.hash(id, interval, columns, size, numRows, aggregators);
return Objects.hash(id, interval, columns, size, numRows, aggregators, queryGranularity);
}
@Override

View File

@ -54,7 +54,8 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
SIZE,
INTERVAL,
AGGREGATORS,
MINMAX;
MINMAX,
QUERYGRANULARITY;
@JsonValue
@Override
@ -179,6 +180,11 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
return analysisTypes.contains(AnalysisType.AGGREGATORS);
}
public boolean hasQueryGranularity()
{
return analysisTypes.contains(AnalysisType.QUERYGRANULARITY);
}
public boolean hasMinMax()
{
return analysisTypes.contains(AnalysisType.MINMAX);

View File

@ -20,6 +20,7 @@
package io.druid.segment;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.granularity.QueryGranularity;
import io.druid.query.aggregation.AggregatorFactory;
import java.util.ArrayList;
@ -41,6 +42,9 @@ public class Metadata
@JsonProperty
private AggregatorFactory[] aggregators;
@JsonProperty
private QueryGranularity queryGranularity;
public Metadata()
{
container = new ConcurrentHashMap<>();
@ -57,6 +61,17 @@ public class Metadata
return this;
}
public QueryGranularity getQueryGranularity()
{
return queryGranularity;
}
public Metadata setQueryGranularity(QueryGranularity queryGranularity)
{
this.queryGranularity = queryGranularity;
return this;
}
public Metadata putAll(Map<String, Object> other)
{
if (other != null) {
@ -96,17 +111,24 @@ public class Metadata
? new ArrayList<AggregatorFactory[]>()
: null;
List<QueryGranularity> gransToMerge = new ArrayList<>();
for (Metadata metadata : toBeMerged) {
if (metadata != null) {
foundSomeMetadata = true;
if (aggregatorsToMerge != null) {
aggregatorsToMerge.add(metadata.getAggregators());
}
if (gransToMerge != null) {
gransToMerge.add(metadata.getQueryGranularity());
}
mergedContainer.putAll(metadata.container);
} else {
//if metadata and hence aggregators for some segment being merged are unknown then
//final merged segment should not have aggregators in the metadata
//if metadata and hence aggregators and queryGranularity for some segment being merged are unknown then
//final merged segment should not have same in metadata
aggregatorsToMerge = null;
gransToMerge = null;
}
}
@ -120,6 +142,11 @@ public class Metadata
} else {
result.setAggregators(overrideMergedAggregators);
}
if (gransToMerge != null) {
result.setQueryGranularity(QueryGranularity.mergeQueryGranularities(gransToMerge));
}
result.container.putAll(mergedContainer);
return result;
@ -141,7 +168,12 @@ public class Metadata
return false;
}
// Probably incorrect - comparing Object[] arrays with Arrays.equals
return Arrays.equals(aggregators, metadata.aggregators);
if (!Arrays.equals(aggregators, metadata.aggregators)) {
return false;
}
return !(queryGranularity != null
? !queryGranularity.equals(metadata.queryGranularity)
: metadata.queryGranularity != null);
}
@ -150,6 +182,7 @@ public class Metadata
{
int result = container.hashCode();
result = 31 * result + (aggregators != null ? Arrays.hashCode(aggregators) : 0);
result = 31 * result + (queryGranularity != null ? queryGranularity.hashCode() : 0);
return result;
}
@ -157,8 +190,10 @@ public class Metadata
public String toString()
{
return "Metadata{" +
"container=" + container +
", aggregators=" + Arrays.toString(aggregators) +
", queryGranularity=" + queryGranularity +
'}';
}
}

View File

@ -398,7 +398,9 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
this.reportParseExceptions = reportParseExceptions;
this.sortFacts = sortFacts;
this.metadata = new Metadata().setAggregators(getCombiningAggregators(metrics));
this.metadata = new Metadata()
.setAggregators(getCombiningAggregators(metrics))
.setQueryGranularity(this.gran);
this.aggs = initAggs(metrics, rowSupplier, deserializeComplexMetrics);
this.columnCapabilities = Maps.newHashMap();

View File

@ -20,10 +20,19 @@
package io.druid.granularity;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import io.druid.jackson.DefaultObjectMapper;
import org.joda.time.*;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.Days;
import org.joda.time.Hours;
import org.joda.time.Minutes;
import org.joda.time.Months;
import org.joda.time.Period;
import org.joda.time.Weeks;
import org.joda.time.Years;
import org.junit.Assert;
import org.junit.Test;
@ -698,6 +707,32 @@ public class QueryGranularityTest
Assert.assertEquals(new DurationGranularity(15 * 60 * 1000, null), gran);
}
@Test
public void testMerge()
{
Assert.assertNull(QueryGranularity.mergeQueryGranularities(null));
Assert.assertNull(QueryGranularity.mergeQueryGranularities(ImmutableList.<QueryGranularity>of()));
Assert.assertNull(QueryGranularity.mergeQueryGranularities(Lists.newArrayList(null, QueryGranularity.DAY)));
Assert.assertNull(QueryGranularity.mergeQueryGranularities(Lists.newArrayList(QueryGranularity.DAY, null)));
Assert.assertNull(
QueryGranularity.mergeQueryGranularities(
Lists.newArrayList(
QueryGranularity.DAY,
null,
QueryGranularity.DAY
)
)
);
Assert.assertNull(
QueryGranularity.mergeQueryGranularities(ImmutableList.of(QueryGranularity.ALL, QueryGranularity.DAY))
);
Assert.assertEquals(
QueryGranularity.ALL,
QueryGranularity.mergeQueryGranularities(ImmutableList.of(QueryGranularity.ALL, QueryGranularity.ALL))
);
}
private void assertSame(List<DateTime> expected, Iterable<Long> actual)
{
Assert.assertEquals(expected.size(), Iterables.size(actual));

View File

@ -85,6 +85,7 @@ public class SegmentMetadataQueryQueryToolChestTest
)
), 71982,
100,
null,
null
);
@ -113,7 +114,8 @@ public class SegmentMetadataQueryQueryToolChestTest
ImmutableMap.of(
"foo", new LongSumAggregatorFactory("foo", "foo"),
"baz", new DoubleSumAggregatorFactory("baz", "baz")
)
),
null
);
final SegmentAnalysis analysis2 = new SegmentAnalysis(
"id",
@ -124,7 +126,8 @@ public class SegmentMetadataQueryQueryToolChestTest
ImmutableMap.of(
"foo", new LongSumAggregatorFactory("foo", "foo"),
"bar", new DoubleSumAggregatorFactory("bar", "bar")
)
),
null
);
Assert.assertEquals(
@ -154,6 +157,7 @@ public class SegmentMetadataQueryQueryToolChestTest
Maps.<String, ColumnAnalysis>newHashMap(),
0,
0,
null,
null
);
final SegmentAnalysis analysis2 = new SegmentAnalysis(
@ -165,7 +169,8 @@ public class SegmentMetadataQueryQueryToolChestTest
ImmutableMap.of(
"foo", new LongSumAggregatorFactory("foo", "foo"),
"bar", new DoubleSumAggregatorFactory("bar", "bar")
)
),
null
);
Assert.assertNull(mergeStrict(analysis1, analysis2).getAggregators());
@ -187,6 +192,7 @@ public class SegmentMetadataQueryQueryToolChestTest
Maps.<String, ColumnAnalysis>newHashMap(),
0,
0,
null,
null
);
final SegmentAnalysis analysis2 = new SegmentAnalysis(
@ -195,6 +201,7 @@ public class SegmentMetadataQueryQueryToolChestTest
Maps.<String, ColumnAnalysis>newHashMap(),
0,
0,
null,
null
);
@ -214,7 +221,8 @@ public class SegmentMetadataQueryQueryToolChestTest
ImmutableMap.of(
"foo", new LongSumAggregatorFactory("foo", "foo"),
"bar", new DoubleSumAggregatorFactory("bar", "bar")
)
),
null
);
final SegmentAnalysis analysis2 = new SegmentAnalysis(
"id",
@ -226,7 +234,8 @@ public class SegmentMetadataQueryQueryToolChestTest
"foo", new LongSumAggregatorFactory("foo", "foo"),
"bar", new DoubleMaxAggregatorFactory("bar", "bar"),
"baz", new LongMaxAggregatorFactory("baz", "baz")
)
),
null
);
final Map<String, AggregatorFactory> expectedLenient = Maps.newHashMap();

View File

@ -28,6 +28,7 @@ import com.google.common.collect.Maps;
import com.google.common.util.concurrent.MoreExecutors;
import com.metamx.common.guava.Sequences;
import io.druid.common.utils.JodaUtils;
import io.druid.granularity.QueryGranularity;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.BySegmentResultValue;
import io.druid.query.BySegmentResultValueClass;
@ -179,6 +180,7 @@ public class SegmentMetadataQueryTest
)
), mmap1 ? 71982 : 72755,
1209,
null,
null
);
expectedSegmentAnalysis2 = new SegmentAnalysis(
@ -220,6 +222,7 @@ public class SegmentMetadataQueryTest
// null_column will be included only for incremental index, which makes a little bigger result than expected
), mmap2 ? 71982 : 72755,
1209,
null,
null
);
}
@ -266,6 +269,7 @@ public class SegmentMetadataQueryTest
),
0,
expectedSegmentAnalysis1.getNumRows() + expectedSegmentAnalysis2.getNumRows(),
null,
null
);
@ -332,6 +336,7 @@ public class SegmentMetadataQueryTest
),
0,
expectedSegmentAnalysis1.getNumRows() + expectedSegmentAnalysis2.getNumRows(),
null,
null
);
@ -447,6 +452,7 @@ public class SegmentMetadataQueryTest
),
expectedSegmentAnalysis1.getSize() + expectedSegmentAnalysis2.getSize(),
expectedSegmentAnalysis1.getNumRows() + expectedSegmentAnalysis2.getNumRows(),
null,
null
);
@ -496,6 +502,7 @@ public class SegmentMetadataQueryTest
),
0,
expectedSegmentAnalysis1.getNumRows() + expectedSegmentAnalysis2.getNumRows(),
null,
null
);
@ -556,7 +563,8 @@ public class SegmentMetadataQueryTest
),
0,
expectedSegmentAnalysis1.getNumRows() + expectedSegmentAnalysis2.getNumRows(),
expectedAggregators
expectedAggregators,
null
);
QueryToolChest toolChest = FACTORY.getToolchest();
@ -592,6 +600,64 @@ public class SegmentMetadataQueryTest
exec.shutdownNow();
}
@Test
public void testSegmentMetadataQueryWithQueryGranularityMerge()
{
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
differentIds ? "merged" : "testSegment",
null,
ImmutableMap.of(
"placement",
new ColumnAnalysis(
ValueType.STRING.toString(),
false,
0,
0,
null,
null,
null
)
),
0,
expectedSegmentAnalysis1.getNumRows() + expectedSegmentAnalysis2.getNumRows(),
null,
QueryGranularity.NONE
);
QueryToolChest toolChest = FACTORY.getToolchest();
ExecutorService exec = Executors.newCachedThreadPool();
QueryRunner myRunner = new FinalizeResultsQueryRunner<>(
toolChest.mergeResults(
FACTORY.mergeRunners(
MoreExecutors.sameThreadExecutor(),
Lists.<QueryRunner<SegmentAnalysis>>newArrayList(
toolChest.preMergeQueryDecoration(runner1),
toolChest.preMergeQueryDecoration(runner2)
)
)
),
toolChest
);
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(
Druids.newSegmentMetadataQueryBuilder()
.dataSource("testing")
.intervals("2013/2014")
.toInclude(new ListColumnIncluderator(Arrays.asList("placement")))
.analysisTypes(SegmentMetadataQuery.AnalysisType.QUERYGRANULARITY)
.merge(true)
.build(),
Maps.newHashMap()
),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
}
@Test
public void testBySegmentResults()
{

View File

@ -182,6 +182,11 @@ public class IndexMergerTest
IncrementalIndexTest.getDefaultCombiningAggregatorFactories(),
index.getMetadata().getAggregators()
);
Assert.assertEquals(
QueryGranularity.NONE,
index.getMetadata().getQueryGranularity()
);
}
@Test
@ -267,6 +272,7 @@ public class IndexMergerTest
.setAggregators(
IncrementalIndexTest.getDefaultCombiningAggregatorFactories()
)
.setQueryGranularity(QueryGranularity.NONE)
.putAll(metadataElems),
index.getMetadata()
);

View File

@ -21,6 +21,7 @@ package io.druid.segment;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import io.druid.granularity.QueryGranularity;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
@ -49,6 +50,7 @@ public class MetadataTest
new LongSumAggregatorFactory("out", "in")
};
metadata.setAggregators(aggregators);
metadata.setQueryGranularity(QueryGranularity.ALL);
Metadata other = jsonMapper.readValue(
jsonMapper.writeValueAsString(metadata),
@ -76,10 +78,12 @@ public class MetadataTest
Metadata m1 = new Metadata();
m1.put("k", "v");
m1.setAggregators(aggs);
m1.setQueryGranularity(QueryGranularity.ALL);
Metadata m2 = new Metadata();
m2.put("k", "v");
m2.setAggregators(aggs);
m2.setQueryGranularity(QueryGranularity.ALL);
Metadata merged = new Metadata();
merged.put("k", "v");
@ -88,6 +92,7 @@ public class MetadataTest
new LongMaxAggregatorFactory("n", "n")
}
);
merged.setQueryGranularity(QueryGranularity.ALL);
Assert.assertEquals(merged, Metadata.merge(ImmutableList.of(m1, m2), null));
//merge check with one metadata being null
@ -97,6 +102,7 @@ public class MetadataTest
metadataToBeMerged.add(null);
merged.setAggregators(null);
merged.setQueryGranularity(null);
Assert.assertEquals(merged, Metadata.merge(metadataToBeMerged, null));
//merge check with client explicitly providing merged aggregators
@ -110,6 +116,7 @@ public class MetadataTest
Metadata.merge(metadataToBeMerged, explicitAggs)
);
merged.setQueryGranularity(QueryGranularity.ALL);
Assert.assertEquals(
merged,
Metadata.merge(ImmutableList.of(m1, m2), explicitAggs)