[Rollup] Remove builders from DateHistogramGroupConfig (#32555)

Same motivation as #32507 but for the DateHistogramGroupConfig
configuration object. This pull request also changes the format of the
time zone from a Joda's DateTimeZone to a simple String.

It should help to port the API to the high level rest client and allows
clients to not be forced to use the Joda Time library. Serialization is
impacted but does not need a backward compatibility layer as
DateTimeZone are serialized as String anyway. XContent also expects
a String for timezone, so I found it easier to move everything to String.

Related to #29827
This commit is contained in:
Tanguy Leroux 2018-08-03 13:11:00 +02:00 committed by GitHub
parent d05f39de8b
commit 21f660d801
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 289 additions and 315 deletions

View File

@ -7,6 +7,7 @@ package org.elasticsearch.xpack.core.rollup.job;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.fieldcaps.FieldCapabilities;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
@ -15,8 +16,8 @@ import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.rounding.DateTimeUnit;
import org.elasticsearch.common.rounding.Rounding;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.ToXContentFragment;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.bucket.composite.CompositeValuesSourceBuilder;
@ -33,6 +34,10 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg;
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg;
import static org.elasticsearch.common.xcontent.ObjectParser.ValueType;
/**
* The configuration object for the histograms in the rollup config
*
@ -47,71 +52,100 @@ import java.util.Objects;
* ]
* }
*/
public class DateHistoGroupConfig implements Writeable, ToXContentFragment {
private static final String NAME = "date_histo_group_config";
public static final ObjectParser<DateHistoGroupConfig.Builder, Void> PARSER
= new ObjectParser<>(NAME, DateHistoGroupConfig.Builder::new);
private static final ParseField INTERVAL = new ParseField("interval");
private static final ParseField DELAY = new ParseField("delay");
private static final ParseField FIELD = new ParseField("field");
public static final ParseField TIME_ZONE = new ParseField("time_zone");
private final DateHistogramInterval interval;
private final String field;
private final DateTimeZone timeZone;
private final DateHistogramInterval delay;
public class DateHistogramGroupConfig implements Writeable, ToXContentObject {
private static final String NAME = "date_histogram";
private static final String INTERVAL = "interval";
private static final String FIELD = "field";
public static final String TIME_ZONE = "time_zone";
private static final String DELAY = "delay";
private static final String DEFAULT_TIMEZONE = "UTC";
private static final ConstructingObjectParser<DateHistogramGroupConfig, Void> PARSER;
static {
PARSER.declareField(DateHistoGroupConfig.Builder::setInterval,
p -> new DateHistogramInterval(p.text()), INTERVAL, ObjectParser.ValueType.STRING);
PARSER.declareString(DateHistoGroupConfig.Builder::setField, FIELD);
PARSER.declareField(DateHistoGroupConfig.Builder::setDelay,
p -> new DateHistogramInterval(p.text()), DELAY, ObjectParser.ValueType.LONG);
PARSER.declareField(DateHistoGroupConfig.Builder::setTimeZone, p -> {
if (p.currentToken() == XContentParser.Token.VALUE_STRING) {
return DateTimeZone.forID(p.text());
} else {
return DateTimeZone.forOffsetHours(p.intValue());
}
}, TIME_ZONE, ObjectParser.ValueType.LONG);
PARSER = new ConstructingObjectParser<>(NAME, a ->
new DateHistogramGroupConfig((String) a[0], (DateHistogramInterval) a[1], (DateHistogramInterval) a[2], (String) a[3]));
PARSER.declareString(constructorArg(), new ParseField(FIELD));
PARSER.declareField(constructorArg(), p -> new DateHistogramInterval(p.text()), new ParseField(INTERVAL), ValueType.STRING);
PARSER.declareField(optionalConstructorArg(), p -> new DateHistogramInterval(p.text()), new ParseField(DELAY), ValueType.STRING);
PARSER.declareString(optionalConstructorArg(), new ParseField(TIME_ZONE));
}
private DateHistoGroupConfig(DateHistogramInterval interval,
String field,
DateHistogramInterval delay,
DateTimeZone timeZone) {
private final String field;
private final DateHistogramInterval interval;
private final DateHistogramInterval delay;
private final String timeZone;
/**
* Create a new {@link DateHistogramGroupConfig} using the given field and interval parameters.
*/
public DateHistogramGroupConfig(final String field, final DateHistogramInterval interval) {
this(field, interval, null, null);
}
/**
* Create a new {@link DateHistogramGroupConfig} using the given configuration parameters.
* <p>
* The {@code field} and {@code interval} are required to compute the date histogram for the rolled up documents.
* The {@code delay} is optional and can be set to {@code null}. It defines how long to wait before rolling up new documents.
* The {@code timeZone} is optional and can be set to {@code null}. When configured, the time zone value is resolved using
* ({@link DateTimeZone#forID(String)} and must match a time zone identifier provided by the Joda Time library.
* </p>
* @param field the name of the date field to use for the date histogram (required)
* @param interval the interval to use for the date histogram (required)
* @param delay the time delay (optional)
* @param timeZone the id of time zone to use to calculate the date histogram (optional). When {@code null}, the UTC timezone is used.
*/
public DateHistogramGroupConfig(final String field,
final DateHistogramInterval interval,
final @Nullable DateHistogramInterval delay,
final @Nullable String timeZone) {
if (field == null || field.isEmpty()) {
throw new IllegalArgumentException("Field must be a non-null, non-empty string");
}
if (interval == null) {
throw new IllegalArgumentException("Interval must be non-null");
}
this.interval = interval;
this.field = field;
this.delay = delay;
this.timeZone = Objects.requireNonNull(timeZone);
this.timeZone = (timeZone != null && timeZone.isEmpty() == false) ? timeZone : DEFAULT_TIMEZONE;
// validate interval
createRounding(this.interval.toString(), this.timeZone);
if (delay != null) {
// and delay
TimeValue.parseTimeValue(this.delay.toString(), DELAY);
}
}
DateHistoGroupConfig(StreamInput in) throws IOException {
DateHistogramGroupConfig(final StreamInput in) throws IOException {
interval = new DateHistogramInterval(in);
field = in.readString();
delay = in.readOptionalWriteable(DateHistogramInterval::new);
timeZone = in.readTimeZone();
timeZone = in.readString();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
public void writeTo(final StreamOutput out) throws IOException {
interval.writeTo(out);
out.writeString(field);
out.writeOptionalWriteable(delay);
out.writeTimeZone(timeZone);
out.writeString(timeZone);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.field(INTERVAL.getPreferredName(), interval.toString());
builder.field(FIELD.getPreferredName(), field);
if (delay != null) {
builder.field(DELAY.getPreferredName(), delay.toString());
public XContentBuilder toXContent(final XContentBuilder builder, final Params params) throws IOException {
builder.startObject();
{
builder.field(INTERVAL, interval.toString());
builder.field(FIELD, field);
if (delay != null) {
builder.field(DELAY, delay.toString());
}
builder.field(TIME_ZONE, timeZone);
}
builder.field(TIME_ZONE.getPreferredName(), timeZone.toString());
return builder;
return builder.endObject();
}
/**
@ -138,7 +172,7 @@ public class DateHistoGroupConfig implements Writeable, ToXContentFragment {
/**
* Get the timezone to apply
*/
public DateTimeZone getTimeZone() {
public String getTimeZone() {
return timeZone;
}
@ -146,9 +180,9 @@ public class DateHistoGroupConfig implements Writeable, ToXContentFragment {
* Create the rounding for this date histogram
*/
public Rounding createRounding() {
return createRounding(interval.toString(), timeZone, "");
return createRounding(interval.toString(), timeZone);
}
;
/**
* This returns a set of aggregation builders which represent the configured
* set of date histograms. Used by the rollup indexer to iterate over historical data
@ -158,7 +192,7 @@ public class DateHistoGroupConfig implements Writeable, ToXContentFragment {
new DateHistogramValuesSourceBuilder(RollupField.formatIndexerAggName(field, DateHistogramAggregationBuilder.NAME));
vsBuilder.dateHistogramInterval(interval);
vsBuilder.field(field);
vsBuilder.timeZone(timeZone);
vsBuilder.timeZone(toDateTimeZone(timeZone));
return Collections.singletonList(vsBuilder);
}
@ -168,11 +202,11 @@ public class DateHistoGroupConfig implements Writeable, ToXContentFragment {
public Map<String, Object> toAggCap() {
Map<String, Object> map = new HashMap<>(3);
map.put("agg", DateHistogramAggregationBuilder.NAME);
map.put(INTERVAL.getPreferredName(), interval.toString());
map.put(INTERVAL, interval.toString());
if (delay != null) {
map.put(DELAY.getPreferredName(), delay.toString());
map.put(DELAY, delay.toString());
}
map.put(TIME_ZONE.getPreferredName(), timeZone.toString());
map.put(TIME_ZONE, timeZone);
return map;
}
@ -204,21 +238,18 @@ public class DateHistoGroupConfig implements Writeable, ToXContentFragment {
}
@Override
public boolean equals(Object other) {
public boolean equals(final Object other) {
if (this == other) {
return true;
}
if (other == null || getClass() != other.getClass()) {
return false;
}
DateHistoGroupConfig that = (DateHistoGroupConfig) other;
return Objects.equals(this.interval, that.interval)
&& Objects.equals(this.field, that.field)
&& Objects.equals(this.delay, that.delay)
&& Objects.equals(this.timeZone, that.timeZone);
final DateHistogramGroupConfig that = (DateHistogramGroupConfig) other;
return Objects.equals(interval, that.interval)
&& Objects.equals(field, that.field)
&& Objects.equals(delay, that.delay)
&& Objects.equals(timeZone, that.timeZone);
}
@Override
@ -231,77 +262,28 @@ public class DateHistoGroupConfig implements Writeable, ToXContentFragment {
return Strings.toString(this, true, true);
}
private static Rounding createRounding(String expr, DateTimeZone timeZone, String settingName) {
public static DateHistogramGroupConfig fromXContent(final XContentParser parser) throws IOException {
return PARSER.parse(parser, null);
}
private static Rounding createRounding(final String expr, final String timeZone) {
DateTimeUnit timeUnit = DateHistogramAggregationBuilder.DATE_FIELD_UNITS.get(expr);
final Rounding.Builder rounding;
if (timeUnit != null) {
rounding = new Rounding.Builder(timeUnit);
} else {
rounding = new Rounding.Builder(TimeValue.parseTimeValue(expr, settingName));
rounding = new Rounding.Builder(TimeValue.parseTimeValue(expr, "createRounding"));
}
rounding.timeZone(timeZone);
rounding.timeZone(toDateTimeZone(timeZone));
return rounding.build();
}
public static class Builder {
private DateHistogramInterval interval;
private String field;
private DateHistogramInterval delay;
private DateTimeZone timeZone;
public DateHistogramInterval getInterval() {
return interval;
}
public DateHistoGroupConfig.Builder setInterval(DateHistogramInterval interval) {
this.interval = interval;
return this;
}
public String getField() {
return field;
}
public DateHistoGroupConfig.Builder setField(String field) {
this.field = field;
return this;
}
public DateTimeZone getTimeZone() {
return timeZone;
}
public DateHistoGroupConfig.Builder setTimeZone(DateTimeZone timeZone) {
this.timeZone = timeZone;
return this;
}
public DateHistogramInterval getDelay() {
return delay;
}
public DateHistoGroupConfig.Builder setDelay(DateHistogramInterval delay) {
this.delay = delay;
return this;
}
public DateHistoGroupConfig build() {
if (field == null || field.isEmpty()) {
throw new IllegalArgumentException("Parameter [" + FIELD.getPreferredName() + "] is mandatory.");
}
if (timeZone == null) {
timeZone = DateTimeZone.UTC;
}
if (interval == null) {
throw new IllegalArgumentException("Parameter [" + INTERVAL.getPreferredName() + "] is mandatory.");
}
// validate interval
createRounding(interval.toString(), timeZone, INTERVAL.getPreferredName());
if (delay != null) {
// and delay
TimeValue.parseTimeValue(delay.toString(), INTERVAL.getPreferredName());
}
return new DateHistoGroupConfig(interval, field, delay, timeZone);
private static DateTimeZone toDateTimeZone(final String timezone) {
try {
return DateTimeZone.forOffsetHours(Integer.parseInt(timezone));
} catch (NumberFormatException e) {
return DateTimeZone.forID(timezone);
}
}
}

View File

@ -43,31 +43,31 @@ public class GroupConfig implements Writeable, ToXContentObject {
private static final ParseField HISTO = new ParseField("histogram");
private static final ParseField TERMS = new ParseField("terms");
private final DateHistoGroupConfig dateHisto;
private final DateHistogramGroupConfig dateHisto;
private final HistogramGroupConfig histo;
private final TermsGroupConfig terms;
public static final ObjectParser<GroupConfig.Builder, Void> PARSER = new ObjectParser<>(NAME, GroupConfig.Builder::new);
static {
PARSER.declareObject(GroupConfig.Builder::setDateHisto, (p,c) -> DateHistoGroupConfig.PARSER.apply(p,c).build(), DATE_HISTO);
PARSER.declareObject(GroupConfig.Builder::setDateHisto, (p,c) -> DateHistogramGroupConfig.fromXContent(p), DATE_HISTO);
PARSER.declareObject(GroupConfig.Builder::setHisto, (p,c) -> HistogramGroupConfig.fromXContent(p), HISTO);
PARSER.declareObject(GroupConfig.Builder::setTerms, (p,c) -> TermsGroupConfig.fromXContent(p), TERMS);
}
private GroupConfig(DateHistoGroupConfig dateHisto, @Nullable HistogramGroupConfig histo, @Nullable TermsGroupConfig terms) {
private GroupConfig(DateHistogramGroupConfig dateHisto, @Nullable HistogramGroupConfig histo, @Nullable TermsGroupConfig terms) {
this.dateHisto = Objects.requireNonNull(dateHisto, "A date_histogram group is mandatory");
this.histo = histo;
this.terms = terms;
}
GroupConfig(StreamInput in) throws IOException {
dateHisto = new DateHistoGroupConfig(in);
dateHisto = new DateHistogramGroupConfig(in);
histo = in.readOptionalWriteable(HistogramGroupConfig::new);
terms = in.readOptionalWriteable(TermsGroupConfig::new);
}
public DateHistoGroupConfig getDateHisto() {
public DateHistogramGroupConfig getDateHisto() {
return dateHisto;
}
@ -105,9 +105,7 @@ public class GroupConfig implements Writeable, ToXContentObject {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.startObject(DATE_HISTO.getPreferredName());
dateHisto.toXContent(builder, params);
builder.endObject();
builder.field(DATE_HISTO.getPreferredName(), dateHisto);
if (histo != null) {
builder.field(HISTO.getPreferredName(), histo);
}
@ -153,15 +151,15 @@ public class GroupConfig implements Writeable, ToXContentObject {
}
public static class Builder {
private DateHistoGroupConfig dateHisto;
private DateHistogramGroupConfig dateHisto;
private HistogramGroupConfig histo;
private TermsGroupConfig terms;
public DateHistoGroupConfig getDateHisto() {
public DateHistogramGroupConfig getDateHisto() {
return dateHisto;
}
public GroupConfig.Builder setDateHisto(DateHistoGroupConfig dateHisto) {
public GroupConfig.Builder setDateHisto(DateHistogramGroupConfig dateHisto) {
this.dateHisto = dateHisto;
return this;
}

View File

@ -9,7 +9,7 @@ import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xpack.core.rollup.job.DateHistoGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.GroupConfig;
import org.elasticsearch.xpack.core.rollup.job.HistogramGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.MetricConfig;
@ -25,6 +25,7 @@ import java.util.stream.IntStream;
import static com.carrotsearch.randomizedtesting.generators.RandomNumbers.randomIntBetween;
import static com.carrotsearch.randomizedtesting.generators.RandomStrings.randomAsciiAlphanumOfLengthBetween;
import static org.elasticsearch.test.ESTestCase.randomDateTimeZone;
public class ConfigTestHelpers {
@ -46,7 +47,7 @@ public class ConfigTestHelpers {
public static GroupConfig.Builder getGroupConfig() {
GroupConfig.Builder groupBuilder = new GroupConfig.Builder();
groupBuilder.setDateHisto(getDateHisto().build());
groupBuilder.setDateHisto(randomDateHistogramGroupConfig(ESTestCase.random()));
if (ESTestCase.randomBoolean()) {
groupBuilder.setHisto(randomHistogramGroupConfig(ESTestCase.random()));
}
@ -61,17 +62,12 @@ public class ConfigTestHelpers {
return ESTestCase.randomIntBetween(1, 1000) + ESTestCase.randomFrom(TIME_SUFFIXES);
}
public static DateHistoGroupConfig.Builder getDateHisto() {
DateHistoGroupConfig.Builder dateHistoBuilder = new DateHistoGroupConfig.Builder();
dateHistoBuilder.setInterval(new DateHistogramInterval(randomPositiveTimeValue()));
if (ESTestCase.randomBoolean()) {
dateHistoBuilder.setTimeZone(ESTestCase.randomDateTimeZone());
}
if (ESTestCase.randomBoolean()) {
dateHistoBuilder.setDelay(new DateHistogramInterval(randomPositiveTimeValue()));
}
dateHistoBuilder.setField(ESTestCase.randomAlphaOfLengthBetween(5, 10));
return dateHistoBuilder;
public static DateHistogramGroupConfig randomDateHistogramGroupConfig(final Random random) {
final String field = randomField(random);
final DateHistogramInterval interval = randomInterval();
final DateHistogramInterval delay = random.nextBoolean() ? randomInterval() : null;
final String timezone = random.nextBoolean() ? randomDateTimeZone().toString() : null;
return new DateHistogramGroupConfig(field, interval, delay, timezone);
}
public static List<String> getFields() {
@ -140,10 +136,14 @@ public class ConfigTestHelpers {
return fields;
}
private static String randomField(final Random random) {
public static String randomField(final Random random) {
return randomAsciiAlphanumOfLengthBetween(random, 5, 10);
}
public static DateHistogramInterval randomInterval() {
return new DateHistogramInterval(randomPositiveTimeValue());
}
private static long randomInterval(final Random random) {
return RandomNumbers.randomLongBetween(random, 1L, Long.MAX_VALUE);
}

View File

@ -7,52 +7,52 @@ package org.elasticsearch.xpack.core.rollup.job;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.fieldcaps.FieldCapabilities;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
import org.elasticsearch.test.AbstractSerializingTestCase;
import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers;
import org.joda.time.DateTimeZone;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import static org.elasticsearch.xpack.core.rollup.ConfigTestHelpers.randomDateHistogramGroupConfig;
import static org.hamcrest.Matchers.equalTo;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class DateHistoGroupConfigSerializingTests extends AbstractSerializingTestCase<DateHistoGroupConfig> {
public class DateHistogramGroupConfigSerializingTests extends AbstractSerializingTestCase<DateHistogramGroupConfig> {
@Override
protected DateHistoGroupConfig doParseInstance(XContentParser parser) throws IOException {
return DateHistoGroupConfig.PARSER.apply(parser, null).build();
protected DateHistogramGroupConfig doParseInstance(final XContentParser parser) throws IOException {
return DateHistogramGroupConfig.fromXContent(parser);
}
@Override
protected Writeable.Reader<DateHistoGroupConfig> instanceReader() {
return DateHistoGroupConfig::new;
protected Writeable.Reader<DateHistogramGroupConfig> instanceReader() {
return DateHistogramGroupConfig::new;
}
@Override
protected DateHistoGroupConfig createTestInstance() {
return ConfigTestHelpers.getDateHisto().build();
protected DateHistogramGroupConfig createTestInstance() {
return randomDateHistogramGroupConfig(random());
}
public void testValidateNoMapping() throws IOException {
public void testValidateNoMapping() {
ActionRequestValidationException e = new ActionRequestValidationException();
Map<String, Map<String, FieldCapabilities>> responseMap = new HashMap<>();
DateHistoGroupConfig config = new DateHistoGroupConfig.Builder()
.setField("my_field")
.setInterval(new DateHistogramInterval("1d"))
.build();
DateHistogramGroupConfig config = new DateHistogramGroupConfig("my_field", new DateHistogramInterval("1d"), null, null);
config.validateMappings(responseMap, e);
assertThat(e.validationErrors().get(0), equalTo("Could not find a [date] field with name [my_field] in any of the " +
"indices matching the index pattern."));
}
public void testValidateNomatchingField() throws IOException {
public void testValidateNomatchingField() {
ActionRequestValidationException e = new ActionRequestValidationException();
Map<String, Map<String, FieldCapabilities>> responseMap = new HashMap<>();
@ -60,17 +60,13 @@ public class DateHistoGroupConfigSerializingTests extends AbstractSerializingTes
FieldCapabilities fieldCaps = mock(FieldCapabilities.class);
responseMap.put("some_other_field", Collections.singletonMap("date", fieldCaps));
DateHistoGroupConfig config = new DateHistoGroupConfig.Builder()
.setField("my_field")
.setInterval(new DateHistogramInterval("1d"))
.build();
DateHistogramGroupConfig config = new DateHistogramGroupConfig("my_field", new DateHistogramInterval("1d"), null, null);
config.validateMappings(responseMap, e);
assertThat(e.validationErrors().get(0), equalTo("Could not find a [date] field with name [my_field] in any of the " +
"indices matching the index pattern."));
}
public void testValidateFieldWrongType() throws IOException {
public void testValidateFieldWrongType() {
ActionRequestValidationException e = new ActionRequestValidationException();
Map<String, Map<String, FieldCapabilities>> responseMap = new HashMap<>();
@ -78,17 +74,13 @@ public class DateHistoGroupConfigSerializingTests extends AbstractSerializingTes
FieldCapabilities fieldCaps = mock(FieldCapabilities.class);
responseMap.put("my_field", Collections.singletonMap("keyword", fieldCaps));
DateHistoGroupConfig config = new DateHistoGroupConfig.Builder()
.setField("my_field")
.setInterval(new DateHistogramInterval("1d"))
.build();
DateHistogramGroupConfig config = new DateHistogramGroupConfig("my_field", new DateHistogramInterval("1d"), null, null);
config.validateMappings(responseMap, e);
assertThat(e.validationErrors().get(0), equalTo("The field referenced by a date_histo group must be a [date] type across all " +
"indices in the index pattern. Found: [keyword] for field [my_field]"));
}
public void testValidateFieldMixtureTypes() throws IOException {
public void testValidateFieldMixtureTypes() {
ActionRequestValidationException e = new ActionRequestValidationException();
Map<String, Map<String, FieldCapabilities>> responseMap = new HashMap<>();
@ -99,17 +91,13 @@ public class DateHistoGroupConfigSerializingTests extends AbstractSerializingTes
types.put("keyword", fieldCaps);
responseMap.put("my_field", types);
DateHistoGroupConfig config = new DateHistoGroupConfig.Builder()
.setField("my_field")
.setInterval(new DateHistogramInterval("1d"))
.build();
DateHistogramGroupConfig config = new DateHistogramGroupConfig("my_field", new DateHistogramInterval("1d"), null, null);
config.validateMappings(responseMap, e);
assertThat(e.validationErrors().get(0), equalTo("The field referenced by a date_histo group must be a [date] type across all " +
"indices in the index pattern. Found: [date, keyword] for field [my_field]"));
}
public void testValidateFieldMatchingNotAggregatable() throws IOException {
public void testValidateFieldMatchingNotAggregatable() {
ActionRequestValidationException e = new ActionRequestValidationException();
Map<String, Map<String, FieldCapabilities>> responseMap = new HashMap<>();
@ -118,15 +106,12 @@ public class DateHistoGroupConfigSerializingTests extends AbstractSerializingTes
when(fieldCaps.isAggregatable()).thenReturn(false);
responseMap.put("my_field", Collections.singletonMap("date", fieldCaps));
DateHistoGroupConfig config = new DateHistoGroupConfig.Builder()
.setField("my_field")
.setInterval(new DateHistogramInterval("1d"))
.build();
DateHistogramGroupConfig config =new DateHistogramGroupConfig("my_field", new DateHistogramInterval("1d"), null, null);
config.validateMappings(responseMap, e);
assertThat(e.validationErrors().get(0), equalTo("The field [my_field] must be aggregatable across all indices, but is not."));
}
public void testValidateMatchingField() throws IOException {
public void testValidateMatchingField() {
ActionRequestValidationException e = new ActionRequestValidationException();
Map<String, Map<String, FieldCapabilities>> responseMap = new HashMap<>();
@ -135,11 +120,49 @@ public class DateHistoGroupConfigSerializingTests extends AbstractSerializingTes
when(fieldCaps.isAggregatable()).thenReturn(true);
responseMap.put("my_field", Collections.singletonMap("date", fieldCaps));
DateHistoGroupConfig config = new DateHistoGroupConfig.Builder()
.setField("my_field")
.setInterval(new DateHistogramInterval("1d"))
.build();
DateHistogramGroupConfig config = new DateHistogramGroupConfig("my_field", new DateHistogramInterval("1d"), null, null);
config.validateMappings(responseMap, e);
assertThat(e.validationErrors().size(), equalTo(0));
}
/**
* Tests that a DateHistogramGroupConfig can be serialized/deserialized correctly after
* the timezone was changed from DateTimeZone to String.
*/
public void testBwcSerialization() throws IOException {
for (int runs = 0; runs < NUMBER_OF_TEST_RUNS; runs++) {
final DateHistogramGroupConfig reference = ConfigTestHelpers.randomDateHistogramGroupConfig(random());
final BytesStreamOutput out = new BytesStreamOutput();
reference.writeTo(out);
// previous way to deserialize a DateHistogramGroupConfig
final StreamInput in = out.bytes().streamInput();
DateHistogramInterval interval = new DateHistogramInterval(in);
String field = in.readString();
DateHistogramInterval delay = in.readOptionalWriteable(DateHistogramInterval::new);
DateTimeZone timeZone = in.readTimeZone();
assertEqualInstances(reference, new DateHistogramGroupConfig(field, interval, delay, timeZone.getID()));
}
for (int runs = 0; runs < NUMBER_OF_TEST_RUNS; runs++) {
final String field = ConfigTestHelpers.randomField(random());
final DateHistogramInterval interval = ConfigTestHelpers.randomInterval();
final DateHistogramInterval delay = randomBoolean() ? ConfigTestHelpers.randomInterval() : null;
final DateTimeZone timezone = randomDateTimeZone();
// previous way to serialize a DateHistogramGroupConfig
final BytesStreamOutput out = new BytesStreamOutput();
interval.writeTo(out);
out.writeString(field);
out.writeOptionalWriteable(delay);
out.writeTimeZone(timezone);
final StreamInput in = out.bytes().streamInput();
DateHistogramGroupConfig deserialized = new DateHistogramGroupConfig(in);
assertEqualInstances(new DateHistogramGroupConfig(field, interval, delay, timezone.getID()), deserialized);
}
}
}

View File

@ -13,7 +13,7 @@ import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilde
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
import org.elasticsearch.xpack.core.rollup.RollupField;
import org.elasticsearch.xpack.core.rollup.action.RollupJobCaps;
import org.elasticsearch.xpack.core.rollup.job.DateHistoGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig;
import org.joda.time.DateTimeZone;
import java.util.ArrayList;
@ -94,7 +94,7 @@ public class RollupJobIdentifierUtils {
for (Map<String, Object> agg : fieldCaps.getAggs()) {
if (agg.get(RollupField.AGG).equals(DateHistogramAggregationBuilder.NAME)) {
TimeValue interval = TimeValue.parseTimeValue((String)agg.get(RollupField.INTERVAL), "date_histogram.interval");
String thisTimezone = (String)agg.get(DateHistoGroupConfig.TIME_ZONE.getPreferredName());
String thisTimezone = (String) agg.get(DateHistogramGroupConfig.TIME_ZONE);
String sourceTimeZone = source.timeZone() == null ? DateTimeZone.UTC.toString() : source.timeZone().toString();
// Ensure we are working on the same timezone

View File

@ -20,7 +20,7 @@ import org.elasticsearch.search.aggregations.metrics.avg.AvgAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.sum.SumAggregationBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
import org.elasticsearch.xpack.core.rollup.RollupField;
import org.elasticsearch.xpack.core.rollup.job.DateHistoGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig;
import org.joda.time.DateTimeZone;
import java.util.ArrayList;
@ -221,7 +221,7 @@ public class RollupRequestTranslator {
String timezone = source.timeZone() == null ? DateTimeZone.UTC.toString() : source.timeZone().toString();
filterConditions.add(new TermQueryBuilder(RollupField.formatFieldName(source,
DateHistoGroupConfig.TIME_ZONE.getPreferredName()), timezone));
DateHistogramGroupConfig.TIME_ZONE), timezone));
rolledDateHisto.offset(source.offset());
if (source.extendedBounds() != null) {

View File

@ -56,7 +56,7 @@ import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.core.rollup.RollupField;
import org.elasticsearch.xpack.core.rollup.action.RollupJobCaps;
import org.elasticsearch.xpack.core.rollup.action.RollupSearchAction;
import org.elasticsearch.xpack.core.rollup.job.DateHistoGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig;
import org.elasticsearch.xpack.rollup.Rollup;
import org.elasticsearch.xpack.rollup.RollupJobIdentifierUtils;
import org.elasticsearch.xpack.rollup.RollupRequestTranslator;
@ -328,10 +328,10 @@ public class TransportRollupSearchAction extends TransportAction<SearchRequest,
// If the cap is for a date_histo, and the query is a range, the timezones need to match
if (type.equals(DateHistogramAggregationBuilder.NAME) && timeZone != null) {
boolean matchingTZ = ((String)agg.get(DateHistoGroupConfig.TIME_ZONE.getPreferredName()))
boolean matchingTZ = ((String)agg.get(DateHistogramGroupConfig.TIME_ZONE))
.equalsIgnoreCase(timeZone);
if (matchingTZ == false) {
incompatibleTimeZones.add((String)agg.get(DateHistoGroupConfig.TIME_ZONE.getPreferredName()));
incompatibleTimeZones.add((String)agg.get(DateHistogramGroupConfig.TIME_ZONE));
}
return matchingTZ;
}

View File

@ -16,7 +16,7 @@ import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregati
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation;
import org.elasticsearch.xpack.core.rollup.RollupField;
import org.elasticsearch.xpack.core.rollup.job.DateHistoGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.GroupConfig;
import org.elasticsearch.xpack.core.rollup.job.RollupJobStats;
import org.elasticsearch.xpack.rollup.Rollup;
@ -83,7 +83,7 @@ class IndexerUtils {
assert v != null;
doc.put(k + "." + RollupField.TIMESTAMP, v);
doc.put(k + "." + RollupField.INTERVAL, groupConfig.getDateHisto().getInterval());
doc.put(k + "." + DateHistoGroupConfig.TIME_ZONE, groupConfig.getDateHisto().getTimeZone().toString());
doc.put(k + "." + DateHistogramGroupConfig.TIME_ZONE, groupConfig.getDateHisto().getTimeZone());
docID.update(Numbers.longToBytes((Long)v), 0, 8);
} else if (k.endsWith("." + HistogramAggregationBuilder.NAME)) {
doc.put(k + "." + RollupField.VALUE, v);

View File

@ -21,7 +21,7 @@ import org.elasticsearch.search.aggregations.bucket.composite.CompositeValuesSou
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.xpack.core.rollup.RollupField;
import org.elasticsearch.xpack.core.rollup.job.DateHistoGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.GroupConfig;
import org.elasticsearch.xpack.core.rollup.job.IndexerState;
import org.elasticsearch.xpack.core.rollup.job.RollupJob;
@ -208,7 +208,7 @@ public abstract class RollupIndexer {
// rounds the current time to its current bucket based on the date histogram interval.
// this is needed to exclude buckets that can still receive new documents.
DateHistoGroupConfig dateHisto = job.getConfig().getGroupConfig().getDateHisto();
DateHistogramGroupConfig dateHisto = job.getConfig().getGroupConfig().getDateHisto();
long rounded = dateHisto.createRounding().round(now);
if (dateHisto.getDelay() != null) {
// if the job has a delay we filter all documents that appear before it.
@ -413,7 +413,7 @@ public abstract class RollupIndexer {
*/
private QueryBuilder createBoundaryQuery(Map<String, Object> position) {
assert maxBoundary < Long.MAX_VALUE;
DateHistoGroupConfig dateHisto = job.getConfig().getGroupConfig().getDateHisto();
DateHistogramGroupConfig dateHisto = job.getConfig().getGroupConfig().getDateHisto();
String fieldName = dateHisto.getField();
String rollupFieldName = fieldName + "." + DateHistogramAggregationBuilder.NAME;
long lowerBound = 0L;

View File

@ -17,7 +17,7 @@ import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers;
import org.elasticsearch.xpack.core.rollup.action.RollupJobCaps;
import org.elasticsearch.xpack.core.rollup.job.DateHistoGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.GroupConfig;
import org.elasticsearch.xpack.core.rollup.job.HistogramGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.MetricConfig;
@ -37,7 +37,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
public void testOneMatch() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build());
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")));
job.setGroupConfig(group.build());
RollupJobCaps cap = new RollupJobCaps(job.build());
Set<RollupJobCaps> caps = singletonSet(cap);
@ -52,7 +52,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
public void testBiggerButCompatibleInterval() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build());
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")));
job.setGroupConfig(group.build());
RollupJobCaps cap = new RollupJobCaps(job.build());
Set<RollupJobCaps> caps = singletonSet(cap);
@ -67,7 +67,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
public void testIncompatibleInterval() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1d")).build());
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1d")));
job.setGroupConfig(group.build());
RollupJobCaps cap = new RollupJobCaps(job.build());
Set<RollupJobCaps> caps = singletonSet(cap);
@ -83,8 +83,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
public void testBadTimeZone() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1d"))
.setTimeZone(DateTimeZone.forID("EST")).build());
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h"), null, "EST"));
job.setGroupConfig(group.build());
RollupJobCaps cap = new RollupJobCaps(job.build());
Set<RollupJobCaps> caps = singletonSet(cap);
@ -101,7 +100,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
public void testMetricOnlyAgg() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build());
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")));
job.setGroupConfig(group.build());
job.setMetricsConfig(singletonList(new MetricConfig("bar", singletonList("max"))));
RollupJobCaps cap = new RollupJobCaps(job.build());
@ -116,7 +115,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
public void testOneOfTwoMatchingCaps() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build());
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")));
job.setGroupConfig(group.build());
RollupJobCaps cap = new RollupJobCaps(job.build());
Set<RollupJobCaps> caps = singletonSet(cap);
@ -133,7 +132,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
public void testTwoJobsSameRollupIndex() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build());
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")));
group.setTerms(null);
group.setHisto(null);
job.setGroupConfig(group.build());
@ -143,7 +142,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
RollupJobConfig.Builder job2 = ConfigTestHelpers.getRollupJob("foo2");
GroupConfig.Builder group2 = ConfigTestHelpers.getGroupConfig();
group2.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build());
group2.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")));
group2.setTerms(null);
group2.setHisto(null);
job2.setGroupConfig(group.build());
@ -163,7 +162,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
public void testTwoJobsButBothPartialMatches() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build());
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")));
job.setGroupConfig(group.build());
job.setMetricsConfig(singletonList(new MetricConfig("bar", singletonList("max"))));
RollupJobCaps cap = new RollupJobCaps(job.build());
@ -172,7 +171,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
RollupJobConfig.Builder job2 = ConfigTestHelpers.getRollupJob("foo2");
GroupConfig.Builder group2 = ConfigTestHelpers.getGroupConfig();
group2.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build());
group2.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")));
job2.setGroupConfig(group.build());
job.setMetricsConfig(singletonList(new MetricConfig("bar", singletonList("min"))));
RollupJobCaps cap2 = new RollupJobCaps(job2.build());
@ -191,7 +190,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
public void testComparableDifferentDateIntervals() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build())
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")))
.setHisto(null)
.setTerms(null);
job.setGroupConfig(group.build());
@ -199,7 +198,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
RollupJobConfig.Builder job2 = ConfigTestHelpers.getRollupJob("foo2").setRollupIndex(job.getRollupIndex());
GroupConfig.Builder group2 = ConfigTestHelpers.getGroupConfig();
group2.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1d")).build())
group2.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1d")))
.setHisto(null)
.setTerms(null);
job2.setGroupConfig(group2.build());
@ -220,7 +219,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
public void testComparableDifferentDateIntervalsOnlyOneWorks() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build())
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")))
.setHisto(null)
.setTerms(null);
job.setGroupConfig(group.build());
@ -228,7 +227,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
RollupJobConfig.Builder job2 = ConfigTestHelpers.getRollupJob("foo2").setRollupIndex(job.getRollupIndex());
GroupConfig.Builder group2 = ConfigTestHelpers.getGroupConfig();
group2.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1d")).build())
group2.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1d")))
.setHisto(null)
.setTerms(null);
job2.setGroupConfig(group2.build());
@ -249,7 +248,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
public void testComparableNoHistoVsHisto() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build())
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")))
.setHisto(null)
.setTerms(null);
job.setGroupConfig(group.build());
@ -257,7 +256,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
RollupJobConfig.Builder job2 = ConfigTestHelpers.getRollupJob("foo2").setRollupIndex(job.getRollupIndex());
GroupConfig.Builder group2 = ConfigTestHelpers.getGroupConfig();
group2.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build())
group2.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")))
.setHisto(new HistogramGroupConfig(100L, "bar"))
.setTerms(null);
job2.setGroupConfig(group2.build());
@ -279,7 +278,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
public void testComparableNoTermsVsTerms() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build())
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")))
.setHisto(null)
.setTerms(null);
job.setGroupConfig(group.build());
@ -287,7 +286,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
RollupJobConfig.Builder job2 = ConfigTestHelpers.getRollupJob("foo2").setRollupIndex(job.getRollupIndex());
GroupConfig.Builder group2 = ConfigTestHelpers.getGroupConfig();
group2.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build())
group2.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")))
.setHisto(null)
.setTerms(new TermsGroupConfig("bar"));
job2.setGroupConfig(group2.build());
@ -315,13 +314,10 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
RollupJobConfig job = ConfigTestHelpers.getRollupJob("foo")
.setGroupConfig(ConfigTestHelpers.getGroupConfig()
.setDateHisto(new DateHistoGroupConfig.Builder()
.setInterval(new DateHistogramInterval("1d"))
.setField("foo") // <-- NOTE same name but wrong type
.setTimeZone(DateTimeZone.UTC)
.build())
.setHisto(new HistogramGroupConfig(1L, "baz")) // <-- NOTE right type but wrong name
.build())
// NOTE same name but wrong type
.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1d"), null, DateTimeZone.UTC.getID()))
.setHisto(new HistogramGroupConfig(1L, "baz")) // <-- NOTE right type but wrong name
.build())
.setMetricsConfig(
Arrays.asList(new MetricConfig("max_field", singletonList("max")), new MetricConfig("avg_field", singletonList("avg"))))
.build();
@ -341,11 +337,7 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
RollupJobConfig job = ConfigTestHelpers.getRollupJob("foo")
.setGroupConfig(ConfigTestHelpers.getGroupConfig()
.setDateHisto(new DateHistoGroupConfig.Builder()
.setInterval(new DateHistogramInterval("1d"))
.setField("foo")
.setTimeZone(DateTimeZone.UTC)
.build())
.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1d"), null, DateTimeZone.UTC.getID()))
.build())
.setMetricsConfig(
Arrays.asList(new MetricConfig("max_field", singletonList("max")), new MetricConfig("avg_field", singletonList("avg"))))
@ -366,12 +358,9 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
RollupJobConfig job = ConfigTestHelpers.getRollupJob("foo")
.setGroupConfig(ConfigTestHelpers.getGroupConfig()
.setDateHisto(new DateHistoGroupConfig.Builder()
.setInterval(new DateHistogramInterval("100d")) // <- interval in job is much higher than agg interval above
.setField("foo")
.setTimeZone(DateTimeZone.UTC)
.build())
.build())
// interval in job is much higher than agg interval above
.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("100d"), null, DateTimeZone.UTC.getID()))
.build())
.build();
Set<RollupJobCaps> caps = singletonSet(new RollupJobCaps(job));
@ -389,12 +378,9 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
RollupJobConfig job = ConfigTestHelpers.getRollupJob("foo")
.setGroupConfig(ConfigTestHelpers.getGroupConfig()
.setDateHisto(new DateHistoGroupConfig.Builder()
.setInterval(new DateHistogramInterval("1d"))
.setField("bar") // <-- NOTE different field from the one in the query
.setTimeZone(DateTimeZone.UTC)
.build())
.build())
// NOTE different field from the one in the query
.setDateHisto(new DateHistogramGroupConfig("bar", new DateHistogramInterval("1d"), null, DateTimeZone.UTC.getID()))
.build())
.setMetricsConfig(
Arrays.asList(new MetricConfig("max_field", singletonList("max")), new MetricConfig("avg_field", singletonList("avg"))))
.build();
@ -414,13 +400,9 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
RollupJobConfig job = ConfigTestHelpers.getRollupJob("foo")
.setGroupConfig(ConfigTestHelpers.getGroupConfig()
.setDateHisto(new DateHistoGroupConfig.Builder()
.setInterval(new DateHistogramInterval("1d"))
.setField("bar")
.setTimeZone(DateTimeZone.UTC)
.build())
.setHisto(new HistogramGroupConfig(1L, "baz")) // <-- NOTE right type but wrong name
.build())
.setDateHisto(new DateHistogramGroupConfig("bar", new DateHistogramInterval("1d"), null, DateTimeZone.UTC.getID()))
.setHisto(new HistogramGroupConfig(1L, "baz")) // <-- NOTE right type but wrong name
.build())
.setMetricsConfig(
Arrays.asList(new MetricConfig("max_field", singletonList("max")), new MetricConfig("avg_field", singletonList("avg"))))
.build();
@ -440,13 +422,9 @@ public class RollupJobIdentifierUtilTests extends ESTestCase {
RollupJobConfig job = ConfigTestHelpers.getRollupJob("foo")
.setGroupConfig(ConfigTestHelpers.getGroupConfig()
.setDateHisto(new DateHistoGroupConfig.Builder()
.setInterval(new DateHistogramInterval("1d"))
.setField("foo")
.setTimeZone(DateTimeZone.UTC)
.build())
.setHisto(new HistogramGroupConfig(1L, "baz")) // <-- NOTE right type but wrong name
.build())
.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1d"), null, DateTimeZone.UTC.getID()))
.setHisto(new HistogramGroupConfig(1L, "baz")) // <-- NOTE right type but wrong name
.build())
.build();
Set<RollupJobCaps> caps = singletonSet(new RollupJobCaps(job));

