diff --git a/docs/changelog/82944.yaml b/docs/changelog/82944.yaml new file mode 100644 index 0000000000000..1d283212d31b1 --- /dev/null +++ b/docs/changelog/82944.yaml @@ -0,0 +1,6 @@ +pr: 82944 +summary: [RollupV2]: make RollupAction available and improve some features +area: Rollup +type: feature +issues: + - 42720 diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupAction.java index 736edcc61ea26..9b2c2b41e1ec2 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupAction.java @@ -26,6 +26,8 @@ import java.util.Map; import java.util.Objects; +import static org.elasticsearch.action.ValidateActions.addValidationError; + public class RollupAction extends ActionType { public static final RollupAction INSTANCE = new RollupAction(); public static final String NAME = "indices:admin/xpack/rollup"; @@ -89,9 +91,23 @@ public RollupActionConfig getRollupConfig() { return rollupConfig; } + public void setRollupConfig(RollupActionConfig rollupConfig) { + this.rollupConfig = rollupConfig; + } + @Override public ActionRequestValidationException validate() { - return null; + ActionRequestValidationException validationException = null; + if (sourceIndex == null) { + validationException = addValidationError("rollup origin index is missing", null); + } + if (rollupIndex == null) { + validationException = addValidationError("rollup index is missing", validationException); + } + if (rollupConfig == null) { + validationException = addValidationError("rollup config is missing", validationException); + } + return validationException; } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupIndexerAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupIndexerAction.java index 29657fe364cf9..63f79affec87d 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupIndexerAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupIndexerAction.java @@ -184,6 +184,20 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); request.writeTo(out); } + + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + return new RollupShardTask( + id, + type, + action, + parentTaskId, + request.rollupRequest.getRollupIndex(), + request.rollupRequest.getRollupConfig(), + headers, + shardId() + ); + } } public static class ShardResponse extends BroadcastShardResponse { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupShardStatus.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupShardStatus.java new file mode 100644 index 0000000000000..e4af961885c6c --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupShardStatus.java @@ -0,0 +1,220 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.core.rollup.action; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.xcontent.ConstructingObjectParser; +import org.elasticsearch.xcontent.ParseField; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentParser; + +import java.io.IOException; +import java.time.Instant; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicLong; + +import static org.elasticsearch.xcontent.ConstructingObjectParser.constructorArg; + +public class RollupShardStatus implements Task.Status { + public static final String NAME = "rollup-index-shard"; + private static final ParseField SHARD_FIELD = new ParseField("shard"); + private static final ParseField STATUS_FIELD = new ParseField("status"); + private static final ParseField START_TIME_FIELD = new ParseField("start_time"); + private static final ParseField IN_NUM_DOCS_RECEIVED_FIELD = new ParseField("in_num_docs_received"); + private static final ParseField IN_NUM_DOCS_SKIPPED_FIELD = new ParseField("in_num_docs_skipped"); + private static final ParseField OUT_NUM_DOCS_SENT_FIELD = new ParseField("out_num_docs_sent"); + private static final ParseField OUT_NUM_DOCS_INDEXED_FIELD = new ParseField("out_num_docs_indexed"); + private static final ParseField OUT_NUM_DOCS_FAILED_FIELD = new ParseField("out_num_docs_failed"); + + private final ShardId shardId; + private final long rollupStart; + private Status status; + private AtomicLong numReceived = new AtomicLong(0); + private AtomicLong numSkip = new AtomicLong(0); + private AtomicLong numSent = new AtomicLong(0); + private AtomicLong numIndexed = new AtomicLong(0); + private AtomicLong numFailed = new AtomicLong(0); + + private static final ConstructingObjectParser PARSER; + static { + PARSER = new ConstructingObjectParser<>( + NAME, + args -> new RollupShardStatus( + ShardId.fromString((String) args[0]), + Status.valueOf((String) args[1]), + Instant.parse((String) args[2]).toEpochMilli(), + new AtomicLong((Long) args[3]), + new AtomicLong((Long) args[4]), + new AtomicLong((Long) args[5]), + new AtomicLong((Long) args[6]), + new AtomicLong((Long) args[7]) + ) + ); + + PARSER.declareString(constructorArg(), SHARD_FIELD); + PARSER.declareString(constructorArg(), STATUS_FIELD); + PARSER.declareString(constructorArg(), START_TIME_FIELD); + PARSER.declareLong(constructorArg(), IN_NUM_DOCS_RECEIVED_FIELD); + PARSER.declareLong(constructorArg(), IN_NUM_DOCS_SKIPPED_FIELD); + PARSER.declareLong(constructorArg(), OUT_NUM_DOCS_SENT_FIELD); + PARSER.declareLong(constructorArg(), OUT_NUM_DOCS_INDEXED_FIELD); + PARSER.declareLong(constructorArg(), OUT_NUM_DOCS_FAILED_FIELD); + } + + public RollupShardStatus(StreamInput in) throws IOException { + shardId = new ShardId(in); + status = in.readEnum(Status.class); + rollupStart = in.readLong(); + numReceived = new AtomicLong(in.readLong()); + numSkip = new AtomicLong(in.readLong()); + numSent = new AtomicLong(in.readLong()); + numIndexed = new AtomicLong(in.readLong()); + numFailed = new AtomicLong(in.readLong()); + } + + public RollupShardStatus( + ShardId shardId, + Status status, + long rollupStart, + AtomicLong numReceived, + AtomicLong numSkip, + AtomicLong numSent, + AtomicLong numIndexed, + AtomicLong numFailed + ) { + this.shardId = shardId; + this.status = status; + this.rollupStart = rollupStart; + this.numReceived = numReceived; + this.numSkip = numSkip; + this.numSent = numSent; + this.numIndexed = numIndexed; + this.numFailed = numFailed; + } + + public RollupShardStatus(ShardId shardId) { + status = Status.INIT; + this.shardId = shardId; + this.rollupStart = System.currentTimeMillis(); + } + + public void init(AtomicLong numReceived, AtomicLong numSkip, AtomicLong numSent, AtomicLong numIndexed, AtomicLong numFailed) { + this.numReceived = numReceived; + this.numSkip = numSkip; + this.numSent = numSent; + this.numIndexed = numIndexed; + this.numFailed = numFailed; + } + + public Status getStatus() { + return status; + } + + public void setStatus(Status status) { + this.status = status; + } + + public static RollupShardStatus fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(SHARD_FIELD.getPreferredName(), shardId); + builder.field(STATUS_FIELD.getPreferredName(), status); + builder.field(START_TIME_FIELD.getPreferredName(), Instant.ofEpochMilli(rollupStart).toString()); + builder.field(IN_NUM_DOCS_RECEIVED_FIELD.getPreferredName(), numReceived.get()); + builder.field(IN_NUM_DOCS_SKIPPED_FIELD.getPreferredName(), numSkip.get()); + builder.field(OUT_NUM_DOCS_SENT_FIELD.getPreferredName(), numSent.get()); + builder.field(OUT_NUM_DOCS_INDEXED_FIELD.getPreferredName(), numIndexed.get()); + builder.field(OUT_NUM_DOCS_FAILED_FIELD.getPreferredName(), numFailed.get()); + builder.endObject(); + return builder; + } + + @Override + public String getWriteableName() { + return NAME; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + shardId.writeTo(out); + out.writeEnum(status); + out.writeLong(rollupStart); + out.writeLong(numReceived.get()); + out.writeLong(numSkip.get()); + out.writeLong(numSent.get()); + out.writeLong(numIndexed.get()); + out.writeLong(numFailed.get()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RollupShardStatus that = (RollupShardStatus) o; + return rollupStart == that.rollupStart + && Objects.equals(shardId.getIndexName(), that.shardId.getIndexName()) + && Objects.equals(shardId.id(), that.shardId.id()) + && status == that.status + && Objects.equals(numReceived.get(), that.numReceived.get()) + && Objects.equals(numSkip.get(), that.numSkip.get()) + && Objects.equals(numSent.get(), that.numSent.get()) + && Objects.equals(numIndexed.get(), that.numIndexed.get()) + && Objects.equals(numFailed.get(), that.numFailed.get()); + } + + @Override + public int hashCode() { + return Objects.hash( + shardId.getIndexName(), + shardId.id(), + rollupStart, + status, + numReceived.get(), + numSkip.get(), + numSent.get(), + numIndexed.get(), + numFailed.get() + ); + } + + @Override + public String toString() { + return Strings.toString(this); + } + + public enum Status { + INIT, + ROLLING, + STOP, + ABORT + } + + public void setNumSent(AtomicLong numSent) { + this.numSent = numSent; + } + + public void setNumIndexed(AtomicLong numIndexed) { + this.numIndexed = numIndexed; + } + + public void setNumFailed(AtomicLong numFailed) { + this.numFailed = numFailed; + } +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupShardTask.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupShardTask.java new file mode 100644 index 0000000000000..2d3b0eb80ddbd --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupShardTask.java @@ -0,0 +1,56 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.core.rollup.action; + +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.xpack.core.rollup.RollupActionConfig; +import org.elasticsearch.xpack.core.rollup.RollupField; + +import java.util.Map; + +public class RollupShardTask extends CancellableTask { + private String rollupIndex; + private RollupActionConfig config; + private volatile RollupShardStatus status; + + public RollupShardTask( + long id, + String type, + String action, + TaskId parentTask, + String rollupIndex, + RollupActionConfig config, + Map headers, + ShardId shardId + ) { + super(id, type, action, RollupField.NAME + "_" + rollupIndex + "[" + shardId.id() + "]", parentTask, headers); + this.rollupIndex = rollupIndex; + this.config = config; + this.status = new RollupShardStatus(shardId); + } + + public String getRollupIndex() { + return rollupIndex; + } + + public RollupActionConfig config() { + return config; + } + + @Override + public Status getStatus() { + return status; + } + + @Override + public void onCancelled() { + status.setStatus(RollupShardStatus.Status.ABORT); + } +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/TermsGroupConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/TermsGroupConfig.java index 93f1bbffa87ac..6528a26052337 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/TermsGroupConfig.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/TermsGroupConfig.java @@ -14,6 +14,7 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.index.mapper.KeywordFieldMapper; import org.elasticsearch.index.mapper.TextFieldMapper; +import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper; import org.elasticsearch.xcontent.ConstructingObjectParser; import org.elasticsearch.xcontent.ParseField; import org.elasticsearch.xcontent.ToXContentObject; @@ -85,7 +86,9 @@ public void validateMappings( Map fieldCaps = fieldCapsResponse.get(field); if (fieldCaps != null && fieldCaps.isEmpty() == false) { fieldCaps.forEach((key, value) -> { - if (key.equals(KeywordFieldMapper.CONTENT_TYPE) || key.equals(TextFieldMapper.CONTENT_TYPE)) { + if (key.equals(KeywordFieldMapper.CONTENT_TYPE) + || key.equals(TextFieldMapper.CONTENT_TYPE) + || key.equals(TimeSeriesIdFieldMapper.CONTENT_TYPE)) { if (value.isAggregatable() == false) { validationException.addValidationError( "The field [" + field + "] must be aggregatable across all indices, " + "but is not." diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/action/RollupShardStatusSerializingTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/action/RollupShardStatusSerializingTests.java new file mode 100644 index 0000000000000..c8e42bf9785e2 --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/action/RollupShardStatusSerializingTests.java @@ -0,0 +1,43 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.core.rollup.action; + +import org.elasticsearch.common.io.stream.Writeable.Reader; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.test.AbstractSerializingTestCase; +import org.elasticsearch.xcontent.XContentParser; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicLong; + +public class RollupShardStatusSerializingTests extends AbstractSerializingTestCase { + @Override + protected RollupShardStatus doParseInstance(XContentParser parser) throws IOException { + return RollupShardStatus.fromXContent(parser); + } + + @Override + protected Reader instanceReader() { + return RollupShardStatus::new; + } + + @Override + protected RollupShardStatus createTestInstance() { + RollupShardStatus rollupShardStatus = new RollupShardStatus( + new ShardId(randomAlphaOfLength(5), randomAlphaOfLength(5), randomInt(5)) + ); + rollupShardStatus.init( + new AtomicLong(randomNonNegativeLong()), + new AtomicLong(randomNonNegativeLong()), + new AtomicLong(randomNonNegativeLong()), + new AtomicLong(randomNonNegativeLong()), + new AtomicLong(randomNonNegativeLong()) + ); + return rollupShardStatus; + } +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/TermsGroupConfigSerializingTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/TermsGroupConfigSerializingTests.java index 2d373359ab976..4e56b73147c12 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/TermsGroupConfigSerializingTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/TermsGroupConfigSerializingTests.java @@ -9,6 +9,7 @@ import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.fieldcaps.FieldCapabilities; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper; import org.elasticsearch.test.AbstractSerializingTestCase; import org.elasticsearch.xcontent.XContentParser; @@ -123,6 +124,23 @@ public void testValidateMatchingField() { } } + public void testVaildateTsidMatchingField() { + ActionRequestValidationException e = new ActionRequestValidationException(); + Map> responseMap = new HashMap<>(); + String type = TimeSeriesIdFieldMapper.NAME; + + // Have to mock fieldcaps because the ctor's aren't public... + FieldCapabilities fieldCaps = mock(FieldCapabilities.class); + when(fieldCaps.isAggregatable()).thenReturn(true); + responseMap.put(TimeSeriesIdFieldMapper.NAME, Collections.singletonMap(type, fieldCaps)); + + TermsGroupConfig config = new TermsGroupConfig(TimeSeriesIdFieldMapper.NAME); + config.validateMappings(responseMap, e); + if (e.validationErrors().size() != 0) { + fail(e.getMessage()); + } + } + private String getRandomType() { int n = randomIntBetween(0, 8); if (n == 0) { diff --git a/x-pack/plugin/rollup/build.gradle b/x-pack/plugin/rollup/build.gradle index dbc09c44f43f0..01c981a23b8ac 100644 --- a/x-pack/plugin/rollup/build.gradle +++ b/x-pack/plugin/rollup/build.gradle @@ -1,6 +1,7 @@ import org.elasticsearch.gradle.internal.info.BuildParams apply plugin: 'elasticsearch.internal-es-plugin' +apply plugin: 'elasticsearch.internal-cluster-test' esplugin { name 'x-pack-rollup' description 'Elasticsearch Expanded Pack Plugin - Rollup' @@ -15,13 +16,14 @@ dependencies { compileOnly project(path: xpackModule('core')) compileOnly project(path: xpackModule('analytics')) compileOnly project(path: xpackModule('mapper-aggregate-metric')) + compileOnly project(path: xpackModule('ilm')) compileOnly project(':modules:data-streams') testImplementation(testArtifact(project(xpackModule('core')))) } addQaCheckDependencies() -tasks.named("test").configure { +tasks.named("internalClusterTest").configure { if (BuildParams.isSnapshotBuild() == false) { systemProperty 'es.rollup_v2_feature_flag_enabled', 'true' } diff --git a/x-pack/plugin/rollup/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/rollup/10_basic.yml b/x-pack/plugin/rollup/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/rollup/10_basic.yml index 22b3da13b5e72..f81dabc767683 100644 --- a/x-pack/plugin/rollup/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/rollup/10_basic.yml +++ b/x-pack/plugin/rollup/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/rollup/10_basic.yml @@ -69,7 +69,7 @@ setup: "metrics": [ { "field": "price", - "metrics": ["max", "sum", "avg"] + "metrics": ["max", "min", "sum", "avg", "value_count"] } ] } @@ -85,3 +85,33 @@ setup: index: rollup_docs body: { query: { match_all: {} } } - length: { hits.hits: 3 } + +--- +"Rollup not found index": + - skip: + version: " - 7.99.99" + reason: "rolling up an index directly is only supported in 8.0+" + - do: + catch: /Invalid index name \[no_docs\], rollup origin index metadata missing/ + rollup.rollup: + index: no_docs + rollup_index: rollup_docs + body: > + { + "groups" : { + "date_histogram": { + "field": "timestamp", + "calendar_interval": "1h" + }, + "terms": { + "fields": ["color"] + } + }, + "metrics": [ + { + "field": "price", + "metrics": ["max", "min", "sum", "avg", "value_count"] + } + ] + } + diff --git a/x-pack/plugin/rollup/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/rollup/20_datastream.yml b/x-pack/plugin/rollup/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/rollup/20_datastream.yml new file mode 100644 index 0000000000000..d20dd743237c3 --- /dev/null +++ b/x-pack/plugin/rollup/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/rollup/20_datastream.yml @@ -0,0 +1,101 @@ +setup: + - skip: + features: allowed_warnings + - do: + allowed_warnings: + - "index template [my-template] has index patterns [simple-data-stream] matching patterns from existing older templates [global] with patterns (global => [*]); this template [my-template1] will take precedence during new index creation" + indices.put_index_template: + name: my-template + body: + index_patterns: [ simple-data-stream ] + template: + settings: + index.number_of_replicas: 0 + mappings: + properties: + timestamp: + type: date + color: + type: keyword + price: + type: integer + data_stream: { } + + - do: + indices.create_data_stream: + name: simple-data-stream + + - do: + bulk: + refresh: true + body: + - create: + _index: simple-data-stream + _id: 1 + - '@timestamp': "2020-01-01T05:10:00Z" + color: "blue" + price: 10 + - create: + _index: simple-data-stream + _id: 2 + - '@timestamp': "2020-01-01T05:30:00Z" + color: "blue" + price: 20 + - create: + _index: simple-data-stream + _id: 3 + - '@timestamp': "2020-01-01T06:10:00Z" + color: "red" + price: 30 + - create: + _index: simple-data-stream + _id: 4 + - '@timestamp': "2020-01-01T06:30:00Z" + price: 40 + color: "green" + +--- +"Rollup index": + - skip: + version: " - 7.99.99" + reason: "rolling up an index directly is only supported in 8.0+" + + - do: + indices.rollover: + alias: simple-data-stream + - set: { old_index: idx0name } + + - do: + rollup.rollup: + index: $idx0name + rollup_index: rollup_docs + body: > + { + "groups" : { + "date_histogram": { + "field": "@timestamp", + "calendar_interval": "1h" + }, + "terms": { + "fields": ["color"] + } + }, + "metrics": [ + { + "field": "price", + "metrics": ["max", "min", "sum", "avg", "value_count"] + } + ] + } + - is_true: acknowledged + + - do: + indices.forcemerge: + index: rollup_docs + max_num_segments: 1 + + - do: + search: + index: rollup_docs + body: { query: { match_all: { } } } + - length: { hits.hits: 3 } diff --git a/x-pack/plugin/rollup/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/rollup/30_tsdb.yml b/x-pack/plugin/rollup/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/rollup/30_tsdb.yml new file mode 100644 index 0000000000000..0fa0cf562afcd --- /dev/null +++ b/x-pack/plugin/rollup/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/rollup/30_tsdb.yml @@ -0,0 +1,101 @@ +setup: + - skip: + version: " - 8.0.99" + reason: introduced in 8.1.0 + + - do: + indices.create: + index: test + body: + settings: + index: + mode: time_series + routing_path: [metricset, k8s.pod.uid] + time_series: + start_time: 2021-04-28T00:00:00Z + end_time: 2021-04-29T00:00:00Z + mappings: + properties: + "@timestamp": + type: date + metricset: + type: keyword + time_series_dimension: true + k8s: + properties: + pod: + properties: + uid: + type: keyword + time_series_dimension: true + name: + type: keyword + ip: + type: ip + network: + properties: + tx: + type: long + rx: + type: long + - do: + bulk: + refresh: true + index: test + body: + - '{"index": {}}' + - '{"@timestamp": "2021-04-28T18:50:04.467Z", "metricset": "pod", "k8s": {"pod": {"name": "cat", "uid":"947e4ced-1786-4e53-9e0c-5c447e959507", "ip": "10.10.55.1", "network": {"tx": 2001818691, "rx": 802133794}}}}' + - '{"index": {}}' + - '{"@timestamp": "2021-04-28T18:50:24.467Z", "metricset": "pod", "k8s": {"pod": {"name": "cat", "uid":"947e4ced-1786-4e53-9e0c-5c447e959507", "ip": "10.10.55.1", "network": {"tx": 2005177954, "rx": 801479970}}}}' + - '{"index": {}}' + - '{"@timestamp": "2021-04-28T18:50:44.467Z", "metricset": "pod", "k8s": {"pod": {"name": "cat", "uid":"947e4ced-1786-4e53-9e0c-5c447e959507", "ip": "10.10.55.1", "network": {"tx": 2006223737, "rx": 802337279}}}}' + - '{"index": {}}' + - '{"@timestamp": "2021-04-28T18:51:04.467Z", "metricset": "pod", "k8s": {"pod": {"name": "cat", "uid":"947e4ced-1786-4e53-9e0c-5c447e959507", "ip": "10.10.55.2", "network": {"tx": 2012916202, "rx": 803685721}}}}' + - '{"index": {}}' + - '{"@timestamp": "2021-04-28T18:50:03.142Z", "metricset": "pod", "k8s": {"pod": {"name": "dog", "uid":"df3145b3-0563-4d3b-a0f7-897eb2876ea9", "ip": "10.10.55.3", "network": {"tx": 1434521831, "rx": 530575198}}}}' + - '{"index": {}}' + - '{"@timestamp": "2021-04-28T18:50:23.142Z", "metricset": "pod", "k8s": {"pod": {"name": "dog", "uid":"df3145b3-0563-4d3b-a0f7-897eb2876ea9", "ip": "10.10.55.3", "network": {"tx": 1434577921, "rx": 530600088}}}}' + - '{"index": {}}' + - '{"@timestamp": "2021-04-28T18:50:53.142Z", "metricset": "pod", "k8s": {"pod": {"name": "dog", "uid":"df3145b3-0563-4d3b-a0f7-897eb2876ea9", "ip": "10.10.55.3", "network": {"tx": 1434587694, "rx": 530604797}}}}' + - '{"index": {}}' + - '{"@timestamp": "2021-04-28T18:51:03.142Z", "metricset": "pod", "k8s": {"pod": {"name": "dog", "uid":"df3145b3-0563-4d3b-a0f7-897eb2876ea9", "ip": "10.10.55.3", "network": {"tx": 1434595272, "rx": 530605511}}}}' + +--- +"Rollup index": + - skip: + version: " - 7.99.99" + reason: "rolling up an index directly is only supported in 8.0+" + - do: + rollup.rollup: + index: test + rollup_index: rollup_test + body: > + { + "groups" : { + "date_histogram": { + "field": "@timestamp", + "calendar_interval": "1h" + }, + "terms": { + "fields": ["_tsid"] + } + }, + "metrics": [ + { + "field": "k8s.pod.network.*", + "metrics": ["max", "min", "sum", "avg", "value_count"] + } + ] + } + - is_true: acknowledged + + - do: + indices.forcemerge: + index: rollup_test + max_num_segments: 1 + + - do: + search: + index: rollup_test + body: { query: { match_all: {} } } + - length: { hits.hits: 2 } diff --git a/x-pack/plugin/rollup/src/internalClusterTest/java/org/elasticsearch/xpack/rollup/v2/RollupActionIT.java b/x-pack/plugin/rollup/src/internalClusterTest/java/org/elasticsearch/xpack/rollup/v2/RollupActionIT.java new file mode 100644 index 0000000000000..442ae4d943728 --- /dev/null +++ b/x-pack/plugin/rollup/src/internalClusterTest/java/org/elasticsearch/xpack/rollup/v2/RollupActionIT.java @@ -0,0 +1,530 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ +package org.elasticsearch.xpack.rollup.v2; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ResourceAlreadyExistsException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.indices.alias.Alias; +import org.elasticsearch.action.admin.indices.get.GetIndexResponse; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.time.DateUtils; +import org.elasticsearch.common.util.CancellableThreads.ExecutionCancelledException; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexMode; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.indices.InvalidIndexNameException; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.xcontent.XContentFactory; +import org.elasticsearch.xpack.core.ilm.LifecyclePolicy; +import org.elasticsearch.xpack.core.ilm.LifecycleSettings; +import org.elasticsearch.xpack.core.ilm.action.PutLifecycleAction; +import org.elasticsearch.xpack.core.rollup.RollupActionConfig; +import org.elasticsearch.xpack.core.rollup.RollupActionDateHistogramGroupConfig; +import org.elasticsearch.xpack.core.rollup.RollupActionGroupConfig; +import org.elasticsearch.xpack.core.rollup.action.RollupAction; +import org.elasticsearch.xpack.core.rollup.action.RollupShardStatus; +import org.elasticsearch.xpack.core.rollup.action.RollupShardStatus.Status; +import org.elasticsearch.xpack.core.rollup.job.HistogramGroupConfig; +import org.elasticsearch.xpack.core.rollup.job.MetricConfig; +import org.elasticsearch.xpack.core.rollup.job.TermsGroupConfig; +import org.elasticsearch.xpack.ilm.IndexLifecycle; +import org.elasticsearch.xpack.rollup.v2.indexer.UnSortedRollupShardIndexer; +import org.junit.Before; + +import java.io.IOException; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Locale; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.xpack.core.ilm.LifecycleSettings.LIFECYCLE_HISTORY_INDEX_ENABLED; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class RollupActionIT extends RollupIntegTestCase { + + @Override + protected Collection> getPlugins() { + List> plugins = new ArrayList<>(super.getPlugins()); + plugins.add(IndexLifecycle.class); + return plugins; + } + + @Override + protected Settings nodeSettings() { + return Settings.builder().put(LIFECYCLE_HISTORY_INDEX_ENABLED, false).build(); + } + + @Before + public void init() { + LifecyclePolicy lifecyclePolicy = new LifecyclePolicy("test", Collections.emptyMap()); + PutLifecycleAction.Request putLifecycleRequest = new PutLifecycleAction.Request(lifecyclePolicy); + assertAcked(client().execute(PutLifecycleAction.INSTANCE, putLifecycleRequest).actionGet()); + + client().admin() + .indices() + .prepareCreate(index) + .setSettings( + Settings.builder() + .put("index.number_of_shards", randomIntBetween(1, 2)) + .put(LifecycleSettings.LIFECYCLE_NAME, "test") + .put(LifecycleSettings.LIFECYCLE_INDEXING_COMPLETE, "true") + .build() + ) + .setMapping( + "date_1", + "type=date", + "numeric_1", + "type=double", + "numeric_2", + "type=float", + "numeric_nonaggregatable", + "type=double,doc_values=false", + "categorical_1", + "type=keyword", + "categorical_2", + "type=keyword", + "@timestamp", + "type=date" + ) + .get(); + } + + public void testUnSortedRollupShardIndexer() throws IOException { + // create rollup config and index documents into source index + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("categorical_1", randomAlphaOfLength(1)) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + bulkIndex(sourceSupplier); + + IndicesService indexServices = getInstanceFromNode(IndicesService.class); + Index srcIndex = resolveIndex(index); + IndexService indexService = indexServices.indexServiceSafe(srcIndex); + IndexShard shard = indexService.getShard(0); + + // re-use source index as temp index for test + UnSortedRollupShardIndexer indexer = new UnSortedRollupShardIndexer( + new RollupShardStatus(shard.shardId()), + client(), + indexService, + shard.shardId(), + config, + rollupIndex, + 2 + ); + assertThat(indexer.status.getStatus(), equalTo(Status.ROLLING)); + indexer.execute(); + assertThat(indexer.tmpFilesDeleted, equalTo(indexer.tmpFiles)); + if (indexService.shardIds().size() == 1) { + assertThat(indexer.numReceived.get(), equalTo((long) docCount)); + } + assertThat(indexer.numSkip.get(), equalTo(0L)); + assertThat(indexer.numSent.get(), equalTo(indexer.numIndexed.get())); + assertThat(indexer.numFailed.get(), equalTo(0L)); + assertThat(indexer.status.getStatus(), equalTo(Status.STOP)); + } + + public void testCannotRollupToExistingIndex() throws Exception { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("categorical_1", randomAlphaOfLength(1)) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + bulkIndex(sourceSupplier); + rollup(index, rollupIndex, config); + assertRollupIndex(config, index, rollupIndex); + ElasticsearchException exception = expectThrows(ElasticsearchException.class, () -> rollup(index, rollupIndex, config)); + assertThat(exception.getMessage(), containsString("Invalid index name [" + rollupIndex + "], rollup index already exists")); + } + + public void testCannotRollupToExistingAlias() { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + String aliasName = randomAlphaOfLength(6).toLowerCase(Locale.ROOT); + client().admin() + .indices() + .prepareCreate(randomAlphaOfLength(6).toLowerCase(Locale.ROOT)) + .setSettings(Settings.builder().put("index.number_of_shards", 1).build()) + .addAlias(new Alias(aliasName)) + .get(); + + InvalidIndexNameException exception = expectThrows(InvalidIndexNameException.class, () -> rollup(index, aliasName, config)); + assertThat(exception.getMessage(), equalTo("Invalid index name [" + aliasName + "], rollup index already exists as alias")); + } + + public void testCannotRollupToExistingDataStream() throws Exception { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + String datsStreamName = createDataStream(); + InvalidIndexNameException exception = expectThrows(InvalidIndexNameException.class, () -> rollup(index, datsStreamName, config)); + assertThat( + exception.getMessage(), + equalTo("Invalid index name [" + datsStreamName + "], rollup index already exists as data stream") + ); + } + + public void testTemporaryIndexCannotBeCreatedAlreadyExists() { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + assertTrue(client().admin().indices().prepareCreate(".rolluptmp-" + rollupIndex).get().isAcknowledged()); + Exception exception = expectThrows(ElasticsearchException.class, () -> rollup(index, rollupIndex, config)); + assertThat(exception.getMessage(), containsString("already exists")); + } + + public void testCannotRollupWhileOtherRollupInProgress() throws Exception { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("categorical_1", randomAlphaOfLength(1)) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + bulkIndex(sourceSupplier); + client().execute(RollupAction.INSTANCE, new RollupAction.Request(index, rollupIndex, config), ActionListener.wrap(() -> {})); + ResourceAlreadyExistsException exception = expectThrows( + ResourceAlreadyExistsException.class, + () -> rollup(index, rollupIndex, config) + ); + assertThat(exception.getMessage(), containsString(".rolluptmp-" + rollupIndex)); + } + + public void testTermsGrouping() throws IOException { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("categorical_1", randomAlphaOfLength(1)) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + bulkIndex(sourceSupplier); + rollup(index, rollupIndex, config); + assertRollupIndex(config, index, rollupIndex); + + GetIndexResponse indexSettingsResp = client().admin().indices().prepareGetIndex().addIndices(rollupIndex).get(); + assertEquals(indexSettingsResp.getSetting(rollupIndex, LifecycleSettings.LIFECYCLE_NAME), "test"); + assertEquals(indexSettingsResp.getSetting(rollupIndex, LifecycleSettings.LIFECYCLE_INDEXING_COMPLETE), "true"); + } + + public void testHistogramGrouping() throws IOException { + long interval = randomLongBetween(1, 1000); + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("numeric_1", randomDoubleBetween(0.0, 10000.0, true)) + .field("numeric_2", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, new HistogramGroupConfig(interval, "numeric_1"), null), + Collections.singletonList(new MetricConfig("numeric_2", Collections.singletonList("max"))) + ); + bulkIndex(sourceSupplier); + rollup(index, rollupIndex, config); + assertRollupIndex(config, index, rollupIndex); + } + + public void testMaxMetric() throws IOException { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, null), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + bulkIndex(sourceSupplier); + rollup(index, rollupIndex, config); + assertRollupIndex(config, index, rollupIndex); + } + + public void testMinMetric() throws IOException { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, null), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("min"))) + ); + bulkIndex(sourceSupplier); + rollup(index, rollupIndex, config); + assertRollupIndex(config, index, rollupIndex); + } + + public void testValueCountMetric() throws IOException { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, null), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("value_count"))) + ); + bulkIndex(sourceSupplier); + rollup(index, rollupIndex, config); + assertRollupIndex(config, index, rollupIndex); + } + + public void testAvgMetric() throws IOException { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + // Use integers to ensure that avg is comparable between rollup and original + .field("numeric_1", randomInt()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, null), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("avg"))) + ); + bulkIndex(sourceSupplier); + rollup(index, rollupIndex, config); + assertRollupIndex(config, index, rollupIndex); + } + + public void testValidationCheck() throws IOException { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + // use integers to ensure that avg is comparable between rollup and original + .field("numeric_nonaggregatable", randomInt()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, null), + Collections.singletonList(new MetricConfig("numeric_nonaggregatable", Collections.singletonList("avg"))) + ); + bulkIndex(sourceSupplier); + Exception e = expectThrows(Exception.class, () -> rollup(index, rollupIndex, config)); + assertThat(e.getMessage(), containsString("The field [numeric_nonaggregatable] must be aggregatable")); + } + + public void testRollupDatastream() throws Exception { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig(timestampFieldName); + String dataStreamName = createDataStream(); + + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field(timestampFieldName, randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, null), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("value_count"))) + ); + bulkIndex(dataStreamName, sourceSupplier); + + String oldIndexName = rollover(dataStreamName).getOldIndex(); + String rollupIndexName = ".rollup-" + oldIndexName; + rollup(oldIndexName, rollupIndexName, config); + assertRollupIndex(config, oldIndexName, rollupIndexName); + rollup(oldIndexName, rollupIndexName + "-2", config); + assertRollupIndex(config, oldIndexName, rollupIndexName + "-2"); + } + + public void testWildCardRollup() throws IOException { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("categorical_1", randomAlphaOfLength(1)) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("cate*")), + Collections.singletonList(new MetricConfig("num*_1", Collections.singletonList("max"))) + ); + bulkIndex(sourceSupplier); + rollup(index, rollupIndex, config); + + RollupActionConfig newConfig = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + assertRollupIndex(newConfig, index, rollupIndex); + } + + public void testEmptyTermsAndMetricsRollup() throws IOException { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("categorical_1", randomAlphaOfLength(1)) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_2")), + Collections.singletonList(new MetricConfig("numeric_2", Collections.singletonList("max"))) + ); + bulkIndex(sourceSupplier); + rollup(index, rollupIndex, config); + assertRollupIndex(config, index, rollupIndex); + } + + public void testEmptyDateHistogram() throws IOException { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("@timestamp", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("categorical_1", randomAlphaOfLength(1)) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_2")), + Collections.singletonList(new MetricConfig("numeric_2", Collections.singletonList("max"))) + ); + bulkIndex(sourceSupplier); + rollup(index, rollupIndex, config); + assertRollupIndex(config, index, rollupIndex); + } + + public void testRollupToTimeSeriesIndex() throws IOException { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("@timestamp"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("@timestamp", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("categorical_1", randomAlphaOfLength(1)) + .field("categorical_2", randomAlphaOfLength(1)) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1", "categorical_2")), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + bulkIndex(sourceSupplier); + rollup(index, rollupIndex, config); + assertRollupIndex(config, index, rollupIndex); + + GetIndexResponse indexSettingsResp = client().admin().indices().prepareGetIndex().addIndices(rollupIndex).get(); + assertEquals( + indexSettingsResp.getSetting(rollupIndex, IndexSettings.MODE.getKey()), + IndexMode.TIME_SERIES.name().toLowerCase(Locale.ROOT) + ); + assertEquals( + indexSettingsResp.getSetting(rollupIndex, IndexMetadata.INDEX_ROUTING_PATH.getKey()), + "[categorical_1, categorical_2]" + ); + assertEquals( + indexSettingsResp.getSetting(rollupIndex, IndexSettings.TIME_SERIES_START_TIME.getKey()), + Instant.ofEpochMilli(1).toString() + ); + assertEquals( + indexSettingsResp.getSetting(rollupIndex, IndexSettings.TIME_SERIES_END_TIME.getKey()), + Instant.ofEpochMilli(DateUtils.MAX_MILLIS_BEFORE_9999 - 1).toString() + ); + } + + public void testRollupInvalidRequest() { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, null), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + { + IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> rollup(null, rollupIndex, config)); + assertThat(exception.getMessage(), containsString("rollup origin index is missing;")); + } + { + IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> rollup(index, null, config)); + assertThat(exception.getMessage(), containsString("rollup index is missing;")); + } + { + InvalidIndexNameException exception = expectThrows( + InvalidIndexNameException.class, + () -> rollup("no_index", rollupIndex, config) + ); + assertThat(exception.getMessage(), containsString("Invalid index name [no_index], rollup origin index metadata missing")); + } + + { + IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> rollup(index, rollupIndex, null)); + assertThat(exception.getMessage(), containsString("rollup config is missing;")); + } + } + + public void testCancelRollupIndexer() throws IOException { + // create rollup config and index documents into source index + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("categorical_1", randomAlphaOfLength(1)) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + bulkIndex(sourceSupplier); + + IndicesService indexServices = getInstanceFromNode(IndicesService.class); + Index srcIndex = resolveIndex(index); + IndexService indexService = indexServices.indexServiceSafe(srcIndex); + IndexShard shard = indexService.getShard(0); + + // re-use source index as temp index for test + UnSortedRollupShardIndexer indexer = new UnSortedRollupShardIndexer( + new RollupShardStatus(shard.shardId()), + client(), + indexService, + shard.shardId(), + config, + rollupIndex, + 2 + ); + indexer.status.setStatus(Status.ABORT); + { + ExecutionCancelledException exception = expectThrows(ExecutionCancelledException.class, () -> indexer.execute()); + assertThat(exception.getMessage(), containsString("rollup cancelled")); + } + } +} diff --git a/x-pack/plugin/rollup/src/internalClusterTest/java/org/elasticsearch/xpack/rollup/v2/RollupIntegTestCase.java b/x-pack/plugin/rollup/src/internalClusterTest/java/org/elasticsearch/xpack/rollup/v2/RollupIntegTestCase.java new file mode 100644 index 0000000000000..f69cd20755c9f --- /dev/null +++ b/x-pack/plugin/rollup/src/internalClusterTest/java/org/elasticsearch/xpack/rollup/v2/RollupIntegTestCase.java @@ -0,0 +1,322 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.rollup.v2; + +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.admin.indices.get.GetIndexResponse; +import org.elasticsearch.action.admin.indices.rollover.RolloverRequest; +import org.elasticsearch.action.admin.indices.rollover.RolloverResponse; +import org.elasticsearch.action.admin.indices.template.put.PutComposableIndexTemplateAction; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.datastreams.CreateDataStreamAction; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; +import org.elasticsearch.cluster.metadata.Template; +import org.elasticsearch.common.compress.CompressedXContent; +import org.elasticsearch.common.time.DateFormatter; +import org.elasticsearch.datastreams.DataStreamsPlugin; +import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.composite.CompositeValuesSourceBuilder; +import org.elasticsearch.search.aggregations.bucket.composite.DateHistogramValuesSourceBuilder; +import org.elasticsearch.search.aggregations.bucket.composite.HistogramValuesSourceBuilder; +import org.elasticsearch.search.aggregations.bucket.composite.InternalComposite; +import org.elasticsearch.search.aggregations.bucket.composite.TermsValuesSourceBuilder; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.MinAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; +import org.elasticsearch.test.ESSingleNodeTestCase; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xpack.aggregatemetric.AggregateMetricMapperPlugin; +import org.elasticsearch.xpack.analytics.AnalyticsPlugin; +import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin; +import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers; +import org.elasticsearch.xpack.core.rollup.RollupActionConfig; +import org.elasticsearch.xpack.core.rollup.RollupActionDateHistogramGroupConfig; +import org.elasticsearch.xpack.core.rollup.action.RollupAction; +import org.elasticsearch.xpack.core.rollup.job.HistogramGroupConfig; +import org.elasticsearch.xpack.core.rollup.job.MetricConfig; +import org.elasticsearch.xpack.core.rollup.job.TermsGroupConfig; +import org.elasticsearch.xpack.rollup.Rollup; +import org.junit.Before; + +import java.io.IOException; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutionException; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; +import static org.hamcrest.Matchers.equalTo; + +public abstract class RollupIntegTestCase extends ESSingleNodeTestCase { + private static final DateFormatter DATE_FORMATTER = DateFormatter.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSXXX"); + protected String index; + protected String rollupIndex; + protected long startTime; + protected int docCount; + + protected String timestampFieldName = "@timestamp"; + private final Set createdDataStreams = new HashSet<>(); + + @Override + protected Collection> getPlugins() { + return List.of( + LocalStateCompositeXPackPlugin.class, + Rollup.class, + AnalyticsPlugin.class, + AggregateMetricMapperPlugin.class, + DataStreamsPlugin.class + ); + } + + @Before + public void setup() { + index = randomAlphaOfLength(5).toLowerCase(Locale.ROOT); + rollupIndex = randomAlphaOfLength(6).toLowerCase(Locale.ROOT); + startTime = randomLongBetween(946769284000L, 1607470084000L); // random date between 2000-2020 + docCount = randomIntBetween(10, 1000); + } + + protected RollupActionDateHistogramGroupConfig randomRollupActionDateHistogramGroupConfig(String field) { + RollupActionDateHistogramGroupConfig randomConfig = ConfigTestHelpers.randomRollupActionDateHistogramGroupConfig(random()); + if (randomConfig instanceof RollupActionDateHistogramGroupConfig.FixedInterval) { + return new RollupActionDateHistogramGroupConfig.FixedInterval(field, randomConfig.getInterval(), randomConfig.getTimeZone()); + } + if (randomConfig instanceof RollupActionDateHistogramGroupConfig.CalendarInterval) { + return new RollupActionDateHistogramGroupConfig.CalendarInterval(field, randomConfig.getInterval(), randomConfig.getTimeZone()); + } + throw new IllegalStateException("invalid RollupActionDateHistogramGroupConfig class type"); + } + + protected String randomDateForInterval(DateHistogramInterval interval) { + final long maxNumBuckets = 10; + final long endTime = startTime + maxNumBuckets * interval.estimateMillis(); + return DATE_FORMATTER.formatMillis(randomLongBetween(startTime, endTime)); + } + + protected void bulkIndex(SourceSupplier sourceSupplier) throws IOException { + bulkIndex(index, sourceSupplier); + } + + protected void bulkIndex(String indexName, SourceSupplier sourceSupplier) throws IOException { + long currentCount = client().prepareSearch(indexName).setSize(0).get().getHits().getTotalHits().value; + BulkRequestBuilder bulkRequestBuilder = client().prepareBulk(); + bulkRequestBuilder.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + for (int i = 0; i < docCount; i++) { + IndexRequest indexRequest = new IndexRequest(indexName).opType(DocWriteRequest.OpType.CREATE); + XContentBuilder source = sourceSupplier.get(); + indexRequest.source(source); + bulkRequestBuilder.add(indexRequest); + } + BulkResponse bulkResponse = bulkRequestBuilder.get(); + if (bulkResponse.hasFailures()) { + fail("Failed to index data: " + bulkResponse.buildFailureMessage()); + } + assertHitCount(client().prepareSearch(indexName).setSize(0).get(), docCount + currentCount); + } + + protected void rollup(String sourceIndex, String rollupIndex, RollupActionConfig config) { + AcknowledgedResponse rollupResponse = client().execute( + RollupAction.INSTANCE, + new RollupAction.Request(sourceIndex, rollupIndex, config) + ).actionGet(); + assertTrue(rollupResponse.isAcknowledged()); + } + + protected RolloverResponse rollover(String dataStreamName) throws ExecutionException, InterruptedException { + RolloverResponse response = client().admin().indices().rolloverIndex(new RolloverRequest(dataStreamName, null)).get(); + assertTrue(response.isAcknowledged()); + return response; + } + + @SuppressWarnings("unchecked") + protected void assertRollupIndex(RollupActionConfig config, String sourceIndex, String rollupIndexName) { + final CompositeAggregationBuilder aggregation = buildCompositeAggs("resp", config); + long numBuckets = 0; + InternalComposite origResp = client().prepareSearch(sourceIndex).addAggregation(aggregation).get().getAggregations().get("resp"); + InternalComposite rollupResp = client().prepareSearch(rollupIndexName) + .addAggregation(aggregation) + .get() + .getAggregations() + .get("resp"); + while (origResp.afterKey() != null) { + numBuckets += origResp.getBuckets().size(); + assertThat(origResp, equalTo(rollupResp)); + aggregation.aggregateAfter(origResp.afterKey()); + origResp = client().prepareSearch(sourceIndex).addAggregation(aggregation).get().getAggregations().get("resp"); + rollupResp = client().prepareSearch(rollupIndexName).addAggregation(aggregation).get().getAggregations().get("resp"); + } + assertThat(origResp, equalTo(rollupResp)); + + SearchResponse resp = client().prepareSearch(rollupIndexName).setTrackTotalHits(true).get(); + + GetIndexResponse indexSettingsResp = client().admin().indices().prepareGetIndex().addIndices(sourceIndex, rollupIndexName).get(); + if (indexSettingsResp.getSetting(sourceIndex, "index.number_of_shards").equals("1") && origResp.getBuckets().size() > 0) { + assertThat(resp.getHits().getTotalHits().value, equalTo(numBuckets)); + } + + // Assert rollup metadata are set in index settings + assertEquals( + indexSettingsResp.getSetting(sourceIndex, "index.uuid"), + indexSettingsResp.getSetting(rollupIndexName, "index.rollup.source.uuid") + ); + assertEquals( + indexSettingsResp.getSetting(sourceIndex, "index.provided_name"), + indexSettingsResp.getSetting(rollupIndexName, "index.rollup.source.name") + ); + + // Assert field mappings + Map sourceMappings = indexSettingsResp.getMappings().get(sourceIndex).getSourceAsMap(); + Map rollupMappings = indexSettingsResp.getMappings().get(rollupIndexName).getSourceAsMap(); + assertEquals(sourceMappings.get("_source"), rollupMappings.get("_source")); + Map> properties = (Map>) rollupMappings.get("properties"); + + RollupActionDateHistogramGroupConfig dateHistoConfig = config.getGroupConfig().getDateHistogram(); + assertEquals(DateFieldMapper.CONTENT_TYPE, properties.get(dateHistoConfig.getField()).get("type")); + Map dateTimeMeta = (Map) properties.get(dateHistoConfig.getField()).get("meta"); + assertEquals(dateHistoConfig.getTimeZone(), dateTimeMeta.get("time_zone")); + assertEquals(dateHistoConfig.getInterval().toString(), dateTimeMeta.get(dateHistoConfig.getIntervalTypeName())); + + for (MetricConfig metricsConfig : config.getMetricsConfig()) { + assertEquals("aggregate_metric_double", properties.get(metricsConfig.getField()).get("type")); + List supportedMetrics = (List) properties.get(metricsConfig.getField()).get("metrics"); + for (String m : metricsConfig.getMetrics()) { + if ("avg".equals(m)) { + assertTrue(supportedMetrics.contains("sum") && supportedMetrics.contains("value_count")); + } else { + assertTrue(supportedMetrics.contains(m)); + } + } + } + + HistogramGroupConfig histoConfig = config.getGroupConfig().getHistogram(); + if (histoConfig != null) { + for (String field : histoConfig.getFields()) { + assertTrue((properties.containsKey(field))); + Map meta = (Map) properties.get(field).get("meta"); + assertEquals(String.valueOf(histoConfig.getInterval()), meta.get("interval")); + } + } + + TermsGroupConfig termsConfig = config.getGroupConfig().getTerms(); + if (termsConfig != null) { + for (String field : termsConfig.getFields()) { + assertTrue(properties.containsKey(field)); + } + } + + // Assert that temporary index was removed + expectThrows( + IndexNotFoundException.class, + () -> client().admin().indices().prepareGetIndex().addIndices(".rolluptmp-" + rollupIndexName).get() + ); + } + + protected CompositeAggregationBuilder buildCompositeAggs(String name, RollupActionConfig config) { + List> sources = new ArrayList<>(); + + RollupActionDateHistogramGroupConfig dateHistoConfig = config.getGroupConfig().getDateHistogram(); + DateHistogramValuesSourceBuilder dateHisto = new DateHistogramValuesSourceBuilder(dateHistoConfig.getField()); + dateHisto.field(dateHistoConfig.getField()); + if (dateHistoConfig.getTimeZone() != null) { + dateHisto.timeZone(ZoneId.of(dateHistoConfig.getTimeZone())); + } + if (dateHistoConfig instanceof RollupActionDateHistogramGroupConfig.FixedInterval) { + dateHisto.fixedInterval(dateHistoConfig.getInterval()); + } else if (dateHistoConfig instanceof RollupActionDateHistogramGroupConfig.CalendarInterval) { + dateHisto.calendarInterval(dateHistoConfig.getInterval()); + } else { + throw new IllegalStateException("unsupported RollupActionDateHistogramGroupConfig"); + } + sources.add(dateHisto); + + if (config.getGroupConfig().getHistogram() != null) { + HistogramGroupConfig histoConfig = config.getGroupConfig().getHistogram(); + for (String field : histoConfig.getFields()) { + HistogramValuesSourceBuilder source = new HistogramValuesSourceBuilder(field).field(field) + .interval(histoConfig.getInterval()); + sources.add(source); + } + } + + if (config.getGroupConfig().getTerms() != null) { + TermsGroupConfig termsConfig = config.getGroupConfig().getTerms(); + for (String field : termsConfig.getFields()) { + TermsValuesSourceBuilder source = new TermsValuesSourceBuilder(field).field(field); + sources.add(source); + } + } + + final CompositeAggregationBuilder composite = new CompositeAggregationBuilder(name, sources).size(10000); + if (config.getMetricsConfig() != null) { + for (MetricConfig metricConfig : config.getMetricsConfig()) { + for (String metricName : metricConfig.getMetrics()) { + switch (metricName) { + case "min" -> composite.subAggregation(new MinAggregationBuilder(metricName).field(metricConfig.getField())); + case "max" -> composite.subAggregation(new MaxAggregationBuilder(metricName).field(metricConfig.getField())); + case "sum" -> composite.subAggregation(new SumAggregationBuilder(metricName).field(metricConfig.getField())); + case "value_count" -> composite.subAggregation( + new ValueCountAggregationBuilder(metricName).field(metricConfig.getField()) + ); + case "avg" -> composite.subAggregation(new AvgAggregationBuilder(metricName).field(metricConfig.getField())); + default -> throw new IllegalArgumentException("Unsupported metric type [" + metricName + "]"); + } + } + } + } + return composite; + } + + @FunctionalInterface + public interface SourceSupplier { + XContentBuilder get() throws IOException; + } + + protected String createDataStream() throws Exception { + String dataStreamName = randomAlphaOfLength(10).toLowerCase(Locale.getDefault()); + Template idxTemplate = new Template(null, new CompressedXContent(""" + {"properties":{"%s":{"type":"date"},"data":{"type":"keyword"}}} + """.formatted(timestampFieldName)), null); + ComposableIndexTemplate template = new ComposableIndexTemplate( + List.of(dataStreamName + "*"), + idxTemplate, + null, + null, + null, + null, + new ComposableIndexTemplate.DataStreamTemplate(), + null + ); + assertTrue( + client().execute( + PutComposableIndexTemplateAction.INSTANCE, + new PutComposableIndexTemplateAction.Request(dataStreamName + "_template").indexTemplate(template) + ).actionGet().isAcknowledged() + ); + assertTrue( + client().execute(CreateDataStreamAction.INSTANCE, new CreateDataStreamAction.Request(dataStreamName)).get().isAcknowledged() + ); + createdDataStreams.add(dataStreamName); + return dataStreamName; + } +} diff --git a/x-pack/plugin/rollup/src/internalClusterTest/java/org/elasticsearch/xpack/rollup/v2/RollupTimeSeriesIT.java b/x-pack/plugin/rollup/src/internalClusterTest/java/org/elasticsearch/xpack/rollup/v2/RollupTimeSeriesIT.java new file mode 100644 index 0000000000000..8b16893ab8455 --- /dev/null +++ b/x-pack/plugin/rollup/src/internalClusterTest/java/org/elasticsearch/xpack/rollup/v2/RollupTimeSeriesIT.java @@ -0,0 +1,246 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.rollup.v2; + +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.time.DateUtils; +import org.elasticsearch.common.util.CancellableThreads.ExecutionCancelledException; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.xcontent.XContentFactory; +import org.elasticsearch.xpack.core.rollup.RollupActionConfig; +import org.elasticsearch.xpack.core.rollup.RollupActionDateHistogramGroupConfig; +import org.elasticsearch.xpack.core.rollup.RollupActionGroupConfig; +import org.elasticsearch.xpack.core.rollup.action.RollupShardStatus; +import org.elasticsearch.xpack.core.rollup.action.RollupShardStatus.Status; +import org.elasticsearch.xpack.core.rollup.job.MetricConfig; +import org.elasticsearch.xpack.core.rollup.job.TermsGroupConfig; +import org.elasticsearch.xpack.rollup.v2.indexer.TimeSeriesRollupShardIndexer; +import org.junit.Before; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class RollupTimeSeriesIT extends RollupIntegTestCase { + @Before + public void init() { + client().admin() + .indices() + .prepareCreate(index) + .setSettings( + Settings.builder() + .put("index.number_of_shards", randomIntBetween(1, 2)) + .put(IndexSettings.MODE.getKey(), "time_series") + .put(IndexMetadata.INDEX_ROUTING_PATH.getKey(), "categorical_1") + .put(IndexSettings.TIME_SERIES_START_TIME.getKey(), 1L) + .put(IndexSettings.TIME_SERIES_END_TIME.getKey(), DateUtils.MAX_MILLIS_BEFORE_9999 - 1) + .build() + ) + .setMapping( + "{\n" + + " \"properties\": {\n" + + " \"@timestamp\": {\n" + + " \"type\": \"date\"\n" + + " },\n" + + " \"numeric_1\": {\n" + + " \"type\": \"double\"\n" + + " },\n" + + " \"numeric_2\": {\n" + + " \"type\": \"float\"\n" + + " },\n" + + " \"categorical_1\": {\n" + + " \"type\": \"keyword\",\n" + + " \"time_series_dimension\": true\n" + + " },\n" + + " \"categorical_2\": {\n" + + " \"type\": \"keyword\",\n" + + " \"time_series_dimension\": true\n" + + " }\n" + + " }\n" + + "}" + ) + .get(); + } + + public void testTimeSeriesRollupShardIndexer() throws IOException { + // create rollup config and index documents into source index + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("@timestamp"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("@timestamp", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("categorical_1", randomAlphaOfLength(1)) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("_tsid")), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + bulkIndex(sourceSupplier); + + IndicesService indexServices = getInstanceFromNode(IndicesService.class); + Index srcIndex = resolveIndex(index); + IndexService indexService = indexServices.indexServiceSafe(srcIndex); + IndexShard shard = indexService.getShard(0); + + // re-use source index as temp index for test + TimeSeriesRollupShardIndexer indexer = new TimeSeriesRollupShardIndexer( + new RollupShardStatus(shard.shardId()), + client(), + indexService, + shard.shardId(), + config, + rollupIndex + ); + assertThat(indexer.status.getStatus(), equalTo(Status.ROLLING)); + indexer.execute(); + if (indexService.shardIds().size() == 1) { + assertThat(indexer.numReceived.get(), equalTo((long) docCount)); + } + assertThat(indexer.numSkip.get(), equalTo(0L)); + assertThat(indexer.numSent.get(), equalTo(indexer.numIndexed.get())); + assertThat(indexer.numFailed.get(), equalTo(0L)); + assertThat(indexer.status.getStatus(), equalTo(Status.STOP)); + } + + public void testNormalTermsGrouping() throws IOException { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("@timestamp"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("@timestamp", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("categorical_1", randomAlphaOfLength(1)) + .field("categorical_2", randomAlphaOfLength(1)) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + bulkIndex(sourceSupplier); + rollup(index, rollupIndex, config); + + RollupActionConfig newConfig = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + assertRollupIndex(newConfig, index, rollupIndex); + } + + public void testTsidTermsGrouping() throws IOException { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("@timestamp"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("@timestamp", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("categorical_1", randomAlphaOfLength(1)) + .field("categorical_2", randomAlphaOfLength(1)) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("_tsid")), + Collections.singletonList(new MetricConfig("numeric_1", List.of("max", "min", "value_count", "avg"))) + ); + bulkIndex(sourceSupplier); + // check if segments are more than one segment + bulkIndex(sourceSupplier); + rollup(index, rollupIndex, config); + + RollupActionConfig newConfig = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1", "categorical_2")), + Collections.singletonList(new MetricConfig("numeric_1", List.of("max", "min", "value_count", "avg"))) + ); + assertRollupIndex(newConfig, index, rollupIndex); + } + + public void testTsidAndOtherTerms() throws IOException { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("@timestamp"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("@timestamp", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("categorical_1", randomAlphaOfLength(1)) + .field("categorical_2", randomAlphaOfLength(1)) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("_tsid", "categorical_1")), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + bulkIndex(sourceSupplier); + rollup(index, rollupIndex, config); + + RollupActionConfig newConfig = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1", "categorical_2")), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + assertRollupIndex(newConfig, index, rollupIndex); + } + + public void testEmptyMetricRollup() throws IOException { + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("@timestamp"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("@timestamp", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("categorical_1", randomAlphaOfLength(1)) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("_tsid")), + Collections.singletonList(new MetricConfig("numeric_2", Collections.singletonList("max"))) + ); + bulkIndex(sourceSupplier); + rollup(index, rollupIndex, config); + + RollupActionConfig newConfig = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), + Collections.singletonList(new MetricConfig("numeric_2", List.of("max"))) + ); + assertRollupIndex(newConfig, index, rollupIndex); + } + + public void testCancelRollupIndexer() throws IOException { + // create rollup config and index documents into source index + RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("@timestamp"); + SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() + .startObject() + .field("@timestamp", randomDateForInterval(dateHistogramGroupConfig.getInterval())) + .field("categorical_1", randomAlphaOfLength(1)) + .field("numeric_1", randomDouble()) + .endObject(); + RollupActionConfig config = new RollupActionConfig( + new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("_tsid")), + Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) + ); + bulkIndex(sourceSupplier); + + IndicesService indexServices = getInstanceFromNode(IndicesService.class); + Index srcIndex = resolveIndex(index); + IndexService indexService = indexServices.indexServiceSafe(srcIndex); + IndexShard shard = indexService.getShard(0); + + // re-use source index as temp index for test + TimeSeriesRollupShardIndexer indexer = new TimeSeriesRollupShardIndexer( + new RollupShardStatus(shard.shardId()), + client(), + indexService, + shard.shardId(), + config, + rollupIndex + ); + indexer.status.setStatus(Status.ABORT); + { + ExecutionCancelledException exception = expectThrows(ExecutionCancelledException.class, () -> indexer.execute()); + assertThat(exception.getMessage(), containsString("rollup cancelled")); + } + } +} diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/Rollup.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/Rollup.java index 0039f994fb749..f2a5a9abf0714 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/Rollup.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/Rollup.java @@ -66,9 +66,9 @@ import org.elasticsearch.xpack.rollup.rest.RestRollupSearchAction; import org.elasticsearch.xpack.rollup.rest.RestStartRollupJobAction; import org.elasticsearch.xpack.rollup.rest.RestStopRollupJobAction; -import org.elasticsearch.xpack.rollup.v2.RestRollupAction; import org.elasticsearch.xpack.rollup.v2.TransportRollupAction; import org.elasticsearch.xpack.rollup.v2.TransportRollupIndexerAction; +import org.elasticsearch.xpack.rollup.v2.rest.RestRollupAction; import java.time.Clock; import java.util.ArrayList; diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/FieldMetricsProducer.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/FieldMetricsProducer.java deleted file mode 100644 index 0e7558fbbdec8..0000000000000 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/FieldMetricsProducer.java +++ /dev/null @@ -1,174 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ - -package org.elasticsearch.xpack.rollup.v2; - -import org.elasticsearch.xpack.core.rollup.job.MetricConfig; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -class FieldMetricsProducer { - final String fieldName; - final List metrics; - - FieldMetricsProducer(String fieldName, List metrics) { - this.fieldName = fieldName; - this.metrics = metrics; - } - - void reset() { - for (Metric metric : metrics) { - metric.reset(); - } - } - - abstract static class Metric { - final String name; - - protected Metric(String name) { - this.name = name; - } - - abstract void collect(double number); - - abstract Number get(); - - abstract void reset(); - } - - private static class Max extends Metric { - private Double max; - - Max() { - super("max"); - } - - @Override - void collect(double value) { - this.max = max != null ? Math.max(value, max) : value; - } - - @Override - Number get() { - return max; - } - - @Override - void reset() { - max = null; - } - } - - private static class Min extends Metric { - private Double min; - - private Min() { - super("min"); - } - - @Override - void collect(double value) { - this.min = min != null ? Math.min(value, min) : value; - } - - @Override - Number get() { - return min; - } - - @Override - void reset() { - min = null; - } - } - - private static class Sum extends Metric { - private double sum = 0; - - private Sum() { - super("sum"); - } - - @Override - void collect(double value) { - // TODO: switch to Kahan summation ? - this.sum += value; - } - - @Override - Number get() { - return sum; - } - - @Override - void reset() { - sum = 0; - } - } - - private static class ValueCount extends Metric { - private long count; - - private ValueCount() { - super("value_count"); - } - - @Override - void collect(double value) { - count++; - } - - @Override - Number get() { - return count; - } - - @Override - void reset() { - count = 0; - } - } - - static List buildMetrics(List metricsConfigs) { - final List fields = new ArrayList<>(); - if (metricsConfigs != null) { - for (MetricConfig metricConfig : metricsConfigs) { - final List normalizedMetrics = normalizeMetrics(metricConfig.getMetrics()); - final List list = new ArrayList<>(); - if (normalizedMetrics.isEmpty() == false) { - for (String metricName : normalizedMetrics) { - switch (metricName) { - case "min" -> list.add(new Min()); - case "max" -> list.add(new Max()); - case "sum" -> list.add(new Sum()); - case "value_count" -> list.add(new ValueCount()); - default -> throw new IllegalArgumentException("Unsupported metric type [" + metricName + "]"); - } - } - fields.add(new FieldMetricsProducer(metricConfig.getField(), Collections.unmodifiableList(list))); - } - } - } - return Collections.unmodifiableList(fields); - } - - static List normalizeMetrics(List metrics) { - List newMetrics = new ArrayList<>(metrics); - // avg = sum + value_count - if (newMetrics.remove(MetricConfig.AVG.getPreferredName())) { - if (newMetrics.contains(MetricConfig.VALUE_COUNT.getPreferredName()) == false) { - newMetrics.add(MetricConfig.VALUE_COUNT.getPreferredName()); - } - if (newMetrics.contains(MetricConfig.SUM.getPreferredName()) == false) { - newMetrics.add(MetricConfig.SUM.getPreferredName()); - } - } - return newMetrics; - } -} diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/FieldValueFetcher.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/FieldValueFetcher.java index ff3ff9c24ba0c..30007d42f48e0 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/FieldValueFetcher.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/FieldValueFetcher.java @@ -11,9 +11,14 @@ import org.apache.lucene.util.BytesRef; import org.elasticsearch.index.fielddata.FormattedDocValues; import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.LeafFieldData; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.DocValueFormat.TimeSeriesIdDocValueFormat; +import org.elasticsearch.xpack.rollup.v2.indexer.metrics.LeafMetricField; +import org.elasticsearch.xpack.rollup.v2.indexer.metrics.MetricCollector; +import org.elasticsearch.xpack.rollup.v2.indexer.metrics.NumberLeafMetricField; import java.io.IOException; import java.math.BigInteger; @@ -25,7 +30,7 @@ import java.util.Set; import java.util.function.Function; -class FieldValueFetcher { +public class FieldValueFetcher { private static final Set> VALID_TYPES = Collections.unmodifiableSet( new HashSet<>(Arrays.asList(Long.class, Double.class, BigInteger.class, String.class, BytesRef.class)) ); @@ -44,24 +49,40 @@ protected FieldValueFetcher(String name, MappedFieldType fieldType, IndexFieldDa this.valueFunc = valueFunc; } - FormattedDocValues getLeaf(LeafReaderContext context) { - final FormattedDocValues delegate = fieldData.load(context).getFormattedValues(DocValueFormat.RAW); - return new FormattedDocValues() { - @Override - public boolean advanceExact(int docId) throws IOException { - return delegate.advanceExact(docId); - } + public MappedFieldType getFieldType() { + return fieldType; + } - @Override - public int docValueCount() throws IOException { - return delegate.docValueCount(); - } + public FormattedDocValues getGroupLeaf(LeafReaderContext context) { + if (format instanceof TimeSeriesIdDocValueFormat) { + // TODO RAW format use string to save format data, but the _tsid is a binary data. + // it maybe lost bytes when string decode to binary data. + // so here use it's own format, it will decode in indexBucket + return fieldData.load(context).getFormattedValues(format); + } else { + final FormattedDocValues delegate = fieldData.load(context).getFormattedValues(DocValueFormat.RAW); + return new FormattedDocValues() { + @Override + public boolean advanceExact(int docId) throws IOException { + return delegate.advanceExact(docId); + } - @Override - public Object nextValue() throws IOException { - return valueFunc.apply(delegate.nextValue()); - } - }; + @Override + public int docValueCount() throws IOException { + return delegate.docValueCount(); + } + + @Override + public Object nextValue() throws IOException { + return valueFunc.apply(delegate.nextValue()); + } + }; + } + } + + public LeafMetricField getMetricFieldLeaf(LeafReaderContext context, MetricCollector[] metricCollectors) { + LeafFieldData leafFieldData = fieldData.load(context); + return new NumberLeafMetricField(metricCollectors, leafFieldData); } Object format(Object value) { @@ -78,19 +99,23 @@ Object format(Object value) { } } - static List build(SearchExecutionContext context, String[] fields) { + static List buildList(SearchExecutionContext context, String[] fields) { List fetchers = new ArrayList<>(); for (String field : fields) { - MappedFieldType fieldType = context.getFieldType(field); - if (fieldType == null) { - throw new IllegalArgumentException("Unknown field: [" + field + "]"); - } - IndexFieldData fieldData = context.getForField(fieldType); - fetchers.add(new FieldValueFetcher(field, fieldType, fieldData, getValidator(field))); + fetchers.add(build(context, field)); } return Collections.unmodifiableList(fetchers); } + static FieldValueFetcher build(SearchExecutionContext context, String field) { + MappedFieldType fieldType = context.getFieldType(field); + if (fieldType == null) { + throw new IllegalArgumentException("Unknown field: [" + field + "]"); + } + IndexFieldData fieldData = context.getForField(fieldType); + return new FieldValueFetcher(field, fieldType, fieldData, getValidator(field)); + } + static List buildHistograms(SearchExecutionContext context, String[] fields, double interval) { List fetchers = new ArrayList<>(); for (String field : fields) { diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/RollupExitException.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/RollupExitException.java new file mode 100644 index 0000000000000..c617ea4fc1143 --- /dev/null +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/RollupExitException.java @@ -0,0 +1,16 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.rollup.v2; + +import org.elasticsearch.ElasticsearchException; + +public class RollupExitException extends ElasticsearchException { + public RollupExitException(String msg, Exception e) { + super(msg, e); + } +} diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/RollupShardIndexer.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/RollupShardIndexer.java index b97dceab34c98..b209a737f4293 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/RollupShardIndexer.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/RollupShardIndexer.java @@ -8,22 +8,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.lucene.document.LongPoint; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.PointValues; -import org.apache.lucene.search.CollectionTerminatedException; -import org.apache.lucene.search.Collector; -import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.LeafCollector; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.Scorable; -import org.apache.lucene.search.ScoreMode; -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.FilterDirectory; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.BytesRefIterator; import org.elasticsearch.action.bulk.BackoffPolicy; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; @@ -32,12 +18,8 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.Rounding; -import org.elasticsearch.common.io.stream.ByteBufferStreamInput; -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.util.Maps; import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.IndexService; @@ -49,23 +31,29 @@ import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.DocValueFormat.TimeSeriesIdDocValueFormat; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentFactory; +import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xpack.core.rollup.RollupActionConfig; import org.elasticsearch.xpack.core.rollup.RollupActionDateHistogramGroupConfig; +import org.elasticsearch.xpack.core.rollup.action.RollupShardStatus; +import org.elasticsearch.xpack.core.rollup.action.RollupShardStatus.Status; import org.elasticsearch.xpack.core.rollup.job.HistogramGroupConfig; import org.elasticsearch.xpack.core.rollup.job.MetricConfig; import org.elasticsearch.xpack.core.rollup.job.TermsGroupConfig; +import org.elasticsearch.xpack.rollup.v2.indexer.metrics.LeafMetricField; +import org.elasticsearch.xpack.rollup.v2.indexer.metrics.MetricCollector; +import org.elasticsearch.xpack.rollup.v2.indexer.metrics.MetricField; import java.io.Closeable; import java.io.IOException; -import java.nio.ByteBuffer; import java.time.ZoneId; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -79,70 +67,51 @@ * An indexer for rollup that sorts the buckets from the provided source shard on disk and send them * to the target rollup index. */ -class RollupShardIndexer { +public abstract class RollupShardIndexer { private static final Logger logger = LogManager.getLogger(RollupShardIndexer.class); - private final IndexShard indexShard; + protected final IndexShard indexShard; private final Client client; - private final RollupActionConfig config; - private final String tmpIndex; + protected final RollupActionConfig config; + protected final String tmpIndex; - private final Directory dir; - private final Engine.Searcher searcher; - private final SearchExecutionContext searchExecutionContext; - private final MappedFieldType timestampField; - private final DocValueFormat timestampFormat; - private final Rounding.Prepared rounding; + protected final Engine.Searcher searcher; + protected final SearchExecutionContext searchExecutionContext; - private final List groupFieldFetchers; - private final List metricsFieldFetchers; + protected final FieldValueFetcher timestampFetcher; + protected final List groupFieldFetchers; + protected final MetricField[] metricFields; - private final CompressingOfflineSorter sorter; + protected final Rounding.Prepared rounding; + protected final BulkProcessor bulkProcessor; + protected final AtomicLong numReceived = new AtomicLong(); + protected final AtomicLong numSkip = new AtomicLong(); + protected final AtomicLong numSent = new AtomicLong(); + protected final AtomicLong numIndexed = new AtomicLong(); + protected final AtomicLong numFailed = new AtomicLong(); - private final BulkProcessor bulkProcessor; - private final AtomicLong numSent = new AtomicLong(); - private final AtomicLong numIndexed = new AtomicLong(); + protected RollupShardStatus status; - // for testing - final Set tmpFiles = new HashSet<>(); - final Set tmpFilesDeleted = new HashSet<>(); - - RollupShardIndexer( + protected RollupShardIndexer( + RollupShardStatus rollupShardStatus, Client client, IndexService indexService, ShardId shardId, RollupActionConfig config, - String tmpIndex, - int ramBufferSizeMB + String tmpIndex ) { + this.status = rollupShardStatus; + status.init(numReceived, numSkip, numSent, numIndexed, numFailed); this.client = client; this.indexShard = indexService.getShard(shardId.id()); this.config = config; this.tmpIndex = tmpIndex; + // refresh shard before rollup + this.indexShard.refresh("rollup"); this.searcher = indexShard.acquireSearcher("rollup"); Closeable toClose = searcher; try { - this.dir = new FilterDirectory(searcher.getDirectoryReader().directory()) { - @Override - public IndexOutput createOutput(String name, IOContext context) throws IOException { - tmpFiles.add(name); - return super.createOutput(name, context); - } - - @Override - public IndexOutput createTempOutput(String prefix, String suffix, IOContext context) throws IOException { - IndexOutput output = super.createTempOutput(prefix, suffix, context); - tmpFiles.add(output.getName()); - return output; - } - - @Override - public void deleteFile(String name) throws IOException { - tmpFilesDeleted.add(name); - super.deleteFile(name); - } - }; this.searchExecutionContext = indexService.newSearchExecutionContext( indexShard.shardId().id(), 0, @@ -151,15 +120,14 @@ public void deleteFile(String name) throws IOException { null, Collections.emptyMap() ); - this.timestampField = searchExecutionContext.getFieldType(config.getGroupConfig().getDateHistogram().getField()); - verifyTimestampField(timestampField); - this.timestampFormat = timestampField.docValueFormat(null, null); + this.timestampFetcher = FieldValueFetcher.build(searchExecutionContext, config.getGroupConfig().getDateHistogram().getField()); + verifyTimestampField(this.timestampFetcher.fieldType); this.rounding = createRounding(config.getGroupConfig().getDateHistogram()).prepareForUnknown(); this.groupFieldFetchers = new ArrayList<>(); if (config.getGroupConfig().getTerms() != null) { TermsGroupConfig termsConfig = config.getGroupConfig().getTerms(); - this.groupFieldFetchers.addAll(FieldValueFetcher.build(searchExecutionContext, termsConfig.getFields())); + this.groupFieldFetchers.addAll(FieldValueFetcher.buildList(searchExecutionContext, termsConfig.getFields())); } if (config.getGroupConfig().getHistogram() != null) { @@ -170,21 +138,28 @@ public void deleteFile(String name) throws IOException { } if (config.getMetricsConfig().size() > 0) { - final String[] metricFields = config.getMetricsConfig().stream().map(MetricConfig::getField).toArray(String[]::new); - this.metricsFieldFetchers = FieldValueFetcher.build(searchExecutionContext, metricFields); + List metricFieldList = new ArrayList<>(); + for (MetricConfig metricConfig : config.getMetricsConfig()) { + FieldValueFetcher fetcher = FieldValueFetcher.build(searchExecutionContext, metricConfig.getField()); + metricFieldList.add(MetricField.buildMetricField(metricConfig, fetcher)); + } + this.metricFields = metricFieldList.toArray(new MetricField[0]); } else { - this.metricsFieldFetchers = Collections.emptyList(); + this.metricFields = new MetricField[0]; } - this.sorter = new CompressingOfflineSorter(dir, "rollup-", keyComparator(), ramBufferSizeMB); toClose = null; } finally { IOUtils.closeWhileHandlingException(toClose); } this.bulkProcessor = createBulkProcessor(); + + status.setStatus(Status.ROLLING); } + public abstract void execute() throws IOException; + private void verifyTimestampField(MappedFieldType fieldType) { if (fieldType == null) { throw new IllegalArgumentException("fieldType is null"); @@ -197,16 +172,12 @@ private void verifyTimestampField(MappedFieldType fieldType) { } } - public long execute() throws IOException { - Long bucket = Long.MIN_VALUE; - try (searcher; bulkProcessor) { - do { - bucket = computeBucket(bucket); - } while (bucket != null); + protected boolean isCanceled() { + if (status.getStatus() == Status.ABORT || status.getStatus() == Status.STOP) { + return true; + } else { + return false; } - // TODO: check that numIndexed == numSent, otherwise throw an exception - logger.info("Successfully sent [" + numIndexed.get() + "], indexed [" + numIndexed.get() + "]"); - return numIndexed.get(); } private BulkProcessor createBulkProcessor() { @@ -229,7 +200,14 @@ public void afterBulk(long executionId, BulkRequest request, BulkResponse respon (msg1, msg2) -> Objects.equals(msg1, msg2) ? msg1 : msg1 + "," + msg2 ) ); - logger.error("failures: [{}]", failures); + numFailed.addAndGet(failures.size()); + logger.debug( + "[{}] rollup index failures summery: failed count: [{}], failed sample: [{}]", + indexShard.shardId(), + failures.size(), + failures.values().iterator().next() + ); + logger.trace("[{}] rollup index failures: [{}]", indexShard.shardId(), failures); } } @@ -237,6 +215,8 @@ public void afterBulk(long executionId, BulkRequest request, BulkResponse respon public void afterBulk(long executionId, BulkRequest request, Throwable failure) { long items = request.numberOfActions(); numSent.addAndGet(-items); + numFailed.addAndGet(items); + logger.debug(() -> new ParameterizedMessage("[{}] rollup index bulk failed", indexShard.shardId()), failure); } }; return BulkProcessor.builder(client::bulk, listener, "rollup-shard-indexer") @@ -248,7 +228,7 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) .build(); } - private Rounding createRounding(RollupActionDateHistogramGroupConfig groupConfig) { + protected Rounding createRounding(RollupActionDateHistogramGroupConfig groupConfig) { DateHistogramInterval interval = groupConfig.getInterval(); ZoneId zoneId = groupConfig.getTimeZone() != null ? ZoneId.of(groupConfig.getTimeZone()) : null; Rounding.Builder tzRoundingBuilder; @@ -264,147 +244,74 @@ private Rounding createRounding(RollupActionDateHistogramGroupConfig groupConfig return tzRoundingBuilder.timeZone(zoneId).build(); } - private void indexBucket(BucketKey key, List fieldsMetrics, int docCount) { + protected void indexBucket(BucketKey key, int docCount) throws IOException { IndexRequestBuilder request = client.prepareIndex(tmpIndex); - Map doc = Maps.newMapWithExpectedSize(2 + key.groupFields.size() + fieldsMetrics.size()); - doc.put(DocCountFieldMapper.NAME, docCount); - doc.put(timestampField.name(), timestampFormat.format(key.timestamp)); + XContentBuilder builder = XContentFactory.contentBuilder(XContentType.SMILE); + builder.startObject(); + builder.field(DocCountFieldMapper.NAME, docCount); + builder.field(timestampFetcher.fieldType.name(), timestampFetcher.format.format(key.timestamp)); + Set groups = new HashSet<>(); for (int i = 0; i < key.groupFields.size(); i++) { FieldValueFetcher fetcher = groupFieldFetchers.get(i); if (key.groupFields.get(i) != null) { - doc.put(fetcher.name, fetcher.format(key.groupFields.get(i))); - } - } - - for (FieldMetricsProducer field : fieldsMetrics) { - Map map = new HashMap<>(); - for (FieldMetricsProducer.Metric metric : field.metrics) { - map.put(metric.name, metric.get()); - } - doc.put(field.fieldName, map); - } - request.setSource(doc); - bulkProcessor.add(request.request()); - } - - private Long computeBucket(long lastRounding) throws IOException { - Long nextRounding = findNextRounding(lastRounding); - if (nextRounding == null) { - return null; - } - long nextRoundingLastValue = rounding.nextRoundingValue(nextRounding) - 1; - try (XExternalRefSorter externalSorter = new XExternalRefSorter(sorter)) { - Query rangeQuery = LongPoint.newRangeQuery(timestampField.name(), nextRounding, nextRoundingLastValue); - searcher.search(rangeQuery, new BucketCollector(nextRounding, externalSorter)); - - BytesRefIterator it = externalSorter.iterator(); - BytesRef next = it.next(); - - List fieldsMetrics = FieldMetricsProducer.buildMetrics(config.getMetricsConfig()); - BucketKey lastKey = null; - int docCount = 0; - while (next != null) { - try (StreamInput in = new ByteBufferStreamInput(ByteBuffer.wrap(next.bytes, next.offset, next.length))) { - // skip key size - in.readInt(); - BucketKey key = decodeKey(in, groupFieldFetchers.size()); - if (lastKey != null && lastKey.equals(key) == false) { - indexBucket(lastKey, fieldsMetrics, docCount); - docCount = 0; - for (FieldMetricsProducer producer : fieldsMetrics) { - producer.reset(); + if (fetcher.format instanceof TimeSeriesIdDocValueFormat) { + @SuppressWarnings("unchecked") + Map dimensionFields = (Map) key.groupFields.get(i); + for (Map.Entry entry : dimensionFields.entrySet()) { + if (groups.contains(entry.getKey())) { + continue; } + builder.field(entry.getKey(), entry.getValue()); + groups.add(entry.getKey()); } - for (FieldMetricsProducer field : fieldsMetrics) { - int size = in.readVInt(); - for (int i = 0; i < size; i++) { - double value = in.readDouble(); - for (FieldMetricsProducer.Metric metric : field.metrics) { - metric.collect(value); - } - } - } - ++docCount; - lastKey = key; + } else if (false == groups.contains(fetcher.name)) { + builder.field(fetcher.name, fetcher.format(key.groupFields.get(i))); } - next = it.next(); - } - if (lastKey != null) { - indexBucket(lastKey, fieldsMetrics, docCount); } } - return nextRoundingLastValue; - } - private Long findNextRounding(long lastRounding) throws IOException { - Long nextRounding = null; - for (LeafReaderContext leafReaderContext : searcher.getIndexReader().leaves()) { - PointValues pointValues = leafReaderContext.reader().getPointValues(timestampField.name()); - final NextRoundingVisitor visitor = new NextRoundingVisitor(rounding, lastRounding); - try { - pointValues.intersect(visitor); - } catch (CollectionTerminatedException exc) {} - if (visitor.nextRounding != null) { - nextRounding = nextRounding == null ? visitor.nextRounding : Math.min(nextRounding, visitor.nextRounding); + for (MetricField metricField : metricFields) { + Map map = null; + boolean metricMissing = false; + for (MetricCollector metric : metricField.getCollectors()) { + if (metric.get() == null) { + metricMissing = true; + break; + } + if (map == null) { + map = new HashMap<>(); + } + map.put(metric.name, metric.get()); } - } - return nextRounding; - } - private static BytesRef encodeKey(long timestamp, List groupFields) throws IOException { - try (BytesStreamOutput out = new BytesStreamOutput()) { - out.writeLong(timestamp); - for (Object obj : groupFields) { - out.writeGenericValue(obj); + if (false == metricMissing) { + builder.field(metricField.getName(), map); } - return out.bytes().toBytesRef(); - } - } - - private static BucketKey decodeKey(StreamInput in, int numGroupFields) throws IOException { - long timestamp = in.readLong(); - List values = new ArrayList<>(); - for (int i = 0; i < numGroupFields; i++) { - values.add(in.readGenericValue()); } - return new BucketKey(timestamp, values); - } - - /** - * Returns a {@link Comparator} that can be used to sort inputs created by the {@link BucketCollector}. - * We just want identical buckets to be consecutive for the merge so this comparator doesn't follow the natural - * order and simply checks for identical binary keys. - */ - private static Comparator keyComparator() { - return (o1, o2) -> { - int keySize1 = readInt(o1.bytes, o1.offset); - int keySize2 = readInt(o2.bytes, o2.offset); - return Arrays.compareUnsigned( - o1.bytes, - o1.offset + Integer.BYTES, - keySize1 + o1.offset + Integer.BYTES, - o2.bytes, - o2.offset + Integer.BYTES, - keySize2 + o2.offset + Integer.BYTES - ); - }; - } - private static int readInt(byte[] bytes, int offset) { - return ((bytes[offset] & 0xFF) << 24) | ((bytes[offset + 1] & 0xFF) << 16) | ((bytes[offset + 2] & 0xFF) << 8) | (bytes[offset + 3] - & 0xFF); + builder.endObject(); + request.setSource(builder); + bulkProcessor.add(request.request()); } - private static class BucketKey { - private final long timestamp; - private final List groupFields; + protected static class BucketKey { + final long timestamp; + final List groupFields; - BucketKey(long timestamp, List groupFields) { + public BucketKey(long timestamp, List groupFields) { this.timestamp = timestamp; this.groupFields = groupFields; } + public long getTimestamp() { + return timestamp; + } + + public List getGroupFields() { + return groupFields; + } + @Override public boolean equals(Object o) { if (this == o) return true; @@ -424,143 +331,25 @@ public String toString() { } } - private class BucketCollector implements Collector { - private final long timestamp; - private final XExternalRefSorter externalSorter; - - private BucketCollector(long timestamp, XExternalRefSorter externalSorter) { - this.externalSorter = externalSorter; - this.timestamp = timestamp; - } - - @Override - public LeafCollector getLeafCollector(LeafReaderContext context) { - final List groupFieldLeaves = leafFetchers(context, groupFieldFetchers); - final List metricsFieldLeaves = leafFetchers(context, metricsFieldFetchers); - return new LeafCollector() { - @Override - public void setScorer(Scorable scorer) {} - - @Override - public void collect(int docID) throws IOException { - List> combinationKeys = new ArrayList<>(); - for (FormattedDocValues leafField : groupFieldLeaves) { - if (leafField.advanceExact(docID)) { - List lst = new ArrayList<>(); - for (int i = 0; i < leafField.docValueCount(); i++) { - lst.add(leafField.nextValue()); - } - combinationKeys.add(lst); - } else { - combinationKeys.add(null); - } - } - - final BytesRef valueBytes; - try (BytesStreamOutput out = new BytesStreamOutput()) { - for (FormattedDocValues formattedDocValues : metricsFieldLeaves) { - if (formattedDocValues.advanceExact(docID)) { - out.writeVInt(formattedDocValues.docValueCount()); - for (int i = 0; i < formattedDocValues.docValueCount(); i++) { - Object obj = formattedDocValues.nextValue(); - if (obj instanceof Number == false) { - throw new IllegalArgumentException("Expected [Number], got [" + obj.getClass() + "]"); - } - out.writeDouble(((Number) obj).doubleValue()); - } - } else { - out.writeVInt(0); - } - } - valueBytes = out.bytes().toBytesRef(); - } - for (List groupFields : cartesianProduct(combinationKeys)) { - try (BytesStreamOutput out = new BytesStreamOutput()) { - BytesRef keyBytes = encodeKey(timestamp, groupFields); - out.writeInt(keyBytes.length); - out.writeBytes(keyBytes.bytes, keyBytes.offset, keyBytes.length); - out.writeBytes(valueBytes.bytes, valueBytes.offset, valueBytes.length); - externalSorter.add(out.bytes().toBytesRef()); - } - } - } - }; - } - - private List leafFetchers(LeafReaderContext context, List fetchers) { - List leaves = new ArrayList<>(); - for (FieldValueFetcher fetcher : fetchers) { - leaves.add(fetcher.getLeaf(context)); - } - return leaves; - } - - @Override - public ScoreMode scoreMode() { - return ScoreMode.COMPLETE_NO_SCORES; + protected FormattedDocValues[] leafGroupFetchers(LeafReaderContext context) { + List leaves = new ArrayList<>(); + for (FieldValueFetcher fetcher : groupFieldFetchers) { + leaves.add(fetcher.getGroupLeaf(context)); } + return leaves.toArray(new FormattedDocValues[0]); } - private class NextRoundingVisitor implements PointValues.IntersectVisitor { - final Rounding.Prepared rounding; - final long lastRounding; - - Long nextRounding = null; - - NextRoundingVisitor(Rounding.Prepared rounding, long lastRounding) { - this.rounding = rounding; - this.lastRounding = lastRounding; - } - - @Override - public void visit(int docID) { - throw new IllegalStateException("should never be called"); - } - - @Override - public void visit(DocIdSetIterator iterator, byte[] packedValue) { - long bucket = rounding.round(LongPoint.decodeDimension(packedValue, 0)); - checkMinRounding(bucket); - } - - @Override - public void visit(int docID, byte[] packedValue) { - long bucket = rounding.round(LongPoint.decodeDimension(packedValue, 0)); - checkMinRounding(bucket); - } - - @Override - public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) { - long maxRounding = rounding.round(LongPoint.decodeDimension(maxPackedValue, 0)); - if (maxRounding <= lastRounding) { - return PointValues.Relation.CELL_OUTSIDE_QUERY; - } - long minRounding = rounding.round(LongPoint.decodeDimension(minPackedValue, 0)); - checkMinRounding(minRounding); - return PointValues.Relation.CELL_CROSSES_QUERY; - } - - private void checkMinRounding(long roundingValue) { - if (roundingValue > lastRounding) { - nextRounding = roundingValue; - throw new CollectionTerminatedException(); - } + protected LeafMetricField[] leafMetricFields(LeafReaderContext context) { + List leaves = new ArrayList<>(); + for (MetricField metricField : metricFields) { + leaves.add(metricField.getMetricFieldLeaf(context)); } + return leaves.toArray(new LeafMetricField[0]); } - private static List> cartesianProduct(List> lists) { - List> combinations = Arrays.asList(Arrays.asList()); - for (List list : lists) { - List> extraColumnCombinations = new ArrayList<>(); - for (List combination : combinations) { - for (Object element : list) { - List newCombination = new ArrayList<>(combination); - newCombination.add(element); - extraColumnCombinations.add(newCombination); - } - } - combinations = extraColumnCombinations; + protected void resetMetricCollectors() { + for (MetricField metricField : metricFields) { + metricField.resetCollectors(); } - return combinations; } } diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/TransportRollupAction.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/TransportRollupAction.java index 7e809c6d5e4a4..0b93805d47f84 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/TransportRollupAction.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/TransportRollupAction.java @@ -10,10 +10,10 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.create.CreateIndexClusterStateUpdateRequest; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; -import org.elasticsearch.action.admin.indices.rollover.MetadataRolloverService; import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest; import org.elasticsearch.action.admin.indices.shrink.ResizeRequest; import org.elasticsearch.action.admin.indices.shrink.ResizeType; +import org.elasticsearch.action.fieldcaps.FieldCapabilities; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.AcknowledgedResponse; @@ -29,18 +29,28 @@ import org.elasticsearch.cluster.metadata.IndexAbstraction; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.MappingMetadata; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.metadata.MetadataCreateIndexService; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.Randomness; -import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.time.DateUtils; import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.core.Glob; import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexMode; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.mapper.DataStreamTimestampFieldMapper; import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.index.mapper.NestedObjectMapper; import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.index.mapper.ObjectMapper; +import org.elasticsearch.index.mapper.SourceFieldMapper; +import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper; +import org.elasticsearch.indices.InvalidIndexNameException; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -49,6 +59,7 @@ import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xpack.aggregatemetric.mapper.AggregateDoubleMetricFieldMapper; import org.elasticsearch.xpack.core.ClientHelper; +import org.elasticsearch.xpack.core.ilm.LifecycleSettings; import org.elasticsearch.xpack.core.rollup.RollupActionConfig; import org.elasticsearch.xpack.core.rollup.RollupActionDateHistogramGroupConfig; import org.elasticsearch.xpack.core.rollup.RollupActionGroupConfig; @@ -57,10 +68,20 @@ import org.elasticsearch.xpack.core.rollup.action.RollupIndexerAction; import org.elasticsearch.xpack.core.rollup.job.HistogramGroupConfig; import org.elasticsearch.xpack.core.rollup.job.MetricConfig; +import org.elasticsearch.xpack.core.rollup.job.TermsGroupConfig; +import org.elasticsearch.xpack.rollup.v2.indexer.metrics.MetricField; import java.io.IOException; +import java.time.Instant; import java.util.ArrayList; +import java.util.LinkedHashSet; import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.function.Consumer; +import java.util.function.Predicate; +import java.util.stream.Collectors; /** * The master rollup action that coordinates @@ -70,7 +91,7 @@ */ public class TransportRollupAction extends AcknowledgedTransportMasterNodeAction { - private static final Settings VISIBLE_INDEX_SETTINGS = Settings.builder().put(IndexMetadata.SETTING_INDEX_HIDDEN, false).build(); + public static final String TMP_INDEX_PREFIX = ".rolluptmp-"; private static final Settings WRITE_BLOCKED_SETTINGS = Settings.builder().put(IndexMetadata.SETTING_BLOCKS_WRITE, true).build(); private final Client client; @@ -108,51 +129,21 @@ protected void masterOperation( RollupAction.Request request, ClusterState state, ActionListener listener - ) throws IOException { - String originalIndexName = request.getSourceIndex(); - - final String rollupIndexName; - if (request.getRollupIndex() == null) { - rollupIndexName = "rollup-" + originalIndexName + "-" + UUIDs.randomBase64UUID(Randomness.get()); - } else { - rollupIndexName = request.getRollupIndex(); + ) { + final String originalIndexName = request.getSourceIndex(); + IndexMetadata originalIndexMetadata = state.getMetadata().index(originalIndexName); + if (originalIndexMetadata == null) { + throw new InvalidIndexNameException(originalIndexName, "rollup origin index metadata missing"); } - String tmpIndexName = ".rolluptmp-" + rollupIndexName; + final String rollupIndexName = request.getRollupIndex(); + validExistsRollupIndex(state, rollupIndexName); - final XContentBuilder mapping; - try { - mapping = getMapping(request.getRollupConfig()); - } catch (IOException e) { - listener.onFailure(e); - return; - } + String tmpIndexName = TMP_INDEX_PREFIX + rollupIndexName; FieldCapabilitiesRequest fieldCapsRequest = new FieldCapabilitiesRequest().indices(originalIndexName) .fields(request.getRollupConfig().getAllFields().toArray(new String[0])); fieldCapsRequest.setParentTask(clusterService.localNode().getId(), task.getId()); - // Add the source index name and UUID to the rollup index metadata. If the original index is a rollup index itself, - // we will add the name and UUID of the raw index that we initially rolled up. - IndexMetadata originalIndexMetadata = state.getMetadata().index(originalIndexName); - String sourceIndexName = IndexMetadata.INDEX_ROLLUP_SOURCE_NAME.exists(originalIndexMetadata.getSettings()) - ? IndexMetadata.INDEX_ROLLUP_SOURCE_NAME.get(originalIndexMetadata.getSettings()) - : originalIndexName; - String sourceIndexUuid = IndexMetadata.INDEX_ROLLUP_SOURCE_UUID.exists(originalIndexMetadata.getSettings()) - ? IndexMetadata.INDEX_ROLLUP_SOURCE_UUID.get(originalIndexMetadata.getSettings()) - : originalIndexMetadata.getIndexUUID(); - - CreateIndexClusterStateUpdateRequest createIndexClusterStateUpdateRequest = new CreateIndexClusterStateUpdateRequest( - "rollup", - tmpIndexName, - tmpIndexName - ).settings(MetadataRolloverService.HIDDEN_INDEX_SETTINGS) - .mappings(XContentHelper.convertToJson(BytesReference.bytes(mapping), false, XContentType.JSON)); - - RollupIndexerAction.Request rollupIndexerRequest = new RollupIndexerAction.Request(request); - ResizeRequest resizeRequest = new ResizeRequest(request.getRollupIndex(), tmpIndexName); - resizeRequest.setResizeType(ResizeType.CLONE); - resizeRequest.getTargetIndexRequest().settings(VISIBLE_INDEX_SETTINGS); - UpdateSettingsRequest updateSettingsReq = new UpdateSettingsRequest(WRITE_BLOCKED_SETTINGS, tmpIndexName); // 1. validate Rollup Config against Field Caps // 2. create hidden temporary index @@ -173,12 +164,38 @@ protected void masterOperation( listener.onFailure(validationException); return; } + // rebuild rollup config, deal wildcard fields + request.setRollupConfig(rebuildRollupConfig(request.getRollupConfig(), fieldCapsResponse.get())); request.getRollupConfig().validateMappings(fieldCapsResponse.get(), validationException); if (validationException.validationErrors().size() > 0) { listener.onFailure(validationException); return; } + final XContentBuilder mapping; + try { + mapping = getMapping(request.getRollupConfig(), originalIndexMetadata.mapping()); + } catch (IOException e) { + listener.onFailure(e); + return; + } + + // Add the source index name and UUID to the rollup index metadata. If the original index is a rollup index itself, + // we will add the name and UUID of the raw index that we initially rolled up. + String sourceIndexName = IndexMetadata.INDEX_ROLLUP_SOURCE_NAME.exists(originalIndexMetadata.getSettings()) + ? IndexMetadata.INDEX_ROLLUP_SOURCE_NAME.get(originalIndexMetadata.getSettings()) + : originalIndexName; + String sourceIndexUuid = IndexMetadata.INDEX_ROLLUP_SOURCE_UUID.exists(originalIndexMetadata.getSettings()) + ? IndexMetadata.INDEX_ROLLUP_SOURCE_UUID.get(originalIndexMetadata.getSettings()) + : originalIndexMetadata.getIndexUUID(); + + CreateIndexClusterStateUpdateRequest createIndexClusterStateUpdateRequest = new CreateIndexClusterStateUpdateRequest( + "rollup", + tmpIndexName, + tmpIndexName + ).settings(getSettings(originalIndexMetadata, request.getRollupConfig())) + .mappings(XContentHelper.convertToJson(BytesReference.bytes(mapping), false, XContentType.JSON)); + // 2. clusterService.submitStateUpdateTask("rollup create index", new ClusterStateUpdateTask() { @Override @@ -199,20 +216,27 @@ public ClusterState execute(ClusterState currentState) throws Exception { ); } + @Override public void clusterStateProcessed(ClusterState oldState, ClusterState newState) { // index created // 3. + RollupIndexerAction.Request rollupIndexerRequest = new RollupIndexerAction.Request(request); client.execute(RollupIndexerAction.INSTANCE, rollupIndexerRequest, ActionListener.wrap(indexerResp -> { if (indexerResp.isCreated()) { // 4. + UpdateSettingsRequest updateSettingsReq = new UpdateSettingsRequest(WRITE_BLOCKED_SETTINGS, tmpIndexName); client.admin().indices().updateSettings(updateSettingsReq, ActionListener.wrap(updateSettingsResponse -> { if (updateSettingsResponse.isAcknowledged()) { // 5. + ResizeRequest resizeRequest = new ResizeRequest(rollupIndexName, tmpIndexName); + resizeRequest.setResizeType(ResizeType.CLONE); + resizeRequest.getTargetIndexRequest().settings(addRollupSettings(originalIndexMetadata.getSettings())); client.admin().indices().resizeIndex(resizeRequest, ActionListener.wrap(resizeResponse -> { if (resizeResponse.isAcknowledged()) { // 6. publishMetadata(originalIndexName, tmpIndexName, rollupIndexName, listener); } else { + // resize failed deleteTmpIndex( originalIndexName, tmpIndexName, @@ -222,6 +246,7 @@ public void clusterStateProcessed(ClusterState oldState, ClusterState newState) } }, e -> deleteTmpIndex(originalIndexName, tmpIndexName, listener, e))); } else { + // update settings failed deleteTmpIndex( originalIndexName, tmpIndexName, @@ -231,6 +256,7 @@ public void clusterStateProcessed(ClusterState oldState, ClusterState newState) } }, e -> deleteTmpIndex(originalIndexName, tmpIndexName, listener, e))); } else { + // rollup failed deleteTmpIndex( originalIndexName, tmpIndexName, @@ -254,23 +280,257 @@ protected ClusterBlockException checkBlock(RollupAction.Request request, Cluster return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); } - private XContentBuilder getMapping(RollupActionConfig config) throws IOException { + private void publishMetadata( + String originalIndexName, + String tmpIndexName, + String rollupIndexName, + ActionListener listener + ) { + // Update rollup metadata to include this index + clusterService.submitStateUpdateTask("update-rollup-metadata", new ClusterStateUpdateTask() { + @Override + public void clusterStateProcessed(ClusterState oldState, ClusterState newState) { + // Everything went well, time to delete the temporary index + deleteTmpIndex(originalIndexName, tmpIndexName, listener, null); + } + + @Override + public ClusterState execute(ClusterState currentState) { + IndexMetadata rollupIndexMetadata = currentState.getMetadata().index(rollupIndexName); + Index rollupIndex = rollupIndexMetadata.getIndex(); + IndexAbstraction originalIndex = currentState.getMetadata().getIndicesLookup().get(originalIndexName); + + Metadata.Builder metadataBuilder = Metadata.builder(currentState.metadata()); + if (originalIndex.getParentDataStream() != null) { + // If rolling up a backing index of a data stream, add rolled up index to backing data stream + DataStream originalDataStream = originalIndex.getParentDataStream().getDataStream(); + List backingIndices = new ArrayList<>(originalDataStream.getIndices().size() + 1); + // Adding rollup indices to the beginning of the list will prevent rollup indices from ever being + // considered a write index + backingIndices.add(rollupIndex); + backingIndices.addAll(originalDataStream.getIndices()); + DataStream dataStream = new DataStream( + originalDataStream.getName(), + originalDataStream.getTimeStampField(), + backingIndices, + originalDataStream.getGeneration(), + originalDataStream.getMetadata(), + originalDataStream.isHidden(), + originalDataStream.isReplicated(), + originalDataStream.isSystem(), + originalDataStream.isAllowCustomRouting(), + originalDataStream.getIndexMode() + ); + metadataBuilder.put(dataStream); + } + return ClusterState.builder(currentState).metadata(metadataBuilder.build()).build(); + } + + @Override + public void onFailure(Exception e) { + // update rollup metadata failed + // TODO if update rollup metadata failed, the rollup index has created, but maybe not in data stream indices list. + deleteTmpIndex( + originalIndexName, + tmpIndexName, + listener, + new ElasticsearchException("failed to publish new cluster state with rollup metadata", e) + ); + } + }, ClusterStateTaskExecutor.unbatched()); + } + + private void deleteTmpIndex(String originalIndex, String tmpIndex, ActionListener listener, Exception e) { + client.admin().indices().delete(new DeleteIndexRequest(tmpIndex), new ActionListener<>() { + @Override + public void onResponse(AcknowledgedResponse acknowledgedResponse) { + if (e == null && acknowledgedResponse.isAcknowledged()) { + listener.onResponse(acknowledgedResponse); + } else { + listener.onFailure(new RollupExitException("Unable to rollup index [" + originalIndex + "]", e)); + } + } + + @Override + public void onFailure(Exception deleteException) { + listener.onFailure(new RollupExitException("Unable to delete temp rollup index [" + tmpIndex + "]", e)); + } + }); + } + + static Settings getSettings(IndexMetadata originalIndexMetadata, RollupActionConfig rollupActionConfig) { + return Settings.builder() + .put(filterSettings(originalIndexMetadata.getSettings())) + .put(addTimeSeriesSettings(originalIndexMetadata, rollupActionConfig)) + .put(IndexMetadata.SETTING_INDEX_HIDDEN, true) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .build(); + } + + static Settings filterSettings(Settings originalSettings) { + Settings.Builder settings = Settings.builder().put(originalSettings); + + // remove index inner settings + settings.remove(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey()); + settings.remove(IndexMetadata.SETTING_VERSION_CREATED); + settings.remove(IndexMetadata.SETTING_INDEX_UUID); + settings.remove(IndexMetadata.SETTING_HISTORY_UUID); + settings.remove(IndexMetadata.SETTING_INDEX_PROVIDED_NAME); + settings.remove(IndexMetadata.SETTING_CREATION_DATE); + + // remove replica setting + settings.remove(IndexMetadata.SETTING_NUMBER_OF_REPLICAS); + + // remove ilm settings + settings.remove(LifecycleSettings.LIFECYCLE_NAME); + settings.remove(LifecycleSettings.LIFECYCLE_INDEXING_COMPLETE); + + // remove rollup source settings + settings.remove(IndexMetadata.INDEX_ROLLUP_SOURCE_UUID_KEY); + settings.remove(IndexMetadata.INDEX_ROLLUP_SOURCE_NAME_KEY); + + // remove resize settings + settings.remove(IndexMetadata.INDEX_RESIZE_SOURCE_UUID_KEY); + settings.remove(IndexMetadata.INDEX_RESIZE_SOURCE_NAME_KEY); + + // remove time_series settings + settings.remove(IndexSettings.MODE.getKey()); + settings.remove(IndexMetadata.INDEX_ROUTING_PATH.getKey()); + settings.remove(IndexSettings.TIME_SERIES_START_TIME.getKey()); + settings.remove(IndexSettings.TIME_SERIES_END_TIME.getKey()); + + return settings.build(); + } + + /** + * set rollup index to a time_series index + * and deal the rollup index's routing path: + * 1. if the origin index has no routing path, the rollup index will use the group terms field to set index path + * 2. if the rollup group terms contian {@link TimeSeriesIdFieldMapper#NAME} field, + * the rollup index will reuse the origin index's routing path + * 3. if the rollup group terms contain some fields that are not in the origin index's routing path, + * the rollup index will use the group terms field to set index path + * 4. if the rollup group terms contain all the routing path fields, the rollup index will reuse the origin index's routing path + */ + static Settings addTimeSeriesSettings(IndexMetadata originalIndexMetadata, RollupActionConfig config) { + if (config == null) { + return Settings.EMPTY; + } + + RollupActionGroupConfig groupConfig = config.getGroupConfig(); + if (groupConfig == null || groupConfig.getTerms() == null || groupConfig.getDateHistogram() == null) { + return Settings.EMPTY; + } + + // since the timestamp field of time_series index is set to a fixed value, named @timestamp + // so if the rollup date_histogram field is not @timestamp, the rollup index can't set index.mode=time_series, + // because the time_series timestamp field must be @timestamp + if (false == DataStreamTimestampFieldMapper.DEFAULT_PATH.equals(config.getGroupConfig().getDateHistogram().getField())) { + return Settings.EMPTY; + } + + String[] groupTerms = groupConfig.getTerms().getFields(); + Settings defaultTimeSeriesSettings = Settings.builder() + .put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES.name().toLowerCase(Locale.ROOT)) + .putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), groupTerms) + .put(IndexSettings.TIME_SERIES_START_TIME.getKey(), Instant.ofEpochMilli(1).toString()) + .put(IndexSettings.TIME_SERIES_END_TIME.getKey(), Instant.ofEpochMilli(DateUtils.MAX_MILLIS_BEFORE_9999 - 1).toString()) + .build(); + + String indexMode = originalIndexMetadata.getSettings().get(IndexSettings.MODE.getKey()); + if (indexMode == null || false == indexMode.equals(IndexMode.TIME_SERIES.name().toLowerCase(Locale.ROOT))) { + return defaultTimeSeriesSettings; + } + + Set terms = Set.of(groupTerms); + List routingPaths = originalIndexMetadata.getRoutingPaths(); + if (terms.contains(TimeSeriesIdFieldMapper.NAME)) { + return Settings.builder() + .put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES.name().toLowerCase(Locale.ROOT)) + .putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), routingPaths) + .put( + IndexSettings.TIME_SERIES_START_TIME.getKey(), + originalIndexMetadata.getSettings().get(IndexSettings.TIME_SERIES_START_TIME.getKey()) + ) + .put( + IndexSettings.TIME_SERIES_END_TIME.getKey(), + originalIndexMetadata.getSettings().get(IndexSettings.TIME_SERIES_END_TIME.getKey()) + ) + .build(); + } + + List newRoutingPaths = new ArrayList<>(); + boolean routingPathNotInTerms = false; + for (String routingPath : routingPaths) { + if (terms.contains(routingPath)) { + newRoutingPaths.add(routingPath); + } else if (routingPath.contains("*")) { + boolean matchWildCard = false; + for (String term : terms) { + if (Glob.globMatch(routingPath, term)) { + newRoutingPaths.add(routingPath); + matchWildCard = true; + break; + } + } + if (false == matchWildCard) { + routingPathNotInTerms = true; + break; + } + } else { + routingPathNotInTerms = true; + break; + } + } + + if (routingPathNotInTerms) { + return defaultTimeSeriesSettings; + } + + return Settings.builder() + .put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES.name().toLowerCase(Locale.ROOT)) + .putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), newRoutingPaths) + .put( + IndexSettings.TIME_SERIES_START_TIME.getKey(), + originalIndexMetadata.getSettings().get(IndexSettings.TIME_SERIES_START_TIME.getKey()) + ) + .put( + IndexSettings.TIME_SERIES_END_TIME.getKey(), + originalIndexMetadata.getSettings().get(IndexSettings.TIME_SERIES_END_TIME.getKey()) + ) + .build(); + } + + static XContentBuilder getMapping(RollupActionConfig config, MappingMetadata mappingMetadata) throws IOException { XContentBuilder builder = XContentFactory.jsonBuilder().startObject(); - builder = getDynamicTemplates(builder); - builder = getProperties(builder, config); + if (mappingMetadata != null) { + getMetadataMapping(builder, mappingMetadata.getSourceAsMap()); + } + getDynamicTemplates(builder); + getProperties(builder, config); return builder.endObject(); } + /** + * TODO: now we add origin index's _source mapping, maybe we can add more metadata mapping + */ + private static void getMetadataMapping(XContentBuilder builder, Map mappings) throws IOException { + if (mappings != null && mappings.containsKey(SourceFieldMapper.NAME)) { + builder.field(SourceFieldMapper.NAME, mappings.get(SourceFieldMapper.NAME)); + } + } + /** * Configure the dynamic templates to always map strings to the keyword field type. */ - private static XContentBuilder getDynamicTemplates(XContentBuilder builder) throws IOException { - return builder.startArray("dynamic_templates") + private static void getDynamicTemplates(XContentBuilder builder) throws IOException { + builder.startArray("dynamic_templates") .startObject() .startObject("strings") .field("match_mapping_type", "string") .startObject("mapping") .field("type", "keyword") + .field("time_series_dimension", "true") .endObject() .endObject() .endObject() @@ -280,7 +540,7 @@ private static XContentBuilder getDynamicTemplates(XContentBuilder builder) thro /** * Creates the rollup mapping properties from the provided {@link RollupActionConfig}. */ - private static XContentBuilder getProperties(XContentBuilder builder, RollupActionConfig config) throws IOException { + private static void getProperties(XContentBuilder builder, RollupActionConfig config) throws IOException { builder.startObject("properties"); RollupActionGroupConfig groupConfig = config.getGroupConfig(); @@ -312,9 +572,19 @@ private static XContentBuilder getProperties(XContentBuilder builder, RollupActi } } + TermsGroupConfig termsGroupConfig = groupConfig.getTerms(); + if (termsGroupConfig != null) { + for (String field : termsGroupConfig.getFields()) { + if (field.equals(TimeSeriesIdFieldMapper.NAME)) { + continue; + } + builder.startObject(field).field("type", "keyword").field("time_series_dimension", "true").endObject(); + } + } + List metricConfigs = config.getMetricsConfig(); for (MetricConfig metricConfig : metricConfigs) { - List metrics = FieldMetricsProducer.normalizeMetrics(metricConfig.getMetrics()); + List metrics = MetricField.normalizeMetrics(metricConfig.getMetrics()); String defaultMetric = metrics.contains("value_count") ? "value_count" : metrics.get(0); builder.startObject(metricConfig.getField()) .field("type", AggregateDoubleMetricFieldMapper.CONTENT_TYPE) @@ -323,82 +593,135 @@ private static XContentBuilder getProperties(XContentBuilder builder, RollupActi .endObject(); } - return builder.endObject(); + builder.endObject(); } - private void publishMetadata( - String originalIndexName, - String tmpIndexName, - String rollupIndexName, - ActionListener listener - ) { - // Update rollup metadata to include this index - clusterService.submitStateUpdateTask("update-rollup-metadata", new ClusterStateUpdateTask() { - @Override - public void clusterStateProcessed(ClusterState oldState, ClusterState newState) { - // Everything went well, time to delete the temporary index - deleteTmpIndex(originalIndexName, tmpIndexName, listener, null); + private static void rebuildRollupField(String field, Map> fieldCaps, Consumer consumer) { + Predicate> predicate = (fieldCap) -> { + boolean isValueType = true; + for (Map.Entry entry : fieldCap.entrySet()) { + String fieldType = entry.getKey(); + if (fieldType.equals(ObjectMapper.CONTENT_TYPE) || fieldType.equals(NestedObjectMapper.CONTENT_TYPE)) { + isValueType = false; + break; + } } + return isValueType; + }; + if (field.contains("*")) { + fieldCaps.forEach((key, value) -> { + if (Glob.globMatch(field, key) && predicate.test(value)) { + consumer.accept(key); + } + }); + } else { + consumer.accept(field); + } + } - @Override - public ClusterState execute(ClusterState currentState) { - IndexMetadata rollupIndexMetadata = currentState.getMetadata().index(rollupIndexName); - Index rollupIndex = rollupIndexMetadata.getIndex(); - IndexAbstraction originalIndex = currentState.getMetadata().getIndicesLookup().get(originalIndexName); + private static Set rebuildRollupFields(List groupFields, Map> fieldCaps) { + long wildcardCount = groupFields.stream().filter(field -> field.contains("*")).count(); + // no wildcard fields + if (wildcardCount == 0) { + Set.of(groupFields); + } - Metadata.Builder metadataBuilder = Metadata.builder(currentState.metadata()); - if (originalIndex.getParentDataStream() != null) { - // If rolling up a backing index of a data stream, add rolled up index to backing data stream - DataStream originalDataStream = originalIndex.getParentDataStream().getDataStream(); - List backingIndices = new ArrayList<>(originalDataStream.getIndices().size() + 1); - // Adding rollup indices to the beginning of the list will prevent rollup indices from ever being - // considered a write index - backingIndices.add(rollupIndex); - backingIndices.addAll(originalDataStream.getIndices()); - DataStream dataStream = new DataStream( - originalDataStream.getName(), - originalDataStream.getTimeStampField(), - backingIndices, - originalDataStream.getGeneration(), - originalDataStream.getMetadata(), - originalDataStream.isHidden(), - originalDataStream.isReplicated(), - originalDataStream.isSystem(), - originalDataStream.isAllowCustomRouting(), - originalDataStream.getIndexMode() - ); - metadataBuilder.put(dataStream); - } - return ClusterState.builder(currentState).metadata(metadataBuilder.build()).build(); + Set newGroupFields = new LinkedHashSet<>(); + for (String field : groupFields) { + rebuildRollupField(field, fieldCaps, (config) -> newGroupFields.add(config)); + } + return newGroupFields; + } + + private static RollupActionGroupConfig rebuildRollupGroupConfig( + RollupActionGroupConfig groupConfig, + Map> fieldCaps + ) { + TermsGroupConfig termsGroupConfig; + if (groupConfig.getTerms() != null) { + Set groupFields = rebuildRollupFields(List.of(groupConfig.getTerms().getFields()), fieldCaps); + if (groupFields.size() == 0) { + throw new IllegalArgumentException( + "Could not find a field match the group terms " + List.of(groupConfig.getTerms().getFields()) + ); } + termsGroupConfig = new TermsGroupConfig(groupFields.toArray(Strings.EMPTY_ARRAY)); + } else { + termsGroupConfig = null; + } - @Override - public void onFailure(Exception e) { - deleteTmpIndex( - originalIndexName, - tmpIndexName, - listener, - new ElasticsearchException("failed to publish new cluster state with rollup metadata", e) + HistogramGroupConfig histogramGroupConfig; + if (groupConfig.getHistogram() != null) { + Set groupFields = rebuildRollupFields(List.of(groupConfig.getHistogram().getFields()), fieldCaps); + if (groupFields.size() == 0) { + throw new IllegalArgumentException( + "Could not find a field match the group histograms " + List.of(groupConfig.getHistogram().getFields()) ); } - }, ClusterStateTaskExecutor.unbatched()); + histogramGroupConfig = new HistogramGroupConfig( + groupConfig.getHistogram().getInterval(), + groupFields.toArray(Strings.EMPTY_ARRAY) + ); + } else { + histogramGroupConfig = null; + } + + return new RollupActionGroupConfig(groupConfig.getDateHistogram(), histogramGroupConfig, termsGroupConfig); } - private void deleteTmpIndex(String originalIndex, String tmpIndex, ActionListener listener, Exception e) { - client.admin().indices().delete(new DeleteIndexRequest(tmpIndex), new ActionListener<>() { - @Override - public void onResponse(AcknowledgedResponse acknowledgedResponse) { - if (e == null && acknowledgedResponse.isAcknowledged()) { - listener.onResponse(acknowledgedResponse); - } else { - listener.onFailure(new ElasticsearchException("Unable to rollup index [" + originalIndex + "]", e)); - } - } + private static List rebuildRollupMetricConfig( + List metricConfigs, + Map> fieldCaps + ) { + long wildcardCount = metricConfigs.stream().filter(metric -> metric.getField().contains("*")).count(); + // no wildcard fields + if (wildcardCount == 0) { + return metricConfigs; + } - @Override - public void onFailure(Exception deleteException) { - listener.onFailure(new ElasticsearchException("Unable to delete temp rollup index [" + tmpIndex + "]", e)); - } - }); + List newMetricConfigs = new ArrayList<>(); + for (MetricConfig metricConfig : metricConfigs) { + String field = metricConfig.getField(); + rebuildRollupField(field, fieldCaps, (config) -> newMetricConfigs.add(new MetricConfig(config, metricConfig.getMetrics()))); + } + if (newMetricConfigs.size() == 0) { + throw new IllegalArgumentException( + "Could not find a field match the metric fields " + + metricConfigs.stream().map(MetricConfig::getField).collect(Collectors.toSet()) + ); + } + return newMetricConfigs; + } + + static RollupActionConfig rebuildRollupConfig(RollupActionConfig config, Map> fieldCaps) { + RollupActionGroupConfig groupConfig = rebuildRollupGroupConfig(config.getGroupConfig(), fieldCaps); + List metricConfigs = rebuildRollupMetricConfig(config.getMetricsConfig(), fieldCaps); + return new RollupActionConfig(groupConfig, metricConfigs); + } + + static Settings addRollupSettings(Settings originalSettings) { + Settings.Builder builder = Settings.builder().put(IndexMetadata.SETTING_INDEX_HIDDEN, true); + if (originalSettings.hasValue(LifecycleSettings.LIFECYCLE_NAME)) { + builder.put(LifecycleSettings.LIFECYCLE_NAME, originalSettings.get(LifecycleSettings.LIFECYCLE_NAME)); + } + if (originalSettings.hasValue(LifecycleSettings.LIFECYCLE_INDEXING_COMPLETE)) { + builder.put(LifecycleSettings.LIFECYCLE_INDEXING_COMPLETE, originalSettings.get(LifecycleSettings.LIFECYCLE_INDEXING_COMPLETE)); + } + + return builder.build(); + } + + private void validExistsRollupIndex(ClusterState state, String rollupIndex) { + if (state.metadata().hasIndex(rollupIndex)) { + throw new InvalidIndexNameException(rollupIndex, "rollup index already exists"); + } + + if (state.metadata().hasAlias(rollupIndex)) { + throw new InvalidIndexNameException(rollupIndex, "rollup index already exists as alias"); + } + + if (state.metadata().dataStreams().containsKey(rollupIndex)) { + throw new InvalidIndexNameException(rollupIndex, "rollup index already exists as data stream"); + } } } diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/TransportRollupIndexerAction.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/TransportRollupIndexerAction.java index 4e2e68732417e..8cdc9892180d5 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/TransportRollupIndexerAction.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/TransportRollupIndexerAction.java @@ -20,23 +20,30 @@ import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.Randomness; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.index.IndexMode; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.mapper.DataStreamTimestampFieldMapper; +import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.tasks.Task; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.core.ClientHelper; +import org.elasticsearch.xpack.core.rollup.RollupActionGroupConfig; import org.elasticsearch.xpack.core.rollup.action.RollupIndexerAction; +import org.elasticsearch.xpack.core.rollup.action.RollupShardStatus; +import org.elasticsearch.xpack.core.rollup.job.TermsGroupConfig; +import org.elasticsearch.xpack.rollup.v2.indexer.TimeSeriesRollupShardIndexer; +import org.elasticsearch.xpack.rollup.v2.indexer.UnSortedRollupShardIndexer; import java.io.IOException; import java.util.Arrays; -import java.util.Map; -import java.util.Set; import java.util.concurrent.atomic.AtomicReferenceArray; import static org.elasticsearch.xpack.rollup.Rollup.TASK_THREAD_POOL_NAME; +import static org.elasticsearch.xpack.rollup.v2.TransportRollupAction.TMP_INDEX_PREFIX; /** * A {@link TransportBroadcastAction} that rollups all the shards of a single index into a new one. @@ -86,12 +93,9 @@ protected GroupShardsIterator shards( String[] concreteIndices ) { if (concreteIndices.length > 1) { - throw new IllegalArgumentException("multiple indices: " + Arrays.toString(concreteIndices)); + throw new IllegalArgumentException("multiple indices: [" + Arrays.toString(concreteIndices) + "] not allowed"); } - // Random routing to limit request to a single shard - String routing = Integer.toString(Randomness.get().nextInt(1000)); - Map> routingMap = indexNameExpressionResolver.resolveSearchRouting(clusterState, routing, request.indices()); - return clusterService.operationRouting().searchShards(clusterState, concreteIndices, routingMap, null); + return clusterService.operationRouting().searchShards(clusterState, concreteIndices, null, null); } @Override @@ -117,19 +121,63 @@ protected RollupIndexerAction.ShardRequest newShardRequest(int numShards, ShardR @Override protected RollupIndexerAction.ShardResponse shardOperation(RollupIndexerAction.ShardRequest request, Task task) throws IOException { IndexService indexService = indicesService.indexService(request.shardId().getIndex()); - String tmpIndexName = ".rolluptmp-" + request.getRollupIndex(); - RollupShardIndexer indexer = new RollupShardIndexer( - client, - indexService, - request.shardId(), - request.getRollupConfig(), - tmpIndexName, - SORTER_RAM_SIZE_MB - ); - indexer.execute(); + String tmpIndexName = TMP_INDEX_PREFIX + request.getRollupIndex(); + if (isRollupTimeSeries(indexService.getIndexSettings(), request.getRollupConfig().getGroupConfig())) { + new TimeSeriesRollupShardIndexer( + (RollupShardStatus) task.getStatus(), + client, + indexService, + request.shardId(), + request.getRollupConfig(), + tmpIndexName + ).execute(); + } else { + new UnSortedRollupShardIndexer( + (RollupShardStatus) task.getStatus(), + client, + indexService, + request.shardId(), + request.getRollupConfig(), + tmpIndexName, + SORTER_RAM_SIZE_MB + ).execute(); + } return new RollupIndexerAction.ShardResponse(request.shardId()); } + /** + * if rollup a time_series, and the rollup group are _tsid and @timestamp, it can use a sorted mode to rollup data + * + * TODO: + * if rollup config match the index sorting config, it can use a sorted mode to rollup data. + * but if the index sorting field is a multi value field, it's not a real sorting mode. + * when index setting support setting single value field, the method can change to rollup the sorted data + */ + static boolean isRollupTimeSeries(IndexSettings indexSettings, RollupActionGroupConfig groupConfig) { + if (groupConfig == null) { + return false; + } + + if (indexSettings.getMode() != IndexMode.TIME_SERIES) { + return false; + } + + if (false == groupConfig.getDateHistogram().getField().equals(DataStreamTimestampFieldMapper.DEFAULT_PATH)) { + return false; + } + + if (groupConfig.getHistogram() != null) { + return false; + } + + TermsGroupConfig terms = groupConfig.getTerms(); + if (terms != null && terms.getFields().length == 1 && terms.getFields()[0].equals(TimeSeriesIdFieldMapper.NAME)) { + return true; + } + + return false; + } + @Override protected RollupIndexerAction.ShardResponse readShardResponse(StreamInput in) throws IOException { return new RollupIndexerAction.ShardResponse(in); diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/CompressingOfflineSorter.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/CompressingOfflineSorter.java similarity index 99% rename from x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/CompressingOfflineSorter.java rename to x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/CompressingOfflineSorter.java index e95b989158d3f..e8e828e110ec3 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/CompressingOfflineSorter.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/CompressingOfflineSorter.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.xpack.rollup.v2; +package org.elasticsearch.xpack.rollup.v2.indexer; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.store.ChecksumIndexInput; diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/TimeSeriesRollupShardIndexer.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/TimeSeriesRollupShardIndexer.java new file mode 100644 index 0000000000000..a2d358416ff60 --- /dev/null +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/TimeSeriesRollupShardIndexer.java @@ -0,0 +1,181 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.rollup.v2.indexer; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.ScoreMode; +import org.elasticsearch.client.internal.Client; +import org.elasticsearch.common.util.CancellableThreads.ExecutionCancelledException; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.fielddata.FormattedDocValues; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.search.aggregations.BucketCollector; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.bucket.DocCountProvider; +import org.elasticsearch.search.aggregations.timeseries.TimeSeriesIndexSearcher; +import org.elasticsearch.xpack.core.rollup.RollupActionConfig; +import org.elasticsearch.xpack.core.rollup.action.RollupShardStatus; +import org.elasticsearch.xpack.core.rollup.action.RollupShardStatus.Status; +import org.elasticsearch.xpack.rollup.v2.RollupShardIndexer; +import org.elasticsearch.xpack.rollup.v2.indexer.metrics.LeafMetricField; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +/** + * rollup data in sort mode + */ +public class TimeSeriesRollupShardIndexer extends RollupShardIndexer { + private static final Logger logger = LogManager.getLogger(TimeSeriesRollupShardIndexer.class); + + public TimeSeriesRollupShardIndexer( + RollupShardStatus rollupShardStatus, + Client client, + IndexService indexService, + ShardId shardId, + RollupActionConfig config, + String tmpIndex + ) { + super(rollupShardStatus, client, indexService, shardId, config, tmpIndex); + } + + @Override + public void execute() throws IOException { + long start = System.currentTimeMillis(); + try (searcher; bulkProcessor) { + TimeSeriesIndexSearcher timeSeriesIndexSearcher = new TimeSeriesIndexSearcher(searcher); + TimeSeriesCollector timeSeriesCollector = new TimeSeriesCollector(); + timeSeriesCollector.preCollection(); + timeSeriesIndexSearcher.search(new MatchAllDocsQuery(), timeSeriesCollector); + timeSeriesCollector.postCollection(); + bulkProcessor.flush(); + } + + if (status.getStatus() == Status.ABORT) { + logger.warn( + "[{}] rolling abort, sent [{}], indexed [{}], failed[{}]", + indexShard.shardId(), + numIndexed.get(), + numIndexed.get(), + numFailed.get() + ); + throw new ExecutionCancelledException("[" + indexShard.shardId() + "] rollup cancelled"); + } + + logger.info( + "sorted rollup execute [{}], cost [{}], Received [{}], Skip [{}], Successfully sent [{}], indexed [{}], failed[{}]", + indexShard.shardId(), + (System.currentTimeMillis() - start), + numReceived.get(), + numSkip.get(), + numSent.get(), + numIndexed.get(), + numFailed.get() + ); + status.setStatus(Status.STOP); + return; + } + + private class TimeSeriesCollector extends BucketCollector { + final AtomicReference currentKey = new AtomicReference<>(); + final AtomicInteger docCount = new AtomicInteger(0); + final AtomicInteger keyCount = new AtomicInteger(0); + final AtomicLong nextBucket = new AtomicLong(Long.MIN_VALUE); + + @Override + public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException { + final FormattedDocValues timestampField = timestampFetcher.getGroupLeaf(ctx); + final FormattedDocValues[] groupFieldLeaf = leafGroupFetchers(ctx); + final LeafMetricField[] metricsFieldLeaf = leafMetricFields(ctx); + DocCountProvider docCountProvider = new DocCountProvider(); + docCountProvider.setLeafReaderContext(ctx); + + return new LeafBucketCollector() { + @Override + public void collect(int doc, long owningBucketOrd) throws IOException { + if (isCanceled()) { + // TODO check if this can throw cancel exception + return; + } + + numReceived.incrementAndGet(); + Long timestamp = null; + if (timestampField.advanceExact(doc)) { + Object obj = timestampField.nextValue(); + if (obj instanceof Number == false) { + throw new IllegalArgumentException("Expected [Number], got [" + obj.getClass() + "]"); + } + timestamp = ((Number) obj).longValue(); + } + + if (timestamp == null) { + logger.trace("[{}] timestamp missing, bucket ord [{}], docId [{}]", indexShard.shardId(), owningBucketOrd, doc); + numSkip.incrementAndGet(); + return; + } + + List groupFields = new ArrayList<>(); + for (FormattedDocValues leafField : groupFieldLeaf) { + if (leafField.advanceExact(doc)) { + groupFields.add(leafField.nextValue()); + } + } + + if (currentKey.get() != null + && (false == Objects.equals(currentKey.get().getGroupFields(), groupFields) + || timestamp >= nextBucket.get() + || timestamp < currentKey.get().getTimestamp())) { + indexBucket(currentKey.get(), docCount.get()); + keyCount.incrementAndGet(); + // reset + currentKey.set(null); + docCount.set(0); + resetMetricCollectors(); + } + + if (currentKey.get() == null) { + Long currentBucket = rounding.round(timestamp); + nextBucket.set(rounding.nextRoundingValue(currentBucket)); + currentKey.set(new BucketKey(currentBucket, groupFields)); + } + + docCount.addAndGet(docCountProvider.getDocCount(doc)); + for (LeafMetricField metricField : metricsFieldLeaf) { + metricField.collectMetric(doc); + } + } + }; + } + + @Override + public void preCollection() throws IOException { + + } + + @Override + public void postCollection() throws IOException { + if (currentKey.get() != null) { + indexBucket(currentKey.get(), docCount.get()); + keyCount.incrementAndGet(); + } + } + + @Override + public ScoreMode scoreMode() { + return ScoreMode.COMPLETE_NO_SCORES; + } + } +} diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/UnSortedRollupShardIndexer.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/UnSortedRollupShardIndexer.java new file mode 100644 index 0000000000000..a4fd7bb09f95f --- /dev/null +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/UnSortedRollupShardIndexer.java @@ -0,0 +1,408 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.rollup.v2.indexer; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.PointValues; +import org.apache.lucene.search.CollectionTerminatedException; +import org.apache.lucene.search.Collector; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.Scorable; +import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FilterDirectory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefIterator; +import org.elasticsearch.client.internal.Client; +import org.elasticsearch.common.Rounding; +import org.elasticsearch.common.io.stream.ByteBufferStreamInput; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.util.CancellableThreads.ExecutionCancelledException; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.fielddata.FormattedDocValues; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.search.aggregations.bucket.DocCountProvider; +import org.elasticsearch.xpack.core.rollup.RollupActionConfig; +import org.elasticsearch.xpack.core.rollup.action.RollupShardStatus; +import org.elasticsearch.xpack.core.rollup.action.RollupShardStatus.Status; +import org.elasticsearch.xpack.rollup.v2.RollupShardIndexer; +import org.elasticsearch.xpack.rollup.v2.indexer.metrics.LeafMetricField; +import org.elasticsearch.xpack.rollup.v2.indexer.metrics.MetricField; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * rollup data in unsorted mode + */ +public class UnSortedRollupShardIndexer extends RollupShardIndexer { + private static final Logger logger = LogManager.getLogger(UnSortedRollupShardIndexer.class); + + private final Directory dir; + private final CompressingOfflineSorter sorter; + + // for testing + public final Set tmpFiles = new HashSet<>(); + public final Set tmpFilesDeleted = new HashSet<>(); + + public UnSortedRollupShardIndexer( + RollupShardStatus rollupShardStatus, + Client client, + IndexService indexService, + ShardId shardId, + RollupActionConfig config, + String tmpIndex, + int ramBufferSizeMB + ) { + super(rollupShardStatus, client, indexService, shardId, config, tmpIndex); + + Closeable toClose = searcher; + try { + this.dir = new FilterDirectory(searcher.getDirectoryReader().directory()) { + @Override + public IndexOutput createOutput(String name, IOContext context) throws IOException { + tmpFiles.add(name); + return super.createOutput(name, context); + } + + @Override + public IndexOutput createTempOutput(String prefix, String suffix, IOContext context) throws IOException { + IndexOutput output = super.createTempOutput(prefix, suffix, context); + tmpFiles.add(output.getName()); + return output; + } + + @Override + public void deleteFile(String name) throws IOException { + tmpFilesDeleted.add(name); + super.deleteFile(name); + } + }; + this.sorter = new CompressingOfflineSorter(dir, "rollup-", keyComparator(), ramBufferSizeMB); + toClose = null; + } finally { + IOUtils.closeWhileHandlingException(toClose); + } + } + + @Override + public void execute() throws IOException { + Long bucket = Long.MIN_VALUE; + long count = 0; + long start = System.currentTimeMillis(); + try (searcher; bulkProcessor) { + do { + if (isCanceled()) { + logger.warn( + "[{}] rolling abort, sent [{}], indexed [{}], failed[{}]", + indexShard.shardId(), + numIndexed.get(), + numIndexed.get(), + numFailed.get() + ); + throw new ExecutionCancelledException("[" + indexShard.shardId() + "] rollup cancelled"); + } + + long startCompute = System.currentTimeMillis(); + bucket = computeBucket(bucket); + logger.debug( + "[{}] rollup, current computeBucket cost:{}, bucket:{}, loop:{}", + indexShard.shardId(), + (System.currentTimeMillis() - startCompute), + bucket, + ++count + ); + } while (bucket != null); + + bulkProcessor.flush(); + } + + logger.info( + "sorted rollup execute [{}], cost [{}], Received [{}], Skip [{}], Successfully sent [{}], indexed [{}], failed[{}]", + indexShard.shardId(), + (System.currentTimeMillis() - start), + numReceived.get(), + numSkip.get(), + numSent.get(), + numIndexed.get(), + numFailed.get() + ); + status.setStatus(Status.STOP); + return; + } + + private Long computeBucket(long lastRounding) throws IOException { + Long nextRounding = findNextRounding(lastRounding); + if (nextRounding == null) { + return null; + } + long nextRoundingLastValue = rounding.nextRoundingValue(nextRounding) - 1; + try (XExternalRefSorter externalSorter = new XExternalRefSorter(sorter)) { + long start = System.currentTimeMillis(); + Query rangeQuery = LongPoint.newRangeQuery(timestampFetcher.getFieldType().name(), nextRounding, nextRoundingLastValue); + searcher.search(rangeQuery, new BucketCollector(nextRounding, externalSorter)); + long searchTime = System.currentTimeMillis(); + logger.debug("current round [{}], search cost [{}]", nextRounding, (searchTime - start)); + + BytesRefIterator it = externalSorter.iterator(); + BytesRef next = it.next(); + + BucketKey lastKey = null; + int docCount = 0; + int keyCount = 0; + int totalDocCount = 0; + while (next != null) { + try (StreamInput in = new ByteBufferStreamInput(ByteBuffer.wrap(next.bytes, next.offset, next.length))) { + // skip key size + in.readInt(); + BucketKey key = decodeKey(in); + totalDocCount++; + if (lastKey != null && lastKey.equals(key) == false) { + indexBucket(lastKey, docCount); + keyCount++; + docCount = 0; + resetMetricCollectors(); + } + for (MetricField metricField : metricFields) { + metricField.collectMetric(in); + } + docCount += in.readVInt(); + lastKey = key; + } + next = it.next(); + } + if (lastKey != null) { + indexBucket(lastKey, docCount); + keyCount++; + resetMetricCollectors(); + } + + logger.debug( + "current round [{}], doc count [{}], keyCount [{}], build index data cost [{}]", + nextRounding, + totalDocCount, + keyCount, + (System.currentTimeMillis() - searchTime) + ); + } + return nextRoundingLastValue; + } + + private Long findNextRounding(long lastRounding) throws IOException { + Long nextRounding = null; + for (LeafReaderContext leafReaderContext : searcher.getIndexReader().leaves()) { + PointValues pointValues = leafReaderContext.reader().getPointValues(timestampFetcher.getFieldType().name()); + if (pointValues == null) { + continue; + } + final NextRoundingVisitor visitor = new NextRoundingVisitor(rounding, lastRounding); + try { + pointValues.intersect(visitor); + } catch (CollectionTerminatedException exc) {} + if (visitor.nextRounding != null) { + nextRounding = nextRounding == null ? visitor.nextRounding : Math.min(nextRounding, visitor.nextRounding); + } + } + return nextRounding; + } + + private static BytesRef encodeKey(long timestamp, List groupFields) throws IOException { + try (BytesStreamOutput out = new BytesStreamOutput()) { + out.writeLong(timestamp); + out.writeVInt(groupFields.size()); + for (Object obj : groupFields) { + out.writeGenericValue(obj); + } + return out.bytes().toBytesRef(); + } + } + + private static BucketKey decodeKey(StreamInput in) throws IOException { + long timestamp = in.readLong(); + int numGroupFields = in.readVInt(); + List values = new ArrayList<>(); + for (int i = 0; i < numGroupFields; i++) { + values.add(in.readGenericValue()); + } + return new BucketKey(timestamp, values); + } + + /** + * Returns a {@link Comparator} that can be used to sort inputs created by the {@link BucketCollector}. + * We just want identical buckets to be consecutive for the merge so this comparator doesn't follow the natural + * order and simply checks for identical binary keys. + */ + private static Comparator keyComparator() { + return (o1, o2) -> { + int keySize1 = readInt(o1.bytes, o1.offset); + int keySize2 = readInt(o2.bytes, o2.offset); + return Arrays.compareUnsigned( + o1.bytes, + o1.offset + Integer.BYTES, + keySize1 + o1.offset + Integer.BYTES, + o2.bytes, + o2.offset + Integer.BYTES, + keySize2 + o2.offset + Integer.BYTES + ); + }; + } + + private static int readInt(byte[] bytes, int offset) { + return ((bytes[offset] & 0xFF) << 24) | ((bytes[offset + 1] & 0xFF) << 16) | ((bytes[offset + 2] & 0xFF) << 8) | (bytes[offset + 3] + & 0xFF); + } + + private class BucketCollector implements Collector { + private final long timestamp; + private final XExternalRefSorter externalSorter; + + private BucketCollector(long timestamp, XExternalRefSorter externalSorter) { + this.externalSorter = externalSorter; + this.timestamp = timestamp; + } + + @Override + public LeafCollector getLeafCollector(LeafReaderContext context) { + final FormattedDocValues[] leafGroupFetchers = leafGroupFetchers(context); + final LeafMetricField[] leafMetricFields = leafMetricFields(context); + final DocCountProvider docCountProvider = new DocCountProvider(); + return new LeafCollector() { + @Override + public void setScorer(Scorable scorer) {} + + @Override + public void collect(int docID) throws IOException { + if (isCanceled()) { + // TODO check if this can throw cancel exception + return; + } + + numReceived.incrementAndGet(); + docCountProvider.setLeafReaderContext(context); + List> combinationKeys = new ArrayList<>(); + for (FormattedDocValues leafField : leafGroupFetchers) { + if (leafField.advanceExact(docID)) { + List lst = new ArrayList<>(); + for (int i = 0; i < leafField.docValueCount(); i++) { + lst.add(leafField.nextValue()); + } + combinationKeys.add(lst); + } else { + combinationKeys.add(null); + } + } + + final BytesRef valueBytes; + try (BytesStreamOutput out = new BytesStreamOutput()) { + for (LeafMetricField leafMetricField : leafMetricFields) { + leafMetricField.writeMetrics(docID, out); + } + valueBytes = out.bytes().toBytesRef(); + } + for (List groupFields : cartesianProduct(combinationKeys)) { + try (BytesStreamOutput out = new BytesStreamOutput()) { + BytesRef keyBytes = encodeKey(timestamp, groupFields); + out.writeInt(keyBytes.length); + out.writeBytes(keyBytes.bytes, keyBytes.offset, keyBytes.length); + out.writeBytes(valueBytes.bytes, valueBytes.offset, valueBytes.length); + out.writeVInt(docCountProvider.getDocCount(docID)); + externalSorter.add(out.bytes().toBytesRef()); + } + } + } + }; + } + + @Override + public ScoreMode scoreMode() { + return ScoreMode.COMPLETE_NO_SCORES; + } + } + + private class NextRoundingVisitor implements PointValues.IntersectVisitor { + final Rounding.Prepared rounding; + final long lastRounding; + + Long nextRounding = null; + + NextRoundingVisitor(Rounding.Prepared rounding, long lastRounding) { + this.rounding = rounding; + this.lastRounding = lastRounding; + } + + @Override + public void visit(int docID) { + throw new IllegalStateException("should never be called"); + } + + @Override + public void visit(DocIdSetIterator iterator, byte[] packedValue) { + long bucket = rounding.round(LongPoint.decodeDimension(packedValue, 0)); + checkMinRounding(bucket); + } + + @Override + public void visit(int docID, byte[] packedValue) { + long bucket = rounding.round(LongPoint.decodeDimension(packedValue, 0)); + checkMinRounding(bucket); + } + + @Override + public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) { + long maxRounding = rounding.round(LongPoint.decodeDimension(maxPackedValue, 0)); + if (maxRounding <= lastRounding) { + return PointValues.Relation.CELL_OUTSIDE_QUERY; + } + long minRounding = rounding.round(LongPoint.decodeDimension(minPackedValue, 0)); + checkMinRounding(minRounding); + return PointValues.Relation.CELL_CROSSES_QUERY; + } + + private void checkMinRounding(long roundingValue) { + if (roundingValue > lastRounding) { + nextRounding = roundingValue; + throw new CollectionTerminatedException(); + } + } + } + + private static List> cartesianProduct(List> lists) { + List> combinations = Arrays.asList(Arrays.asList()); + for (List list : lists) { + if (list == null) { + continue; + } + List> extraColumnCombinations = new ArrayList<>(); + for (List combination : combinations) { + for (Object element : list) { + List newCombination = new ArrayList<>(combination); + newCombination.add(element); + extraColumnCombinations.add(newCombination); + } + } + combinations = extraColumnCombinations; + } + return combinations; + } +} diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/XExternalRefSorter.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/XExternalRefSorter.java similarity index 99% rename from x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/XExternalRefSorter.java rename to x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/XExternalRefSorter.java index c08a35886062d..b875531a03631 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/XExternalRefSorter.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/XExternalRefSorter.java @@ -17,7 +17,7 @@ * * Modifications copyright (C) 2020 Elasticsearch B.V. */ -package org.elasticsearch.xpack.rollup.v2; +package org.elasticsearch.xpack.rollup.v2.indexer; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.search.suggest.fst.BytesRefSorter; diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/metrics/LeafMetricField.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/metrics/LeafMetricField.java new file mode 100644 index 0000000000000..b99c7a7fb39a0 --- /dev/null +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/metrics/LeafMetricField.java @@ -0,0 +1,24 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.rollup.v2.indexer.metrics; + +import org.elasticsearch.common.io.stream.BytesStreamOutput; + +import java.io.IOException; + +public abstract class LeafMetricField { + protected final MetricCollector[] metricCollectors; + + public LeafMetricField(MetricCollector[] metricCollectors) { + this.metricCollectors = metricCollectors; + } + + public abstract void collectMetric(int docID) throws IOException; + + public abstract void writeMetrics(int docID, BytesStreamOutput out) throws IOException; +} diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/metrics/MetricCollector.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/metrics/MetricCollector.java new file mode 100644 index 0000000000000..7bd00ceb0a274 --- /dev/null +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/metrics/MetricCollector.java @@ -0,0 +1,115 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.rollup.v2.indexer.metrics; + +public abstract class MetricCollector { + public final String name; + + protected MetricCollector(String name) { + this.name = name; + } + + public abstract void collect(double number); + + public abstract Number get(); + + public abstract void reset(); + + public static class Max extends MetricCollector { + private Double max; + + public Max() { + super("max"); + } + + @Override + public void collect(double value) { + this.max = max != null ? Math.max(value, max) : value; + } + + @Override + public Number get() { + return max; + } + + @Override + public void reset() { + max = null; + } + } + + public static class Min extends MetricCollector { + private Double min; + + public Min() { + super("min"); + } + + @Override + public void collect(double value) { + this.min = min != null ? Math.min(value, min) : value; + } + + @Override + public Number get() { + return min; + } + + @Override + public void reset() { + min = null; + } + } + + public static class Sum extends MetricCollector { + private double sum = 0; + + public Sum() { + super("sum"); + } + + @Override + public void collect(double value) { + // TODO: switch to Kahan summation ? + this.sum += value; + } + + @Override + public Number get() { + return sum; + } + + @Override + public void reset() { + sum = 0; + } + } + + public static class ValueCount extends MetricCollector { + private long count; + + public ValueCount() { + super("value_count"); + } + + @Override + public void collect(double value) { + count++; + } + + @Override + public Number get() { + return count; + } + + @Override + public void reset() { + count = 0; + } + } +} diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/metrics/MetricField.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/metrics/MetricField.java new file mode 100644 index 0000000000000..e4d8dfecd0999 --- /dev/null +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/metrics/MetricField.java @@ -0,0 +1,99 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.rollup.v2.indexer.metrics; + +import org.apache.lucene.index.LeafReaderContext; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.xpack.core.rollup.job.MetricConfig; +import org.elasticsearch.xpack.rollup.v2.FieldValueFetcher; +import org.elasticsearch.xpack.rollup.v2.indexer.metrics.MetricCollector.Max; +import org.elasticsearch.xpack.rollup.v2.indexer.metrics.MetricCollector.Min; +import org.elasticsearch.xpack.rollup.v2.indexer.metrics.MetricCollector.Sum; +import org.elasticsearch.xpack.rollup.v2.indexer.metrics.MetricCollector.ValueCount; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +public abstract class MetricField { + final String name; + protected final MetricCollector[] collectors; + private final FieldValueFetcher fetcher; + + public MetricField(String name, MetricCollector[] collectors, FieldValueFetcher fetcher) { + this.name = name; + this.collectors = collectors; + this.fetcher = fetcher; + } + + public String getName() { + return name; + } + + public MetricCollector[] getCollectors() { + return collectors; + } + + public LeafMetricField getMetricFieldLeaf(LeafReaderContext context) { + return fetcher.getMetricFieldLeaf(context, collectors); + } + + public void resetCollectors() { + for (MetricCollector metricCollector : collectors) { + metricCollector.reset(); + } + } + + public static MetricField buildMetricField(MetricConfig metricConfig, FieldValueFetcher fetcher) { + final List normalizedMetrics = normalizeMetrics(metricConfig.getMetrics()); + final List list = new ArrayList<>(); + MetricCollector[] metricCollectors; + if (normalizedMetrics.isEmpty() == false) { + for (String metricName : normalizedMetrics) { + switch (metricName) { + case "min": + list.add(new Min()); + break; + case "max": + list.add(new Max()); + break; + case "sum": + list.add(new Sum()); + break; + case "value_count": + list.add(new ValueCount()); + break; + default: + throw new IllegalArgumentException("Unsupported metric type [" + metricName + "]"); + } + } + + metricCollectors = list.toArray(new MetricCollector[0]); + } else { + metricCollectors = new MetricCollector[0]; + } + + return new NumberMetricField(metricConfig.getField(), metricCollectors, fetcher); + } + + public static List normalizeMetrics(List metrics) { + List newMetrics = new ArrayList<>(metrics); + // avg = sum + value_count + if (newMetrics.remove(MetricConfig.AVG.getPreferredName())) { + if (newMetrics.contains(MetricConfig.VALUE_COUNT.getPreferredName()) == false) { + newMetrics.add(MetricConfig.VALUE_COUNT.getPreferredName()); + } + if (newMetrics.contains(MetricConfig.SUM.getPreferredName()) == false) { + newMetrics.add(MetricConfig.SUM.getPreferredName()); + } + } + return newMetrics; + } + + public abstract void collectMetric(StreamInput in) throws IOException; +} diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/metrics/NumberLeafMetricField.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/metrics/NumberLeafMetricField.java new file mode 100644 index 0000000000000..e1ffc77f60501 --- /dev/null +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/metrics/NumberLeafMetricField.java @@ -0,0 +1,56 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.rollup.v2.indexer.metrics; + +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.index.fielddata.FormattedDocValues; +import org.elasticsearch.index.fielddata.LeafFieldData; +import org.elasticsearch.search.DocValueFormat; + +import java.io.IOException; + +public class NumberLeafMetricField extends LeafMetricField { + private final FormattedDocValues formattedDocValues; + + public NumberLeafMetricField(MetricCollector[] metricCollectors, LeafFieldData fieldData) { + super(metricCollectors); + this.formattedDocValues = fieldData.getFormattedValues(DocValueFormat.RAW); + } + + @Override + public void collectMetric(int docID) throws IOException { + if (formattedDocValues.advanceExact(docID)) { + for (int i = 0; i < formattedDocValues.docValueCount(); i++) { + Object obj = formattedDocValues.nextValue(); + if (obj instanceof Number == false) { + throw new IllegalArgumentException("Expected [Number], got [" + obj.getClass() + "]"); + } + double value = ((Number) obj).doubleValue(); + for (MetricCollector metric : metricCollectors) { + metric.collect(value); + } + } + } + } + + @Override + public void writeMetrics(int docID, BytesStreamOutput out) throws IOException { + if (formattedDocValues.advanceExact(docID)) { + out.writeVInt(formattedDocValues.docValueCount()); + for (int i = 0; i < formattedDocValues.docValueCount(); i++) { + Object obj = formattedDocValues.nextValue(); + if (obj instanceof Number == false) { + throw new IllegalArgumentException("Expected [Number], got [" + obj.getClass() + "]"); + } + out.writeDouble(((Number) obj).doubleValue()); + } + } else { + out.writeVInt(0); + } + } +} diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/metrics/NumberMetricField.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/metrics/NumberMetricField.java new file mode 100644 index 0000000000000..4c979127333bb --- /dev/null +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/indexer/metrics/NumberMetricField.java @@ -0,0 +1,30 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.rollup.v2.indexer.metrics; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.xpack.rollup.v2.FieldValueFetcher; + +import java.io.IOException; + +public class NumberMetricField extends MetricField { + public NumberMetricField(String fieldName, MetricCollector[] collectors, FieldValueFetcher fetcher) { + super(fieldName, collectors, fetcher); + } + + @Override + public void collectMetric(StreamInput in) throws IOException { + int size = in.readVInt(); + for (int i = 0; i < size; i++) { + double value = in.readDouble(); + for (MetricCollector metricCollector : collectors) { + metricCollector.collect(value); + } + } + } +} diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/RestRollupAction.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/rest/RestRollupAction.java similarity index 96% rename from x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/RestRollupAction.java rename to x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/rest/RestRollupAction.java index 8ddf98935f19d..d5f60023a1900 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/RestRollupAction.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/v2/rest/RestRollupAction.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.xpack.rollup.v2; +package org.elasticsearch.xpack.rollup.v2.rest; import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.rest.BaseRestHandler; diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/v2/RollupActionSingleNodeTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/v2/RollupActionSingleNodeTests.java deleted file mode 100644 index 0ea7e46bc57bc..0000000000000 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/v2/RollupActionSingleNodeTests.java +++ /dev/null @@ -1,565 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ -package org.elasticsearch.xpack.rollup.v2; - -import org.apache.lucene.util.LuceneTestCase; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.ResourceAlreadyExistsException; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.DocWriteRequest; -import org.elasticsearch.action.admin.indices.get.GetIndexResponse; -import org.elasticsearch.action.admin.indices.rollover.RolloverRequest; -import org.elasticsearch.action.admin.indices.rollover.RolloverResponse; -import org.elasticsearch.action.admin.indices.template.put.PutComposableIndexTemplateAction; -import org.elasticsearch.action.bulk.BulkRequestBuilder; -import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.datastreams.CreateDataStreamAction; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.support.WriteRequest; -import org.elasticsearch.action.support.master.AcknowledgedResponse; -import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; -import org.elasticsearch.cluster.metadata.Template; -import org.elasticsearch.common.compress.CompressedXContent; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.time.DateFormatter; -import org.elasticsearch.datastreams.DataStreamsPlugin; -import org.elasticsearch.index.Index; -import org.elasticsearch.index.IndexNotFoundException; -import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.mapper.DateFieldMapper; -import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.indices.IndicesService; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregationBuilder; -import org.elasticsearch.search.aggregations.bucket.composite.CompositeValuesSourceBuilder; -import org.elasticsearch.search.aggregations.bucket.composite.DateHistogramValuesSourceBuilder; -import org.elasticsearch.search.aggregations.bucket.composite.HistogramValuesSourceBuilder; -import org.elasticsearch.search.aggregations.bucket.composite.InternalComposite; -import org.elasticsearch.search.aggregations.bucket.composite.TermsValuesSourceBuilder; -import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; -import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder; -import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder; -import org.elasticsearch.search.aggregations.metrics.MinAggregationBuilder; -import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder; -import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; -import org.elasticsearch.test.ESSingleNodeTestCase; -import org.elasticsearch.xcontent.XContentBuilder; -import org.elasticsearch.xcontent.XContentFactory; -import org.elasticsearch.xpack.aggregatemetric.AggregateMetricMapperPlugin; -import org.elasticsearch.xpack.analytics.AnalyticsPlugin; -import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin; -import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers; -import org.elasticsearch.xpack.core.rollup.RollupActionConfig; -import org.elasticsearch.xpack.core.rollup.RollupActionDateHistogramGroupConfig; -import org.elasticsearch.xpack.core.rollup.RollupActionGroupConfig; -import org.elasticsearch.xpack.core.rollup.action.RollupAction; -import org.elasticsearch.xpack.core.rollup.job.HistogramGroupConfig; -import org.elasticsearch.xpack.core.rollup.job.MetricConfig; -import org.elasticsearch.xpack.core.rollup.job.TermsGroupConfig; -import org.elasticsearch.xpack.rollup.Rollup; -import org.junit.Before; - -import java.io.IOException; -import java.time.ZoneId; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.concurrent.ExecutionException; - -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.equalTo; - -@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/69799") -public class RollupActionSingleNodeTests extends ESSingleNodeTestCase { - - private static final DateFormatter DATE_FORMATTER = DateFormatter.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSXXX"); - private String index; - private String rollupIndex; - private long startTime; - private int docCount; - - private String timestampFieldName = "@timestamp"; - - @Override - protected Collection> getPlugins() { - return List.of( - LocalStateCompositeXPackPlugin.class, - Rollup.class, - AnalyticsPlugin.class, - AggregateMetricMapperPlugin.class, - DataStreamsPlugin.class - ); - } - - @Before - public void setup() { - index = randomAlphaOfLength(5).toLowerCase(Locale.ROOT); - rollupIndex = randomAlphaOfLength(6).toLowerCase(Locale.ROOT); - startTime = randomLongBetween(946769284000L, 1607470084000L); // random date between 2000-2020 - docCount = randomIntBetween(10, 1000); - - client().admin() - .indices() - .prepareCreate(index) - .setSettings(Settings.builder().put("index.number_of_shards", 1).build()) - .setMapping( - "date_1", - "type=date", - "numeric_1", - "type=double", - "numeric_2", - "type=float", - "numeric_nonaggregatable", - "type=double,doc_values=false", - "categorical_1", - "type=keyword" - ) - .get(); - } - - public void testRollupShardIndexerCleansTempFiles() throws IOException { - // create rollup config and index documents into source index - RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); - SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() - .startObject() - .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) - .field("categorical_1", randomAlphaOfLength(1)) - .field("numeric_1", randomDouble()) - .endObject(); - RollupActionConfig config = new RollupActionConfig( - new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), - Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) - ); - bulkIndex(sourceSupplier); - - IndicesService indexServices = getInstanceFromNode(IndicesService.class); - Index srcIndex = resolveIndex(index); - IndexService indexService = indexServices.indexServiceSafe(srcIndex); - IndexShard shard = indexService.getShard(0); - - // re-use source index as temp index for test - RollupShardIndexer indexer = new RollupShardIndexer(client(), indexService, shard.shardId(), config, index, 2); - indexer.execute(); - // assert that files are deleted - assertThat(indexer.tmpFilesDeleted, equalTo(indexer.tmpFiles)); - } - - public void testCannotRollupToExistingIndex() throws Exception { - RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); - SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() - .startObject() - .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) - .field("categorical_1", randomAlphaOfLength(1)) - .field("numeric_1", randomDouble()) - .endObject(); - RollupActionConfig config = new RollupActionConfig( - new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), - Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) - ); - bulkIndex(sourceSupplier); - rollup(index, rollupIndex, config); - assertRollupIndex(config, index, rollupIndex); - ElasticsearchException exception = expectThrows(ElasticsearchException.class, () -> rollup(index, rollupIndex, config)); - assertThat(exception.getMessage(), containsString("Unable to rollup index [" + index + "]")); - } - - public void testTemporaryIndexCannotBeCreatedAlreadyExists() { - RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); - RollupActionConfig config = new RollupActionConfig( - new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), - Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) - ); - assertTrue(client().admin().indices().prepareCreate(".rolluptmp-" + rollupIndex).get().isAcknowledged()); - Exception exception = expectThrows(ElasticsearchException.class, () -> rollup(index, rollupIndex, config)); - assertThat(exception.getMessage(), containsString("already exists")); - } - - public void testCannotRollupWhileOtherRollupInProgress() throws Exception { - RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); - SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() - .startObject() - .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) - .field("categorical_1", randomAlphaOfLength(1)) - .field("numeric_1", randomDouble()) - .endObject(); - RollupActionConfig config = new RollupActionConfig( - new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), - Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) - ); - bulkIndex(sourceSupplier); - client().execute(RollupAction.INSTANCE, new RollupAction.Request(index, rollupIndex, config), ActionListener.wrap(() -> {})); - ResourceAlreadyExistsException exception = expectThrows( - ResourceAlreadyExistsException.class, - () -> rollup(index, rollupIndex, config) - ); - assertThat(exception.getMessage(), containsString(".rolluptmp-" + rollupIndex)); - } - - public void testTermsGrouping() throws IOException { - RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); - SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() - .startObject() - .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) - .field("categorical_1", randomAlphaOfLength(1)) - .field("numeric_1", randomDouble()) - .endObject(); - RollupActionConfig config = new RollupActionConfig( - new RollupActionGroupConfig(dateHistogramGroupConfig, null, new TermsGroupConfig("categorical_1")), - Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) - ); - bulkIndex(sourceSupplier); - rollup(index, rollupIndex, config); - assertRollupIndex(config, index, rollupIndex); - } - - public void testHistogramGrouping() throws IOException { - long interval = randomLongBetween(1, 1000); - RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); - SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() - .startObject() - .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) - .field("numeric_1", randomDoubleBetween(0.0, 10000.0, true)) - .field("numeric_2", randomDouble()) - .endObject(); - RollupActionConfig config = new RollupActionConfig( - new RollupActionGroupConfig(dateHistogramGroupConfig, new HistogramGroupConfig(interval, "numeric_1"), null), - Collections.singletonList(new MetricConfig("numeric_2", Collections.singletonList("max"))) - ); - bulkIndex(sourceSupplier); - rollup(index, rollupIndex, config); - assertRollupIndex(config, index, rollupIndex); - } - - public void testMaxMetric() throws IOException { - RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); - SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() - .startObject() - .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) - .field("numeric_1", randomDouble()) - .endObject(); - RollupActionConfig config = new RollupActionConfig( - new RollupActionGroupConfig(dateHistogramGroupConfig, null, null), - Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("max"))) - ); - bulkIndex(sourceSupplier); - rollup(index, rollupIndex, config); - assertRollupIndex(config, index, rollupIndex); - } - - public void testMinMetric() throws IOException { - RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); - SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() - .startObject() - .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) - .field("numeric_1", randomDouble()) - .endObject(); - RollupActionConfig config = new RollupActionConfig( - new RollupActionGroupConfig(dateHistogramGroupConfig, null, null), - Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("min"))) - ); - bulkIndex(sourceSupplier); - rollup(index, rollupIndex, config); - assertRollupIndex(config, index, rollupIndex); - } - - public void testValueCountMetric() throws IOException { - RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); - SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() - .startObject() - .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) - .field("numeric_1", randomDouble()) - .endObject(); - RollupActionConfig config = new RollupActionConfig( - new RollupActionGroupConfig(dateHistogramGroupConfig, null, null), - Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("value_count"))) - ); - bulkIndex(sourceSupplier); - rollup(index, rollupIndex, config); - assertRollupIndex(config, index, rollupIndex); - } - - public void testAvgMetric() throws IOException { - RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); - SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() - .startObject() - .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) - // Use integers to ensure that avg is comparable between rollup and original - .field("numeric_1", randomInt()) - .endObject(); - RollupActionConfig config = new RollupActionConfig( - new RollupActionGroupConfig(dateHistogramGroupConfig, null, null), - Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("avg"))) - ); - bulkIndex(sourceSupplier); - rollup(index, rollupIndex, config); - assertRollupIndex(config, index, rollupIndex); - } - - public void testValidationCheck() throws IOException { - RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig("date_1"); - SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() - .startObject() - .field("date_1", randomDateForInterval(dateHistogramGroupConfig.getInterval())) - // use integers to ensure that avg is comparable between rollup and original - .field("numeric_nonaggregatable", randomInt()) - .endObject(); - RollupActionConfig config = new RollupActionConfig( - new RollupActionGroupConfig(dateHistogramGroupConfig, null, null), - Collections.singletonList(new MetricConfig("numeric_nonaggregatable", Collections.singletonList("avg"))) - ); - bulkIndex(sourceSupplier); - Exception e = expectThrows(Exception.class, () -> rollup(index, rollupIndex, config)); - assertThat(e.getMessage(), containsString("The field [numeric_nonaggregatable] must be aggregatable")); - } - - public void testRollupDatastream() throws Exception { - RollupActionDateHistogramGroupConfig dateHistogramGroupConfig = randomRollupActionDateHistogramGroupConfig(timestampFieldName); - String dataStreamName = createDataStream(); - - SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder() - .startObject() - .field(timestampFieldName, randomDateForInterval(dateHistogramGroupConfig.getInterval())) - .field("numeric_1", randomDouble()) - .endObject(); - RollupActionConfig config = new RollupActionConfig( - new RollupActionGroupConfig(dateHistogramGroupConfig, null, null), - Collections.singletonList(new MetricConfig("numeric_1", Collections.singletonList("value_count"))) - ); - bulkIndex(dataStreamName, sourceSupplier); - - String oldIndexName = rollover(dataStreamName).getOldIndex(); - String rollupIndexName = ".rollup-" + oldIndexName; - rollup(oldIndexName, rollupIndexName, config); - assertRollupIndex(config, oldIndexName, rollupIndexName); - rollup(oldIndexName, rollupIndexName + "-2", config); - assertRollupIndex(config, oldIndexName, rollupIndexName + "-2"); - } - - private RollupActionDateHistogramGroupConfig randomRollupActionDateHistogramGroupConfig(String field) { - RollupActionDateHistogramGroupConfig randomConfig = ConfigTestHelpers.randomRollupActionDateHistogramGroupConfig(random()); - if (randomConfig instanceof RollupActionDateHistogramGroupConfig.FixedInterval) { - return new RollupActionDateHistogramGroupConfig.FixedInterval(field, randomConfig.getInterval(), randomConfig.getTimeZone()); - } - if (randomConfig instanceof RollupActionDateHistogramGroupConfig.CalendarInterval) { - return new RollupActionDateHistogramGroupConfig.CalendarInterval(field, randomConfig.getInterval(), randomConfig.getTimeZone()); - } - throw new IllegalStateException("invalid RollupActionDateHistogramGroupConfig class type"); - } - - private String randomDateForInterval(DateHistogramInterval interval) { - final long maxNumBuckets = 10; - final long endTime = startTime + maxNumBuckets * interval.estimateMillis(); - return DATE_FORMATTER.formatMillis(randomLongBetween(startTime, endTime)); - } - - private void bulkIndex(SourceSupplier sourceSupplier) throws IOException { - bulkIndex(index, sourceSupplier); - } - - private void bulkIndex(String indexName, SourceSupplier sourceSupplier) throws IOException { - BulkRequestBuilder bulkRequestBuilder = client().prepareBulk(); - bulkRequestBuilder.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); - for (int i = 0; i < docCount; i++) { - IndexRequest indexRequest = new IndexRequest(indexName).opType(DocWriteRequest.OpType.CREATE); - XContentBuilder source = sourceSupplier.get(); - indexRequest.source(source); - bulkRequestBuilder.add(indexRequest); - } - BulkResponse bulkResponse = bulkRequestBuilder.get(); - if (bulkResponse.hasFailures()) { - fail("Failed to index data: " + bulkResponse.buildFailureMessage()); - } - assertHitCount(client().prepareSearch(indexName).setSize(0).get(), docCount); - } - - private void rollup(String sourceIndex, String rollupIndex, RollupActionConfig config) { - AcknowledgedResponse rollupResponse = client().execute( - RollupAction.INSTANCE, - new RollupAction.Request(sourceIndex, rollupIndex, config) - ).actionGet(); - assertTrue(rollupResponse.isAcknowledged()); - } - - private RolloverResponse rollover(String dataStreamName) throws ExecutionException, InterruptedException { - RolloverResponse response = client().admin().indices().rolloverIndex(new RolloverRequest(dataStreamName, null)).get(); - assertTrue(response.isAcknowledged()); - return response; - } - - @SuppressWarnings("unchecked") - private void assertRollupIndex(RollupActionConfig config, String sourceIndex, String rollupIndexName) { - final CompositeAggregationBuilder aggregation = buildCompositeAggs("resp", config); - long numBuckets = 0; - InternalComposite origResp = client().prepareSearch(sourceIndex).addAggregation(aggregation).get().getAggregations().get("resp"); - InternalComposite rollupResp = client().prepareSearch(rollupIndexName) - .addAggregation(aggregation) - .get() - .getAggregations() - .get("resp"); - while (origResp.afterKey() != null) { - numBuckets += origResp.getBuckets().size(); - assertThat(origResp, equalTo(rollupResp)); - aggregation.aggregateAfter(origResp.afterKey()); - origResp = client().prepareSearch(sourceIndex).addAggregation(aggregation).get().getAggregations().get("resp"); - rollupResp = client().prepareSearch(rollupIndexName).addAggregation(aggregation).get().getAggregations().get("resp"); - } - assertThat(origResp, equalTo(rollupResp)); - - SearchResponse resp = client().prepareSearch(rollupIndexName).setTrackTotalHits(true).get(); - assertThat(resp.getHits().getTotalHits().value, equalTo(numBuckets)); - - GetIndexResponse indexSettingsResp = client().admin().indices().prepareGetIndex().addIndices(sourceIndex, rollupIndexName).get(); - // Assert rollup metadata are set in index settings - assertEquals( - indexSettingsResp.getSetting(sourceIndex, "index.uuid"), - indexSettingsResp.getSetting(rollupIndexName, "index.rollup.source.uuid") - ); - assertEquals( - indexSettingsResp.getSetting(sourceIndex, "index.provided_name"), - indexSettingsResp.getSetting(rollupIndexName, "index.rollup.source.name") - ); - - // Assert field mappings - Map> mappings = (Map>) indexSettingsResp.getMappings() - .get(rollupIndexName) - .getSourceAsMap() - .get("properties"); - - RollupActionDateHistogramGroupConfig dateHistoConfig = config.getGroupConfig().getDateHistogram(); - assertEquals(DateFieldMapper.CONTENT_TYPE, mappings.get(dateHistoConfig.getField()).get("type")); - Map dateTimeMeta = (Map) mappings.get(dateHistoConfig.getField()).get("meta"); - assertEquals(dateHistoConfig.getTimeZone(), dateTimeMeta.get("time_zone")); - assertEquals(dateHistoConfig.getInterval().toString(), dateTimeMeta.get(dateHistoConfig.getIntervalTypeName())); - - for (MetricConfig metricsConfig : config.getMetricsConfig()) { - assertEquals("aggregate_metric_double", mappings.get(metricsConfig.getField()).get("type")); - List supportedMetrics = (List) mappings.get(metricsConfig.getField()).get("metrics"); - for (String m : metricsConfig.getMetrics()) { - if ("avg".equals(m)) { - assertTrue(supportedMetrics.contains("sum") && supportedMetrics.contains("value_count")); - } else { - assertTrue(supportedMetrics.contains(m)); - } - } - } - - HistogramGroupConfig histoConfig = config.getGroupConfig().getHistogram(); - if (histoConfig != null) { - for (String field : histoConfig.getFields()) { - assertTrue((mappings.containsKey(field))); - Map meta = (Map) mappings.get(field).get("meta"); - assertEquals(String.valueOf(histoConfig.getInterval()), meta.get("interval")); - } - } - - TermsGroupConfig termsConfig = config.getGroupConfig().getTerms(); - if (termsConfig != null) { - for (String field : termsConfig.getFields()) { - assertTrue(mappings.containsKey(field)); - } - } - - // Assert that temporary index was removed - expectThrows( - IndexNotFoundException.class, - () -> client().admin().indices().prepareGetIndex().addIndices(".rolluptmp-" + rollupIndexName).get() - ); - } - - private CompositeAggregationBuilder buildCompositeAggs(String name, RollupActionConfig config) { - List> sources = new ArrayList<>(); - - RollupActionDateHistogramGroupConfig dateHistoConfig = config.getGroupConfig().getDateHistogram(); - DateHistogramValuesSourceBuilder dateHisto = new DateHistogramValuesSourceBuilder(dateHistoConfig.getField()); - dateHisto.field(dateHistoConfig.getField()); - if (dateHistoConfig.getTimeZone() != null) { - dateHisto.timeZone(ZoneId.of(dateHistoConfig.getTimeZone())); - } - if (dateHistoConfig instanceof RollupActionDateHistogramGroupConfig.FixedInterval) { - dateHisto.fixedInterval(dateHistoConfig.getInterval()); - } else if (dateHistoConfig instanceof RollupActionDateHistogramGroupConfig.CalendarInterval) { - dateHisto.calendarInterval(dateHistoConfig.getInterval()); - } else { - throw new IllegalStateException("unsupported RollupActionDateHistogramGroupConfig"); - } - sources.add(dateHisto); - - if (config.getGroupConfig().getHistogram() != null) { - HistogramGroupConfig histoConfig = config.getGroupConfig().getHistogram(); - for (String field : histoConfig.getFields()) { - HistogramValuesSourceBuilder source = new HistogramValuesSourceBuilder(field).field(field) - .interval(histoConfig.getInterval()); - sources.add(source); - } - } - - if (config.getGroupConfig().getTerms() != null) { - TermsGroupConfig termsConfig = config.getGroupConfig().getTerms(); - for (String field : termsConfig.getFields()) { - TermsValuesSourceBuilder source = new TermsValuesSourceBuilder(field).field(field); - sources.add(source); - } - } - - final CompositeAggregationBuilder composite = new CompositeAggregationBuilder(name, sources).size(100); - if (config.getMetricsConfig() != null) { - for (MetricConfig metricConfig : config.getMetricsConfig()) { - for (String metricName : metricConfig.getMetrics()) { - switch (metricName) { - case "min" -> composite.subAggregation(new MinAggregationBuilder(metricName).field(metricConfig.getField())); - case "max" -> composite.subAggregation(new MaxAggregationBuilder(metricName).field(metricConfig.getField())); - case "sum" -> composite.subAggregation(new SumAggregationBuilder(metricName).field(metricConfig.getField())); - case "value_count" -> composite.subAggregation( - new ValueCountAggregationBuilder(metricName).field(metricConfig.getField()) - ); - case "avg" -> composite.subAggregation(new AvgAggregationBuilder(metricName).field(metricConfig.getField())); - default -> throw new IllegalArgumentException("Unsupported metric type [" + metricName + "]"); - } - } - } - } - return composite; - } - - @FunctionalInterface - public interface SourceSupplier { - XContentBuilder get() throws IOException; - } - - private String createDataStream() throws Exception { - String dataStreamName = randomAlphaOfLength(10).toLowerCase(Locale.getDefault()); - Template idxTemplate = new Template(null, new CompressedXContent(""" - {"properties":{"%s":{"type":"date"},"data":{"type":"keyword"}}} - """.formatted(timestampFieldName)), null); - ComposableIndexTemplate template = new ComposableIndexTemplate( - List.of(dataStreamName + "*"), - idxTemplate, - null, - null, - null, - null, - new ComposableIndexTemplate.DataStreamTemplate(), - null - ); - assertTrue( - client().execute( - PutComposableIndexTemplateAction.INSTANCE, - new PutComposableIndexTemplateAction.Request(dataStreamName + "_template").indexTemplate(template) - ).actionGet().isAcknowledged() - ); - assertTrue( - client().execute(CreateDataStreamAction.INSTANCE, new CreateDataStreamAction.Request(dataStreamName)).get().isAcknowledged() - ); - return dataStreamName; - } - -} diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/v2/RollupShardIndexerTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/v2/RollupShardIndexerTests.java new file mode 100644 index 0000000000000..9a819cd088195 --- /dev/null +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/v2/RollupShardIndexerTests.java @@ -0,0 +1,26 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.rollup.v2; + +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.rollup.v2.RollupShardIndexer.BucketKey; + +import java.util.List; + +public class RollupShardIndexerTests extends ESTestCase { + public void testBucketKey() { + long timestamp = randomNonNegativeLong(); + List groups = List.of("group1", "group2", 3, 4L, 5.0f); + BucketKey key1 = new BucketKey(timestamp, groups); + BucketKey key2 = new BucketKey(timestamp, groups); + assertEquals(key1, key2); + assertEquals(key1.hashCode(), key2.hashCode()); + assertEquals(key1.toString(), key2.toString()); + } + +} diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/v2/RollupTestCase.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/v2/RollupTestCase.java new file mode 100644 index 0000000000000..221d3d39ccd7f --- /dev/null +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/v2/RollupTestCase.java @@ -0,0 +1,30 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.rollup.v2; + +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.test.ESTestCase; + +public abstract class RollupTestCase extends ESTestCase { + protected IndexMetadata newIndexMetadata(Settings settings) { + Settings build = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(settings) + .build(); + return IndexMetadata.builder("test").settings(build).build(); + } + + protected IndexSettings newIndexSettings(Settings settings) { + return new IndexSettings(newIndexMetadata(settings), Settings.EMPTY); + } +} diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/v2/TransportRollupActionTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/v2/TransportRollupActionTests.java new file mode 100644 index 0000000000000..230e3b83a3728 --- /dev/null +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/v2/TransportRollupActionTests.java @@ -0,0 +1,504 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.rollup.v2; + +import org.elasticsearch.action.fieldcaps.FieldCapabilities; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.metadata.MappingMetadata; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.time.DateUtils; +import org.elasticsearch.index.IndexMode; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.mapper.DataStreamTimestampFieldMapper; +import org.elasticsearch.index.mapper.SourceFieldMapper; +import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentFactory; +import org.elasticsearch.xpack.core.ilm.LifecycleSettings; +import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers; +import org.elasticsearch.xpack.core.rollup.RollupActionConfig; +import org.elasticsearch.xpack.core.rollup.RollupActionDateHistogramGroupConfig; +import org.elasticsearch.xpack.core.rollup.RollupActionGroupConfig; +import org.elasticsearch.xpack.core.rollup.job.HistogramGroupConfig; +import org.elasticsearch.xpack.core.rollup.job.MetricConfig; +import org.elasticsearch.xpack.core.rollup.job.TermsGroupConfig; + +import java.io.IOException; +import java.time.Instant; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import static org.elasticsearch.xpack.rollup.v2.TransportRollupAction.addRollupSettings; +import static org.elasticsearch.xpack.rollup.v2.TransportRollupAction.getMapping; +import static org.elasticsearch.xpack.rollup.v2.TransportRollupAction.getSettings; +import static org.elasticsearch.xpack.rollup.v2.TransportRollupAction.rebuildRollupConfig; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class TransportRollupActionTests extends RollupTestCase { + + public void testSettingsFilter() { + Settings settings = Settings.builder() + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) + .put(IndexMetadata.SETTING_INDEX_UUID, randomAlphaOfLength(5)) + .put(IndexMetadata.SETTING_HISTORY_UUID, randomAlphaOfLength(5)) + .put(IndexMetadata.SETTING_INDEX_PROVIDED_NAME, randomAlphaOfLength(5)) + .put(IndexMetadata.SETTING_CREATION_DATE, 0L) + .put(LifecycleSettings.LIFECYCLE_NAME, randomAlphaOfLength(5)) + .put(LifecycleSettings.LIFECYCLE_INDEXING_COMPLETE, randomBoolean()) + .put(IndexMetadata.INDEX_ROLLUP_SOURCE_UUID_KEY, randomAlphaOfLength(5)) + .put(IndexMetadata.INDEX_ROLLUP_SOURCE_NAME_KEY, randomAlphaOfLength(5)) + .put(IndexMetadata.INDEX_RESIZE_SOURCE_UUID_KEY, randomAlphaOfLength(5)) + .put(IndexMetadata.INDEX_RESIZE_SOURCE_NAME_KEY, randomAlphaOfLength(5)) + .put(IndexSettings.MODE.getKey(), "time_series") + .put(IndexMetadata.INDEX_ROUTING_PATH.getKey(), randomAlphaOfLength(5)) + .put(IndexSettings.TIME_SERIES_START_TIME.getKey(), Instant.ofEpochMilli(1).toString()) + .put(IndexSettings.TIME_SERIES_END_TIME.getKey(), Instant.ofEpochMilli(DateUtils.MAX_MILLIS_BEFORE_9999 - 1).toString()) + .build(); + IndexMetadata indexMetadata = newIndexMetadata(settings); + Settings newSettings = getSettings(indexMetadata, null); + Settings expected = Settings.builder() + .put(IndexMetadata.SETTING_INDEX_HIDDEN, true) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + assertThat(newSettings, equalTo(expected)); + } + + public void testNullTermsSettings() { + Settings settings = Settings.builder().build(); + RollupActionGroupConfig groupConfig = new RollupActionGroupConfig( + new RollupActionDateHistogramGroupConfig.FixedInterval( + DataStreamTimestampFieldMapper.DEFAULT_PATH, + ConfigTestHelpers.randomInterval(), + randomZone().getId() + ), + null, + null + ); + IndexMetadata indexMetadata = newIndexMetadata(settings); + Settings newSettings = getSettings( + indexMetadata, + new RollupActionConfig(groupConfig, ConfigTestHelpers.randomMetricsConfigs(random())) + ); + Settings expected = Settings.builder() + .put(IndexMetadata.SETTING_INDEX_HIDDEN, true) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + assertThat(newSettings, equalTo(expected)); + } + + public void testNotTimestampSettings() { + Settings settings = Settings.builder().build(); + IndexMetadata indexMetadata = newIndexMetadata(settings); + Settings newSettings = getSettings( + indexMetadata, + new RollupActionConfig( + ConfigTestHelpers.randomRollupActionGroupConfig(random()), + ConfigTestHelpers.randomMetricsConfigs(random()) + ) + ); + Settings expected = Settings.builder() + .put(IndexMetadata.SETTING_INDEX_HIDDEN, true) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + assertThat(newSettings, equalTo(expected)); + } + + public void testStandardIndexSettings() { + String[] terms = randomArray(1, 5, String[]::new, () -> randomAlphaOfLength(5)); + Settings settings = Settings.builder().build(); + RollupActionGroupConfig groupConfig = new RollupActionGroupConfig( + new RollupActionDateHistogramGroupConfig.FixedInterval( + DataStreamTimestampFieldMapper.DEFAULT_PATH, + ConfigTestHelpers.randomInterval(), + randomZone().getId() + ), + null, + new TermsGroupConfig(terms) + ); + IndexMetadata indexMetadata = newIndexMetadata(settings); + Settings newSettings = getSettings( + indexMetadata, + new RollupActionConfig(groupConfig, ConfigTestHelpers.randomMetricsConfigs(random())) + ); + Settings expected = Settings.builder() + .put(IndexMetadata.SETTING_INDEX_HIDDEN, true) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES.name().toLowerCase(Locale.ROOT)) + .putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), terms) + .put(IndexSettings.TIME_SERIES_START_TIME.getKey(), Instant.ofEpochMilli(1).toString()) + .put(IndexSettings.TIME_SERIES_END_TIME.getKey(), Instant.ofEpochMilli(DateUtils.MAX_MILLIS_BEFORE_9999 - 1).toString()) + .build(); + assertThat(newSettings, equalTo(expected)); + } + + public void testTsidSettings() { + String[] terms = randomArray(1, 5, String[]::new, () -> randomAlphaOfLength(5)); + Settings settings = Settings.builder() + .put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES.name().toLowerCase(Locale.ROOT)) + .putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), terms) + .put(IndexSettings.TIME_SERIES_START_TIME.getKey(), Instant.ofEpochMilli(1).toString()) + .put(IndexSettings.TIME_SERIES_END_TIME.getKey(), Instant.ofEpochMilli(DateUtils.MAX_MILLIS_BEFORE_9999 - 1).toString()) + .build(); + RollupActionGroupConfig groupConfig = new RollupActionGroupConfig( + new RollupActionDateHistogramGroupConfig.FixedInterval( + DataStreamTimestampFieldMapper.DEFAULT_PATH, + ConfigTestHelpers.randomInterval(), + randomZone().getId() + ), + null, + new TermsGroupConfig(TimeSeriesIdFieldMapper.NAME) + ); + IndexMetadata indexMetadata = newIndexMetadata(settings); + Settings newSettings = getSettings( + indexMetadata, + new RollupActionConfig(groupConfig, ConfigTestHelpers.randomMetricsConfigs(random())) + ); + Settings expected = Settings.builder() + .put(IndexMetadata.SETTING_INDEX_HIDDEN, true) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES.name().toLowerCase(Locale.ROOT)) + .putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), terms) + .put(IndexSettings.TIME_SERIES_START_TIME.getKey(), Instant.ofEpochMilli(1).toString()) + .put(IndexSettings.TIME_SERIES_END_TIME.getKey(), Instant.ofEpochMilli(DateUtils.MAX_MILLIS_BEFORE_9999 - 1).toString()) + .build(); + assertThat(newSettings, equalTo(expected)); + } + + public void testRoutingPathNotInTermsSettings() { + List routingPath = List.of("exist", "not_exist", "wildcard*"); + String[] terms = { "exist", "wildcard_1", "wildcard_2" }; + Settings settings = Settings.builder() + .put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES.name().toLowerCase(Locale.ROOT)) + .putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), routingPath) + .build(); + RollupActionGroupConfig groupConfig = new RollupActionGroupConfig( + new RollupActionDateHistogramGroupConfig.FixedInterval( + DataStreamTimestampFieldMapper.DEFAULT_PATH, + ConfigTestHelpers.randomInterval(), + randomZone().getId() + ), + null, + new TermsGroupConfig(terms) + ); + IndexMetadata indexMetadata = newIndexMetadata(settings); + Settings newSettings = getSettings( + indexMetadata, + new RollupActionConfig(groupConfig, ConfigTestHelpers.randomMetricsConfigs(random())) + ); + Settings expected = Settings.builder() + .put(IndexMetadata.SETTING_INDEX_HIDDEN, true) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES.name().toLowerCase(Locale.ROOT)) + .putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), terms) + .put(IndexSettings.TIME_SERIES_START_TIME.getKey(), Instant.ofEpochMilli(1).toString()) + .put(IndexSettings.TIME_SERIES_END_TIME.getKey(), Instant.ofEpochMilli(DateUtils.MAX_MILLIS_BEFORE_9999 - 1).toString()) + .build(); + assertThat(newSettings, equalTo(expected)); + } + + public void testTermNotInRoutingPathSettings() { + List routingPath = List.of("exist", "wildcard*"); + String[] terms = { "exist", "not_exist", "wildcard_1", "wildcard_2" }; + Settings settings = Settings.builder() + .put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES.name().toLowerCase(Locale.ROOT)) + .putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), routingPath) + .put(IndexSettings.TIME_SERIES_START_TIME.getKey(), Instant.ofEpochMilli(1).toString()) + .put(IndexSettings.TIME_SERIES_END_TIME.getKey(), Instant.ofEpochMilli(DateUtils.MAX_MILLIS_BEFORE_9999 - 1).toString()) + .build(); + RollupActionGroupConfig groupConfig = new RollupActionGroupConfig( + new RollupActionDateHistogramGroupConfig.FixedInterval( + DataStreamTimestampFieldMapper.DEFAULT_PATH, + ConfigTestHelpers.randomInterval(), + randomZone().getId() + ), + null, + new TermsGroupConfig(terms) + ); + IndexMetadata indexMetadata = newIndexMetadata(settings); + Settings newSettings = getSettings( + indexMetadata, + new RollupActionConfig(groupConfig, ConfigTestHelpers.randomMetricsConfigs(random())) + ); + Settings expected = Settings.builder() + .put(IndexMetadata.SETTING_INDEX_HIDDEN, true) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES.name().toLowerCase(Locale.ROOT)) + .putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), routingPath) + .put(IndexSettings.TIME_SERIES_START_TIME.getKey(), Instant.ofEpochMilli(1).toString()) + .put(IndexSettings.TIME_SERIES_END_TIME.getKey(), Instant.ofEpochMilli(DateUtils.MAX_MILLIS_BEFORE_9999 - 1).toString()) + .build(); + assertThat(newSettings, equalTo(expected)); + } + + public void testTermsNotMatchWildcardSettings() { + List routingPath = List.of("exist", "wildcard*"); + String[] terms = { "exist", "no_wildcard" }; + Settings settings = Settings.builder() + .put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES.name().toLowerCase(Locale.ROOT)) + .putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), routingPath) + .build(); + RollupActionGroupConfig groupConfig = new RollupActionGroupConfig( + new RollupActionDateHistogramGroupConfig.FixedInterval( + DataStreamTimestampFieldMapper.DEFAULT_PATH, + ConfigTestHelpers.randomInterval(), + randomZone().getId() + ), + null, + new TermsGroupConfig(terms) + ); + IndexMetadata indexMetadata = newIndexMetadata(settings); + Settings newSettings = getSettings( + indexMetadata, + new RollupActionConfig(groupConfig, ConfigTestHelpers.randomMetricsConfigs(random())) + ); + Settings expected = Settings.builder() + .put(IndexMetadata.SETTING_INDEX_HIDDEN, true) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES.name().toLowerCase(Locale.ROOT)) + .putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), terms) + .put(IndexSettings.TIME_SERIES_START_TIME.getKey(), Instant.ofEpochMilli(1).toString()) + .put(IndexSettings.TIME_SERIES_END_TIME.getKey(), Instant.ofEpochMilli(DateUtils.MAX_MILLIS_BEFORE_9999 - 1).toString()) + .build(); + assertThat(newSettings, equalTo(expected)); + } + + public void testGetMapping() throws IOException { + RollupActionGroupConfig groupConfig = new RollupActionGroupConfig( + new RollupActionDateHistogramGroupConfig.FixedInterval( + DataStreamTimestampFieldMapper.DEFAULT_PATH, + DateHistogramInterval.MINUTE, + "W-SU" + ), + new HistogramGroupConfig(10, "number1", "number2"), + new TermsGroupConfig(TimeSeriesIdFieldMapper.NAME, "terms1", "terms2") + ); + + List metricConfigs = List.of( + new MetricConfig("metric1", List.of("max", "min", "sum", "value_count")), + new MetricConfig("metric2", List.of("max", "min")), + new MetricConfig("metric3", List.of("sum")) + ); + + RollupActionConfig rollupActionConfig = new RollupActionConfig(groupConfig, metricConfigs); + MappingMetadata mappingMetadata = randomBoolean() + ? new MappingMetadata("_doc", randomBoolean() ? Map.of(SourceFieldMapper.NAME, Map.of("enabled", "false")) : Map.of()) + : null; + XContentBuilder mapping = getMapping(rollupActionConfig, mappingMetadata); + XContentBuilder expected = XContentFactory.jsonBuilder().startObject(); + if (mappingMetadata != null && mappingMetadata.getSourceAsMap().containsKey(SourceFieldMapper.NAME)) { + expected.field(SourceFieldMapper.NAME, mappingMetadata.getSourceAsMap().get(SourceFieldMapper.NAME)); + } + expected.startArray("dynamic_templates") + .startObject() + .startObject("strings") + .field("match_mapping_type", "string") + .startObject("mapping") + .field("type", "keyword") + .field("time_series_dimension", "true") + .endObject() + .endObject() + .endObject() + .endArray() + .startObject("properties") + .startObject("@timestamp") + .field("type", "date") + .startObject("meta") + .field("fixed_interval", "1m") + .field("time_zone", "W-SU") + .endObject() + .endObject() + .startObject("number1") + .field("type", "double") + .startObject("meta") + .field("interval", "10") + .endObject() + .endObject() + .startObject("number2") + .field("type", "double") + .startObject("meta") + .field("interval", "10") + .endObject() + .endObject() + .startObject("terms1") + .field("type", "keyword") + .field("time_series_dimension", "true") + .endObject() + .startObject("terms2") + .field("type", "keyword") + .field("time_series_dimension", "true") + .endObject() + .startObject("metric1") + .field("type", "aggregate_metric_double") + .startArray("metrics") + .value("max") + .value("min") + .value("sum") + .value("value_count") + .endArray() + .field("default_metric", "value_count") + .endObject() + .startObject("metric2") + .field("type", "aggregate_metric_double") + .startArray("metrics") + .value("max") + .value("min") + .endArray() + .field("default_metric", "max") + .endObject() + .startObject("metric3") + .field("type", "aggregate_metric_double") + .startArray("metrics") + .value("sum") + .endArray() + .field("default_metric", "sum") + .endObject() + .endObject() + .endObject(); + + assertThat(Strings.toString(mapping), is(Strings.toString(expected))); + } + + public void testAddRollupSettings() { + Settings.Builder settings = Settings.builder(); + if (randomBoolean()) { + settings.put(LifecycleSettings.LIFECYCLE_NAME, randomAlphaOfLength(5)); + } + if (randomBoolean()) { + settings.put(LifecycleSettings.LIFECYCLE_INDEXING_COMPLETE, randomBoolean()); + } + Settings newSettings = addRollupSettings(settings.build()); + Settings expected = settings.put(IndexMetadata.SETTING_INDEX_HIDDEN, true).build(); + assertThat(newSettings, equalTo(expected)); + } + + public void testNoWildcardRebuild() { + RollupActionConfig rollupActionConfig = new RollupActionConfig( + ConfigTestHelpers.randomRollupActionGroupConfig(random()), + ConfigTestHelpers.randomMetricsConfigs(random()) + ); + RollupActionConfig newRollupActionConfig = rebuildRollupConfig(rollupActionConfig, Map.of()); + assertThat(newRollupActionConfig, equalTo(rollupActionConfig)); + } + + public void testWildcardRebuild() { + RollupActionGroupConfig groupConfig = new RollupActionGroupConfig( + ConfigTestHelpers.randomRollupActionDateHistogramGroupConfig(random()), + new HistogramGroupConfig(10, "number1", "wildcard_num*"), + new TermsGroupConfig("terms1", "terms2", "wildcard_term*") + ); + List metricConfigs = List.of( + new MetricConfig("metric1", List.of("max")), + new MetricConfig("wildcard_metric*", List.of("max")) + ); + RollupActionConfig rollupActionConfig = new RollupActionConfig(groupConfig, metricConfigs); + RollupActionConfig newRollupActionConfig = rebuildRollupConfig( + rollupActionConfig, + Map.of( + "wildcard_num_1", + Map.of("double", createFieldCapabilities()), + "wildcard_term_1", + Map.of("keyword", createFieldCapabilities()), + "wildcard_metric_1", + Map.of("double", createFieldCapabilities()) + ) + ); + + RollupActionConfig expected = new RollupActionConfig( + new RollupActionGroupConfig( + groupConfig.getDateHistogram(), + new HistogramGroupConfig(10, "number1", "wildcard_num_1"), + new TermsGroupConfig("terms1", "terms2", "wildcard_term_1") + ), + List.of(new MetricConfig("metric1", List.of("max")), new MetricConfig("wildcard_metric_1", List.of("max"))) + ); + assertThat(newRollupActionConfig, equalTo(expected)); + } + + public void testNotMatchWildcardRebuild() { + RollupActionGroupConfig baseGroupConfig = new RollupActionGroupConfig( + ConfigTestHelpers.randomRollupActionDateHistogramGroupConfig(random()), + new HistogramGroupConfig(10, "wildcard_num*"), + new TermsGroupConfig("wildcard_term*") + ); + List baseMetricConfigs = List.of(new MetricConfig("wildcard_metric*", List.of("max"))); + + Map> fieldCaps = Map.of( + "wildcard_num_1", + Map.of("double", createFieldCapabilities()), + "wildcard_term_1", + Map.of("keyword", createFieldCapabilities()), + "wildcard_metric_1", + Map.of("double", createFieldCapabilities()) + ); + + { + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> rebuildRollupConfig( + new RollupActionConfig( + new RollupActionGroupConfig( + baseGroupConfig.getDateHistogram(), + baseGroupConfig.getHistogram(), + new TermsGroupConfig("wildcard_no*") + ), + baseMetricConfigs + ), + fieldCaps + ) + ); + assertThat(e.getMessage(), containsString("Could not find a field match the group terms [wildcard_no*]")); + } + + { + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> rebuildRollupConfig( + new RollupActionConfig( + new RollupActionGroupConfig( + baseGroupConfig.getDateHistogram(), + new HistogramGroupConfig(10, "wildcard_no*"), + baseGroupConfig.getTerms() + ), + baseMetricConfigs + ), + fieldCaps + ) + ); + assertThat(e.getMessage(), containsString("Could not find a field match the group histograms [wildcard_no*]")); + } + + { + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> rebuildRollupConfig( + new RollupActionConfig(baseGroupConfig, List.of(new MetricConfig("wildcard_no*", List.of("max")))), + fieldCaps + ) + ); + assertThat(e.getMessage(), containsString("Could not find a field match the metric fields [wildcard_no*]")); + } + } + + private FieldCapabilities createFieldCapabilities() { + return new FieldCapabilities("field", "type", false, true, true, null, null, null, Collections.emptyMap()); + } +} diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/v2/TransportRollupIndexerActionTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/v2/TransportRollupIndexerActionTests.java new file mode 100644 index 0000000000000..8229a7ab47830 --- /dev/null +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/v2/TransportRollupIndexerActionTests.java @@ -0,0 +1,127 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.rollup.v2; + +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexMode; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.mapper.DataStreamTimestampFieldMapper; +import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper; +import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers; +import org.elasticsearch.xpack.core.rollup.RollupActionDateHistogramGroupConfig; +import org.elasticsearch.xpack.core.rollup.RollupActionGroupConfig; +import org.elasticsearch.xpack.core.rollup.job.HistogramGroupConfig; +import org.elasticsearch.xpack.core.rollup.job.TermsGroupConfig; + +import static org.elasticsearch.xpack.rollup.v2.TransportRollupIndexerAction.isRollupTimeSeries; +import static org.hamcrest.Matchers.equalTo; + +public class TransportRollupIndexerActionTests extends RollupTestCase { + + public void testIsRollupTimeSeries() { + RollupActionGroupConfig groupConfig = new RollupActionGroupConfig( + new RollupActionDateHistogramGroupConfig.FixedInterval( + DataStreamTimestampFieldMapper.DEFAULT_PATH, + ConfigTestHelpers.randomInterval(), + randomZone().getId() + ), + null, + new TermsGroupConfig(TimeSeriesIdFieldMapper.NAME) + ); + + assertThat(isRollupTimeSeries(newTimeSeriesIndexSettings(), groupConfig), equalTo(true)); + } + + public void testRollupNullGroupConfig() { + assertThat(isRollupTimeSeries(newTimeSeriesIndexSettings(), null), equalTo(false)); + } + + public void testRollupStandardIndex() { + IndexSettings indexSettings = newIndexSettings(Settings.EMPTY); + RollupActionGroupConfig groupConfig = ConfigTestHelpers.randomRollupActionGroupConfig(random()); + assertThat(isRollupTimeSeries(indexSettings, groupConfig), equalTo(false)); + } + + public void testRollupNotTimestamp() { + RollupActionGroupConfig groupConfig = new RollupActionGroupConfig( + new RollupActionDateHistogramGroupConfig.FixedInterval( + randomAlphaOfLength(5), + ConfigTestHelpers.randomInterval(), + randomZone().getId() + ), + null, + null + ); + assertThat(isRollupTimeSeries(newTimeSeriesIndexSettings(), groupConfig), equalTo(false)); + } + + public void testRollupNoTerms() { + RollupActionGroupConfig groupConfig = new RollupActionGroupConfig( + new RollupActionDateHistogramGroupConfig.FixedInterval( + DataStreamTimestampFieldMapper.DEFAULT_PATH, + ConfigTestHelpers.randomInterval(), + randomZone().getId() + ), + null, + null + ); + assertThat(isRollupTimeSeries(newTimeSeriesIndexSettings(), groupConfig), equalTo(false)); + } + + public void testRollupHistogram() { + RollupActionGroupConfig groupConfig = new RollupActionGroupConfig( + new RollupActionDateHistogramGroupConfig.FixedInterval( + DataStreamTimestampFieldMapper.DEFAULT_PATH, + ConfigTestHelpers.randomInterval(), + randomZone().getId() + ), + new HistogramGroupConfig(randomLongBetween(10, 100), randomAlphaOfLength(5)), + new TermsGroupConfig(TimeSeriesIdFieldMapper.NAME) + ); + assertThat(isRollupTimeSeries(newTimeSeriesIndexSettings(), groupConfig), equalTo(false)); + } + + public void testRollupNotTsid() { + RollupActionGroupConfig groupConfig = new RollupActionGroupConfig( + new RollupActionDateHistogramGroupConfig.FixedInterval( + DataStreamTimestampFieldMapper.DEFAULT_PATH, + ConfigTestHelpers.randomInterval(), + randomZone().getId() + ), + null, + new TermsGroupConfig(randomAlphaOfLength(5)) + ); + + assertThat(isRollupTimeSeries(newTimeSeriesIndexSettings(), groupConfig), equalTo(false)); + } + + public void testRollupTooManyTerms() { + RollupActionGroupConfig groupConfig = new RollupActionGroupConfig( + new RollupActionDateHistogramGroupConfig.FixedInterval( + DataStreamTimestampFieldMapper.DEFAULT_PATH, + ConfigTestHelpers.randomInterval(), + randomZone().getId() + ), + null, + new TermsGroupConfig(TimeSeriesIdFieldMapper.NAME, randomAlphaOfLength(5)) + ); + + assertThat(isRollupTimeSeries(newTimeSeriesIndexSettings(), groupConfig), equalTo(false)); + } + + private IndexSettings newTimeSeriesIndexSettings() { + final Settings settings = Settings.builder() + .put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES) + .put(IndexMetadata.INDEX_ROUTING_PATH.getKey(), "foo") + .put(IndexSettings.TIME_SERIES_START_TIME.getKey(), "2021-04-28T00:00:00Z") + .put(IndexSettings.TIME_SERIES_END_TIME.getKey(), "2021-04-29T00:00:00Z") + .build(); + return newIndexSettings(settings); + } +}