View File

@ -52,7 +52,7 @@ import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers;
import org.elasticsearch.xpack.core.rollup.RollupField;
import org.elasticsearch.xpack.core.rollup.action.RollupJobCaps;
import org.elasticsearch.xpack.core.rollup.job.DateHistoGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.GroupConfig;
import org.elasticsearch.xpack.core.rollup.job.RollupJobConfig;
import org.elasticsearch.xpack.core.rollup.job.TermsGroupConfig;
@ -120,7 +120,7 @@ public class SearchActionTests extends ESTestCase {
public void testRange() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build());
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")));
job.setGroupConfig(group.build());
RollupJobCaps cap = new RollupJobCaps(job.build());
Set<RollupJobCaps> caps = new HashSet<>();
@ -133,7 +133,7 @@ public class SearchActionTests extends ESTestCase {
public void testRangeNullTimeZone() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build());
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")));
job.setGroupConfig(group.build());
RollupJobCaps cap = new RollupJobCaps(job.build());
Set<RollupJobCaps> caps = new HashSet<>();
@ -146,7 +146,7 @@ public class SearchActionTests extends ESTestCase {
public void testRangeWrongTZ() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build());
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")));
job.setGroupConfig(group.build());
RollupJobCaps cap = new RollupJobCaps(job.build());
Set<RollupJobCaps> caps = new HashSet<>();
@ -190,7 +190,7 @@ public class SearchActionTests extends ESTestCase {
public void testCompounds() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build());
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")));
job.setGroupConfig(group.build());
RollupJobCaps cap = new RollupJobCaps(job.build());
Set<RollupJobCaps> caps = new HashSet<>();
@ -206,7 +206,7 @@ public class SearchActionTests extends ESTestCase {
public void testMatchAll() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build());
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")));
job.setGroupConfig(group.build());
RollupJobCaps cap = new RollupJobCaps(job.build());
Set<RollupJobCaps> caps = new HashSet<>();
@ -218,7 +218,7 @@ public class SearchActionTests extends ESTestCase {
public void testAmbiguousResolution() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build());
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")));
group.setTerms(new TermsGroupConfig("foo"));
job.setGroupConfig(group.build());
RollupJobCaps cap = new RollupJobCaps(job.build());
@ -369,7 +369,7 @@ public class SearchActionTests extends ESTestCase {
public void testGood() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build());
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")));
job.setGroupConfig(group.build());
RollupJobCaps cap = new RollupJobCaps(job.build());
Set<RollupJobCaps> caps = singletonSet(cap);
@ -415,11 +415,7 @@ public class SearchActionTests extends ESTestCase {
RollupJobConfig job = ConfigTestHelpers.getRollupJob("foo")
.setGroupConfig(ConfigTestHelpers.getGroupConfig()
.setDateHisto(new DateHistoGroupConfig.Builder()
.setInterval(new DateHistogramInterval("1d"))
.setField("foo")
.setTimeZone(DateTimeZone.UTC)
.build())
.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1d"), null, DateTimeZone.UTC.getID()))
.build())
.build();
Set<RollupJobCaps> caps = singletonSet(new RollupJobCaps(job));
@ -444,7 +440,7 @@ public class SearchActionTests extends ESTestCase {
public void testTwoMatchingJobs() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build())
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")))
.setHisto(null)
.setTerms(null);
job.setGroupConfig(group.build());
@ -494,7 +490,7 @@ public class SearchActionTests extends ESTestCase {
public void testTwoMatchingJobsOneBetter() {
RollupJobConfig.Builder job = ConfigTestHelpers.getRollupJob("foo");
GroupConfig.Builder group = ConfigTestHelpers.getGroupConfig();
group.setDateHisto(new DateHistoGroupConfig.Builder().setField("foo").setInterval(new DateHistogramInterval("1h")).build())
group.setDateHisto(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h")))
.setHisto(null)
.setTerms(null);
job.setGroupConfig(group.build());

View File

@ -5,9 +5,10 @@
*/
package org.elasticsearch.xpack.rollup.config;
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers;
import org.elasticsearch.xpack.core.rollup.job.DateHistoGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.GroupConfig;
import org.elasticsearch.xpack.core.rollup.job.HistogramGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.MetricConfig;
@ -170,29 +171,38 @@ public class ConfigTests extends ESTestCase {
}
public void testEmptyDateHistoField() {
DateHistoGroupConfig.Builder config = ConfigTestHelpers.getDateHisto();
config.setField(null);
Exception e = expectThrows(IllegalArgumentException.class, config::build);
assertThat(e.getMessage(), equalTo("Parameter [field] is mandatory."));
Exception e = expectThrows(IllegalArgumentException.class,
() -> new DateHistogramGroupConfig(null, DateHistogramInterval.HOUR));
assertThat(e.getMessage(), equalTo("Field must be a non-null, non-empty string"));
config.setField("");
e = expectThrows(IllegalArgumentException.class, config::build);
assertThat(e.getMessage(), equalTo("Parameter [field] is mandatory."));
e = expectThrows(IllegalArgumentException.class, () -> new DateHistogramGroupConfig("", DateHistogramInterval.HOUR));
assertThat(e.getMessage(), equalTo("Field must be a non-null, non-empty string"));
}
public void testEmptyDateHistoInterval() {
DateHistoGroupConfig.Builder config = ConfigTestHelpers.getDateHisto();
config.setField("foo");
config.setInterval(null);
Exception e = expectThrows(IllegalArgumentException.class, config::build);
assertThat(e.getMessage(), equalTo("Parameter [interval] is mandatory."));
Exception e = expectThrows(IllegalArgumentException.class, () -> new DateHistogramGroupConfig("foo", null));
assertThat(e.getMessage(), equalTo("Interval must be non-null"));
}
public void testNullTimeZone() {
DateHistoGroupConfig.Builder config = ConfigTestHelpers.getDateHisto();
config.setTimeZone(null);
DateHistoGroupConfig finalConfig = config.build();
assertThat(finalConfig.getTimeZone(), equalTo(DateTimeZone.UTC));
DateHistogramGroupConfig config = new DateHistogramGroupConfig("foo", DateHistogramInterval.HOUR, null, null);
assertThat(config.getTimeZone(), equalTo(DateTimeZone.UTC.getID()));
}
public void testEmptyTimeZone() {
DateHistogramGroupConfig config = new DateHistogramGroupConfig("foo", DateHistogramInterval.HOUR, null, "");
assertThat(config.getTimeZone(), equalTo(DateTimeZone.UTC.getID()));
}
public void testDefaultTimeZone() {
DateHistogramGroupConfig config = new DateHistogramGroupConfig("foo", DateHistogramInterval.HOUR);
assertThat(config.getTimeZone(), equalTo(DateTimeZone.UTC.getID()));
}
public void testUnkownTimeZone() {
Exception e = expectThrows(IllegalArgumentException.class,
() -> new DateHistogramGroupConfig("foo", DateHistogramInterval.HOUR, null, "FOO"));
assertThat(e.getMessage(), equalTo("The datetime zone id 'FOO' is not recognised"));
}
public void testEmptyHistoField() {

View File

@ -36,7 +36,7 @@ import org.elasticsearch.search.aggregations.metrics.max.MaxAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.sum.SumAggregationBuilder;
import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers;
import org.elasticsearch.xpack.core.rollup.RollupField;
import org.elasticsearch.xpack.core.rollup.job.DateHistoGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.GroupConfig;
import org.elasticsearch.xpack.core.rollup.job.HistogramGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.MetricConfig;
@ -97,10 +97,7 @@ public class IndexerUtilsTests extends AggregatorTestCase {
// Setup the composite agg
//TODO swap this over to DateHistoConfig.Builder once DateInterval is in
DateHistoGroupConfig dateHistoGroupConfig = new DateHistoGroupConfig.Builder()
.setField(timestampField)
.setInterval(DateHistogramInterval.days(1))
.build();
DateHistogramGroupConfig dateHistoGroupConfig = new DateHistogramGroupConfig(timestampField, DateHistogramInterval.DAY);
CompositeAggregationBuilder compositeBuilder =
new CompositeAggregationBuilder(RollupIndexer.AGGREGATION_NAME, dateHistoGroupConfig.toBuilders());
MetricConfig metricConfig = new MetricConfig("does_not_exist", singletonList("max"));

View File

@ -48,7 +48,7 @@ import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregati
import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers;
import org.elasticsearch.xpack.core.rollup.job.DateHistoGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig;
import org.elasticsearch.xpack.core.rollup.job.GroupConfig;
import org.elasticsearch.xpack.core.rollup.job.IndexerState;
import org.elasticsearch.xpack.core.rollup.job.MetricConfig;
@ -93,9 +93,7 @@ public class RollupIndexerIndexingTests extends AggregatorTestCase {
public void testSimpleDateHisto() throws Exception {
String rollupIndex = randomAlphaOfLength(10);
String field = "the_histo";
DateHistoGroupConfig dateHistoConfig = new DateHistoGroupConfig.Builder()
.setField(field)
.setInterval(new DateHistogramInterval("1ms")).build();
DateHistogramGroupConfig dateHistoConfig = new DateHistogramGroupConfig(field, new DateHistogramInterval("1ms"));
RollupJobConfig job = createJob(rollupIndex, new GroupConfig.Builder().setDateHisto(dateHistoConfig).build(),
Collections.emptyList());
final List<Map<String, Object>> dataset = new ArrayList<>();
@ -140,9 +138,7 @@ public class RollupIndexerIndexingTests extends AggregatorTestCase {
public void testDateHistoAndMetrics() throws Exception {
String rollupIndex = randomAlphaOfLength(10);
String field = "the_histo";
DateHistoGroupConfig dateHistoConfig = new DateHistoGroupConfig.Builder()
.setField(field)
.setInterval(new DateHistogramInterval("1h")).build();
DateHistogramGroupConfig dateHistoConfig = new DateHistogramGroupConfig(field, new DateHistogramInterval("1h"));
MetricConfig config = new MetricConfig("counter", Arrays.asList("avg", "sum", "max", "min"));
RollupJobConfig job = createJob(rollupIndex, new GroupConfig.Builder().setDateHisto(dateHistoConfig).build(),
Collections.singletonList(config));
@ -265,10 +261,8 @@ public class RollupIndexerIndexingTests extends AggregatorTestCase {
public void testSimpleDateHistoWithDelay() throws Exception {
String rollupIndex = randomAlphaOfLengthBetween(5, 10);
String field = "the_histo";
DateHistoGroupConfig dateHistoConfig = new DateHistoGroupConfig.Builder()
.setField(field)
.setDelay(new DateHistogramInterval("1h"))
.setInterval(new DateHistogramInterval("1m")).build();
DateHistogramGroupConfig dateHistoConfig =
new DateHistogramGroupConfig(field, new DateHistogramInterval("1m"), new DateHistogramInterval("1h"), null);
RollupJobConfig job = createJob(rollupIndex, new GroupConfig.Builder().setDateHisto(dateHistoConfig).build(),
Collections.emptyList());
final List<Map<String, Object>> dataset = new ArrayList<>();
@ -347,13 +341,10 @@ public class RollupIndexerIndexingTests extends AggregatorTestCase {
)
);
DateTimeZone timeZone = DateTimeZone.forOffsetHours(-3);
String timeZone = DateTimeZone.forOffsetHours(-3).getID();
String rollupIndex = randomAlphaOfLengthBetween(5, 10);
String field = "the_histo";
DateHistoGroupConfig dateHistoConfig = new DateHistoGroupConfig.Builder()
.setField(field)
.setTimeZone(timeZone)
.setInterval(new DateHistogramInterval("1d")).build();
DateHistogramGroupConfig dateHistoConfig = new DateHistogramGroupConfig(field, new DateHistogramInterval("1d"), null, timeZone);
RollupJobConfig job = createJob(rollupIndex, new GroupConfig.Builder().setDateHisto(dateHistoConfig).build(),
Collections.emptyList());
@ -414,9 +405,8 @@ public class RollupIndexerIndexingTests extends AggregatorTestCase {
String valueField = "the_avg";
String timeInterval = randomIntBetween(1, 10) + randomFrom("h", "m");
DateHistoGroupConfig dateHistoConfig = new DateHistoGroupConfig.Builder()
.setField(timestampField)
.setInterval(new DateHistogramInterval(timeInterval)).build();
DateHistogramGroupConfig dateHistoConfig =
new DateHistogramGroupConfig(timestampField, new DateHistogramInterval(timeInterval));
MetricConfig metricConfig = new MetricConfig(valueField, Collections.singletonList("avg"));
RollupJobConfig job = createJob(rollupIndex, new GroupConfig.Builder().setDateHisto(dateHistoConfig).build(),
Collections.singletonList(metricConfig));