mirror of
https://github.com/elastic/elasticsearch.git
synced 2025-06-30 02:13:33 -04:00
The data frame plugin allows users to create feature indexes by pivoting a source index. In a nutshell this can be understood as reindex supporting aggregations or similar to the so called entity centric indexing. Full history is provided in: feature/data-frame-transforms
This commit is contained in:
parent
8895befe51
commit
4f662bd289
93 changed files with 8671 additions and 5 deletions
|
@ -1697,7 +1697,7 @@ public class IndicesClientIT extends ESRestHighLevelClientTestCase {
|
||||||
assertTrue(template2.mappings().containsKey("custom_doc_type"));
|
assertTrue(template2.mappings().containsKey("custom_doc_type"));
|
||||||
|
|
||||||
List<String> names = randomBoolean()
|
List<String> names = randomBoolean()
|
||||||
? Arrays.asList("*-1", "template-2")
|
? Arrays.asList("*plate-1", "template-2")
|
||||||
: Arrays.asList("template-*");
|
: Arrays.asList("template-*");
|
||||||
GetIndexTemplatesRequest getBothRequest = new GetIndexTemplatesRequest(names);
|
GetIndexTemplatesRequest getBothRequest = new GetIndexTemplatesRequest(names);
|
||||||
org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesResponse getBoth = execute(
|
org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesResponse getBoth = execute(
|
||||||
|
@ -1780,7 +1780,7 @@ public class IndicesClientIT extends ESRestHighLevelClientTestCase {
|
||||||
|
|
||||||
|
|
||||||
List<String> names = randomBoolean()
|
List<String> names = randomBoolean()
|
||||||
? Arrays.asList("*-1", "template-2")
|
? Arrays.asList("*plate-1", "template-2")
|
||||||
: Arrays.asList("template-*");
|
: Arrays.asList("template-*");
|
||||||
GetIndexTemplatesRequest getBothRequest = new GetIndexTemplatesRequest(names);
|
GetIndexTemplatesRequest getBothRequest = new GetIndexTemplatesRequest(names);
|
||||||
GetIndexTemplatesResponse getBoth = execute(
|
GetIndexTemplatesResponse getBoth = execute(
|
||||||
|
@ -1834,7 +1834,7 @@ public class IndicesClientIT extends ESRestHighLevelClientTestCase {
|
||||||
|
|
||||||
{
|
{
|
||||||
final List<String> templateNames = randomBoolean()
|
final List<String> templateNames = randomBoolean()
|
||||||
? Arrays.asList("*-1", "template-2")
|
? Arrays.asList("*plate-1", "template-2")
|
||||||
: Arrays.asList("template-*");
|
: Arrays.asList("template-*");
|
||||||
|
|
||||||
final IndexTemplatesExistRequest bothRequest = new IndexTemplatesExistRequest(templateNames);
|
final IndexTemplatesExistRequest bothRequest = new IndexTemplatesExistRequest(templateNames);
|
||||||
|
|
|
@ -68,6 +68,11 @@ Example response:
|
||||||
"available" : true,
|
"available" : true,
|
||||||
"enabled" : true
|
"enabled" : true
|
||||||
},
|
},
|
||||||
|
"data_frame" : {
|
||||||
|
"description" : "Data Frame for the Elastic Stack",
|
||||||
|
"available" : true,
|
||||||
|
"enabled" : true
|
||||||
|
},
|
||||||
"graph" : {
|
"graph" : {
|
||||||
"description" : "Graph Data Exploration for the Elastic Stack",
|
"description" : "Graph Data Exploration for the Elastic Stack",
|
||||||
"available" : true,
|
"available" : true,
|
||||||
|
|
|
@ -553,6 +553,15 @@ public class XPackLicenseState {
|
||||||
return isPlatinumOrTrialOperationMode(operationMode);
|
return isPlatinumOrTrialOperationMode(operationMode);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Data Frame is always available as long as there is a valid license
|
||||||
|
*
|
||||||
|
* @return true if the license is active
|
||||||
|
*/
|
||||||
|
public synchronized boolean isDataFrameAllowed() {
|
||||||
|
return status.active;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Rollup is always available as long as there is a valid license
|
* Rollup is always available as long as there is a valid license
|
||||||
*
|
*
|
||||||
|
|
|
@ -50,6 +50,7 @@ public final class ClientHelper {
|
||||||
public static final String DEPRECATION_ORIGIN = "deprecation";
|
public static final String DEPRECATION_ORIGIN = "deprecation";
|
||||||
public static final String PERSISTENT_TASK_ORIGIN = "persistent_tasks";
|
public static final String PERSISTENT_TASK_ORIGIN = "persistent_tasks";
|
||||||
public static final String ROLLUP_ORIGIN = "rollup";
|
public static final String ROLLUP_ORIGIN = "rollup";
|
||||||
|
public static final String DATA_FRAME_ORIGIN = "data_frame";
|
||||||
|
|
||||||
private ClientHelper() {}
|
private ClientHelper() {}
|
||||||
|
|
||||||
|
|
|
@ -42,6 +42,7 @@ import org.elasticsearch.xpack.core.action.XPackUsageAction;
|
||||||
import org.elasticsearch.xpack.core.beats.BeatsFeatureSetUsage;
|
import org.elasticsearch.xpack.core.beats.BeatsFeatureSetUsage;
|
||||||
import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata;
|
import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata;
|
||||||
import org.elasticsearch.xpack.core.ccr.CCRFeatureSet;
|
import org.elasticsearch.xpack.core.ccr.CCRFeatureSet;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameFeatureSetUsage;
|
||||||
import org.elasticsearch.xpack.core.deprecation.DeprecationInfoAction;
|
import org.elasticsearch.xpack.core.deprecation.DeprecationInfoAction;
|
||||||
import org.elasticsearch.xpack.core.graph.GraphFeatureSetUsage;
|
import org.elasticsearch.xpack.core.graph.GraphFeatureSetUsage;
|
||||||
import org.elasticsearch.xpack.core.graph.action.GraphExploreAction;
|
import org.elasticsearch.xpack.core.graph.action.GraphExploreAction;
|
||||||
|
@ -439,8 +440,9 @@ public class XPackClientPlugin extends Plugin implements ActionPlugin, NetworkPl
|
||||||
new NamedWriteableRegistry.Entry(LifecycleAction.class, DeleteAction.NAME, DeleteAction::new),
|
new NamedWriteableRegistry.Entry(LifecycleAction.class, DeleteAction.NAME, DeleteAction::new),
|
||||||
new NamedWriteableRegistry.Entry(LifecycleAction.class, FreezeAction.NAME, FreezeAction::new),
|
new NamedWriteableRegistry.Entry(LifecycleAction.class, FreezeAction.NAME, FreezeAction::new),
|
||||||
new NamedWriteableRegistry.Entry(LifecycleAction.class, SetPriorityAction.NAME, SetPriorityAction::new),
|
new NamedWriteableRegistry.Entry(LifecycleAction.class, SetPriorityAction.NAME, SetPriorityAction::new),
|
||||||
new NamedWriteableRegistry.Entry(LifecycleAction.class, UnfollowAction.NAME, UnfollowAction::new)
|
new NamedWriteableRegistry.Entry(LifecycleAction.class, UnfollowAction.NAME, UnfollowAction::new),
|
||||||
);
|
// Data Frame
|
||||||
|
new NamedWriteableRegistry.Entry(XPackFeatureSet.Usage.class, XPackField.DATA_FRAME, DataFrameFeatureSetUsage::new));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -35,6 +35,8 @@ public final class XPackField {
|
||||||
public static final String INDEX_LIFECYCLE = "ilm";
|
public static final String INDEX_LIFECYCLE = "ilm";
|
||||||
/** Name constant for the CCR feature. */
|
/** Name constant for the CCR feature. */
|
||||||
public static final String CCR = "ccr";
|
public static final String CCR = "ccr";
|
||||||
|
/** Name constant for the data frame feature. */
|
||||||
|
public static final String DATA_FRAME = "data_frame";
|
||||||
|
|
||||||
private XPackField() {}
|
private XPackField() {}
|
||||||
|
|
||||||
|
|
|
@ -44,6 +44,10 @@ public class XPackSettings {
|
||||||
*/
|
*/
|
||||||
public static final Setting<Boolean> CCR_ENABLED_SETTING = Setting.boolSetting("xpack.ccr.enabled", true, Property.NodeScope);
|
public static final Setting<Boolean> CCR_ENABLED_SETTING = Setting.boolSetting("xpack.ccr.enabled", true, Property.NodeScope);
|
||||||
|
|
||||||
|
/** Setting for enabling or disabling data frame. Defaults to true. */
|
||||||
|
public static final Setting<Boolean> DATA_FRAME_ENABLED = Setting.boolSetting("xpack.data_frame.enabled", true,
|
||||||
|
Setting.Property.NodeScope);
|
||||||
|
|
||||||
/** Setting for enabling or disabling security. Defaults to true. */
|
/** Setting for enabling or disabling security. Defaults to true. */
|
||||||
public static final Setting<Boolean> SECURITY_ENABLED = Setting.boolSetting("xpack.security.enabled", true, Setting.Property.NodeScope);
|
public static final Setting<Boolean> SECURITY_ENABLED = Setting.boolSetting("xpack.security.enabled", true, Setting.Property.NodeScope);
|
||||||
|
|
||||||
|
@ -209,6 +213,7 @@ public class XPackSettings {
|
||||||
settings.add(ROLLUP_ENABLED);
|
settings.add(ROLLUP_ENABLED);
|
||||||
settings.add(PASSWORD_HASHING_ALGORITHM);
|
settings.add(PASSWORD_HASHING_ALGORITHM);
|
||||||
settings.add(INDEX_LIFECYCLE_ENABLED);
|
settings.add(INDEX_LIFECYCLE_ENABLED);
|
||||||
|
settings.add(DATA_FRAME_ENABLED);
|
||||||
return Collections.unmodifiableList(settings);
|
return Collections.unmodifiableList(settings);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,83 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.core.dataframe;
|
||||||
|
|
||||||
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.xpack.core.XPackFeatureSet.Usage;
|
||||||
|
import org.elasticsearch.xpack.core.XPackField;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameIndexerTransformStats;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Map.Entry;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
public class DataFrameFeatureSetUsage extends Usage {
|
||||||
|
|
||||||
|
private final Map<String, Long> transformCountByState;
|
||||||
|
private final DataFrameIndexerTransformStats accumulatedStats;
|
||||||
|
|
||||||
|
public DataFrameFeatureSetUsage(StreamInput in) throws IOException {
|
||||||
|
super(in);
|
||||||
|
this.transformCountByState = in.readMap(StreamInput::readString, StreamInput::readLong);
|
||||||
|
this.accumulatedStats = new DataFrameIndexerTransformStats(in);
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameFeatureSetUsage(boolean available, boolean enabled, Map<String, Long> transformCountByState,
|
||||||
|
DataFrameIndexerTransformStats accumulatedStats) {
|
||||||
|
super(XPackField.DATA_FRAME, available, enabled);
|
||||||
|
this.transformCountByState = Objects.requireNonNull(transformCountByState);
|
||||||
|
this.accumulatedStats = Objects.requireNonNull(accumulatedStats);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
super.writeTo(out);
|
||||||
|
out.writeMap(transformCountByState, StreamOutput::writeString, StreamOutput::writeLong);
|
||||||
|
accumulatedStats.writeTo(out);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void innerXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
super.innerXContent(builder, params);
|
||||||
|
if (transformCountByState.isEmpty() == false) {
|
||||||
|
builder.startObject(DataFrameField.TRANSFORMS.getPreferredName());
|
||||||
|
long all = 0L;
|
||||||
|
for (Entry<String, Long> entry : transformCountByState.entrySet()) {
|
||||||
|
builder.field(entry.getKey(), entry.getValue());
|
||||||
|
all+=entry.getValue();
|
||||||
|
}
|
||||||
|
builder.field(MetaData.ALL, all);
|
||||||
|
builder.endObject();
|
||||||
|
|
||||||
|
// if there are no transforms, do not show any stats
|
||||||
|
builder.field(DataFrameField.STATS_FIELD.getPreferredName(), accumulatedStats);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(enabled, available, transformCountByState, accumulatedStats);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object obj) {
|
||||||
|
if (obj == null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (getClass() != obj.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
DataFrameFeatureSetUsage other = (DataFrameFeatureSetUsage) obj;
|
||||||
|
return Objects.equals(name, other.name) && available == other.available && enabled == other.enabled
|
||||||
|
&& Objects.equals(transformCountByState, other.transformCountByState)
|
||||||
|
&& Objects.equals(accumulatedStats, other.accumulatedStats);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,46 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.core.dataframe;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.ParseField;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Utility class to hold common fields and strings for data frame.
|
||||||
|
*/
|
||||||
|
public final class DataFrameField {
|
||||||
|
|
||||||
|
// common parse fields
|
||||||
|
public static final ParseField AGGREGATIONS = new ParseField("aggregations");
|
||||||
|
public static final ParseField AGGS = new ParseField("aggs");
|
||||||
|
public static final ParseField ID = new ParseField("id");
|
||||||
|
public static final ParseField TRANSFORMS = new ParseField("transforms");
|
||||||
|
public static final ParseField COUNT = new ParseField("count");
|
||||||
|
public static final ParseField GROUP_BY = new ParseField("group_by");
|
||||||
|
public static final ParseField TIMEOUT = new ParseField("timeout");
|
||||||
|
public static final ParseField WAIT_FOR_COMPLETION = new ParseField("wait_for_completion");
|
||||||
|
public static final ParseField STATS_FIELD = new ParseField("stats");
|
||||||
|
|
||||||
|
// common strings
|
||||||
|
public static final String TASK_NAME = "data_frame/transforms";
|
||||||
|
public static final String REST_BASE_PATH = "/_data_frame/";
|
||||||
|
public static final String REST_BASE_PATH_TRANSFORMS_BY_ID = REST_BASE_PATH + "transforms/{id}/";
|
||||||
|
|
||||||
|
// note: this is used to match tasks
|
||||||
|
public static final String PERSISTENT_TASK_DESCRIPTION_PREFIX = "data_frame_";
|
||||||
|
|
||||||
|
// strings for meta information
|
||||||
|
public static final String META_FIELDNAME = "_data_frame";
|
||||||
|
public static final String CREATION_DATE_MILLIS = "creation_date_in_millis";
|
||||||
|
public static final String VERSION = "version";
|
||||||
|
public static final String CREATED = "created";
|
||||||
|
public static final String CREATED_BY = "created_by";
|
||||||
|
public static final String TRANSFORM = "transform";
|
||||||
|
public static final String DATA_FRAME_SIGNATURE = "data-frame-transform";
|
||||||
|
|
||||||
|
private DataFrameField() {
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,73 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.core.dataframe;
|
||||||
|
|
||||||
|
import java.text.MessageFormat;
|
||||||
|
import java.util.Locale;
|
||||||
|
|
||||||
|
public class DataFrameMessages {
|
||||||
|
|
||||||
|
public static final String REST_STOP_TRANSFORM_WAIT_FOR_COMPLETION_TIMEOUT =
|
||||||
|
"Timed out after [{0}] while waiting for data frame transform [{1}] to stop";
|
||||||
|
public static final String REST_STOP_TRANSFORM_WAIT_FOR_COMPLETION_INTERRUPT =
|
||||||
|
"Interrupted while waiting for data frame transform [{0}] to stop";
|
||||||
|
public static final String REST_PUT_DATA_FRAME_TRANSFORM_EXISTS = "Transform with id [{0}] already exists";
|
||||||
|
public static final String REST_DATA_FRAME_UNKNOWN_TRANSFORM = "Transform with id [{0}] could not be found";
|
||||||
|
public static final String REST_PUT_DATA_FRAME_FAILED_TO_VALIDATE_DATA_FRAME_CONFIGURATION =
|
||||||
|
"Failed to validate data frame configuration";
|
||||||
|
public static final String REST_PUT_DATA_FRAME_FAILED_PERSIST_TRANSFORM_CONFIGURATION = "Failed to persist data frame configuration";
|
||||||
|
public static final String REST_PUT_DATA_FRAME_FAILED_TO_DEDUCE_TARGET_MAPPINGS = "Failed to deduce target mappings";
|
||||||
|
public static final String REST_PUT_DATA_FRAME_FAILED_TO_CREATE_TARGET_INDEX = "Failed to create target index";
|
||||||
|
public static final String REST_PUT_DATA_FRAME_FAILED_TO_START_PERSISTENT_TASK =
|
||||||
|
"Failed to start persistent task, configuration has been cleaned up: [{0}]";
|
||||||
|
public static final String REST_DATA_FRAME_FAILED_TO_SERIALIZE_TRANSFORM = "Failed to serialise transform [{0}]";
|
||||||
|
|
||||||
|
public static final String FAILED_TO_CREATE_DESTINATION_INDEX = "Could not create destination index [{0}] for transform[{1}]";
|
||||||
|
public static final String FAILED_TO_LOAD_TRANSFORM_CONFIGURATION =
|
||||||
|
"Failed to load data frame transform configuration for transform [{0}]";
|
||||||
|
public static final String FAILED_TO_PARSE_TRANSFORM_CONFIGURATION =
|
||||||
|
"Failed to parse transform configuration for data frame transform [{0}]";
|
||||||
|
public static final String DATA_FRAME_TRANSFORM_CONFIGURATION_NO_TRANSFORM =
|
||||||
|
"Data frame transform configuration must specify exactly 1 function";
|
||||||
|
public static final String DATA_FRAME_TRANSFORM_CONFIGURATION_PIVOT_NO_GROUP_BY =
|
||||||
|
"Data frame pivot transform configuration must specify at least 1 group_by";
|
||||||
|
public static final String DATA_FRAME_TRANSFORM_CONFIGURATION_PIVOT_NO_AGGREGATION =
|
||||||
|
"Data frame pivot transform configuration must specify at least 1 aggregation";
|
||||||
|
public static final String DATA_FRAME_TRANSFORM_PIVOT_FAILED_TO_CREATE_COMPOSITE_AGGREGATION =
|
||||||
|
"Failed to create composite aggregation from pivot function";
|
||||||
|
public static final String DATA_FRAME_TRANSFORM_CONFIGURATION_INVALID =
|
||||||
|
"Data frame transform configuration [{0}] has invalid elements";
|
||||||
|
|
||||||
|
public static final String LOG_DATA_FRAME_TRANSFORM_CONFIGURATION_BAD_QUERY =
|
||||||
|
"Failed to parse query for data frame transform";
|
||||||
|
public static final String LOG_DATA_FRAME_TRANSFORM_CONFIGURATION_BAD_GROUP_BY =
|
||||||
|
"Failed to parse group_by for data frame pivot transform";
|
||||||
|
public static final String LOG_DATA_FRAME_TRANSFORM_CONFIGURATION_BAD_AGGREGATION =
|
||||||
|
"Failed to parse aggregation for data frame pivot transform";
|
||||||
|
|
||||||
|
private DataFrameMessages() {
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the message parameter
|
||||||
|
*
|
||||||
|
* @param message Should be one of the statics defined in this class
|
||||||
|
*/
|
||||||
|
public static String getMessage(String message) {
|
||||||
|
return message;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Format the message with the supplied arguments
|
||||||
|
*
|
||||||
|
* @param message Should be one of the statics defined in this class
|
||||||
|
* @param args MessageFormat arguments. See {@linkplain MessageFormat#format(Object)}]
|
||||||
|
*/
|
||||||
|
public static String getMessage(String message, Object... args) {
|
||||||
|
return new MessageFormat(message, Locale.ROOT).format(args);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,102 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.core.dataframe.transform;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.ParseField;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.xpack.core.indexing.IndexerJobStats;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg;
|
||||||
|
|
||||||
|
public class DataFrameIndexerTransformStats extends IndexerJobStats {
|
||||||
|
private static ParseField NUM_PAGES = new ParseField("pages_processed");
|
||||||
|
private static ParseField NUM_INPUT_DOCUMENTS = new ParseField("documents_processed");
|
||||||
|
private static ParseField NUM_OUTPUT_DOCUMENTS = new ParseField("documents_indexed");
|
||||||
|
private static ParseField NUM_INVOCATIONS = new ParseField("trigger_count");
|
||||||
|
private static ParseField INDEX_TIME_IN_MS = new ParseField("index_time_in_ms");
|
||||||
|
private static ParseField SEARCH_TIME_IN_MS = new ParseField("search_time_in_ms");
|
||||||
|
private static ParseField INDEX_TOTAL = new ParseField("index_total");
|
||||||
|
private static ParseField SEARCH_TOTAL = new ParseField("search_total");
|
||||||
|
private static ParseField SEARCH_FAILURES = new ParseField("search_failures");
|
||||||
|
private static ParseField INDEX_FAILURES = new ParseField("index_failures");
|
||||||
|
|
||||||
|
public static final ConstructingObjectParser<DataFrameIndexerTransformStats, Void> PARSER = new ConstructingObjectParser<>(
|
||||||
|
NAME.getPreferredName(), args -> new DataFrameIndexerTransformStats((long) args[0], (long) args[1], (long) args[2],
|
||||||
|
(long) args[3], (long) args[4], (long) args[5], (long) args[6], (long) args[7], (long) args[8], (long) args[9]));
|
||||||
|
|
||||||
|
static {
|
||||||
|
PARSER.declareLong(constructorArg(), NUM_PAGES);
|
||||||
|
PARSER.declareLong(constructorArg(), NUM_INPUT_DOCUMENTS);
|
||||||
|
PARSER.declareLong(constructorArg(), NUM_OUTPUT_DOCUMENTS);
|
||||||
|
PARSER.declareLong(constructorArg(), NUM_INVOCATIONS);
|
||||||
|
PARSER.declareLong(constructorArg(), INDEX_TIME_IN_MS);
|
||||||
|
PARSER.declareLong(constructorArg(), SEARCH_TIME_IN_MS);
|
||||||
|
PARSER.declareLong(constructorArg(), INDEX_TOTAL);
|
||||||
|
PARSER.declareLong(constructorArg(), SEARCH_TOTAL);
|
||||||
|
PARSER.declareLong(constructorArg(), INDEX_FAILURES);
|
||||||
|
PARSER.declareLong(constructorArg(), SEARCH_FAILURES);
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameIndexerTransformStats() {
|
||||||
|
super();
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameIndexerTransformStats(long numPages, long numInputDocuments, long numOuputDocuments, long numInvocations,
|
||||||
|
long indexTime, long searchTime, long indexTotal, long searchTotal, long indexFailures, long searchFailures) {
|
||||||
|
super(numPages, numInputDocuments, numOuputDocuments, numInvocations, indexTime, searchTime, indexTotal, searchTotal, indexFailures,
|
||||||
|
searchFailures);
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameIndexerTransformStats(StreamInput in) throws IOException {
|
||||||
|
super(in);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
builder.field(NUM_PAGES.getPreferredName(), numPages);
|
||||||
|
builder.field(NUM_INPUT_DOCUMENTS.getPreferredName(), numInputDocuments);
|
||||||
|
builder.field(NUM_OUTPUT_DOCUMENTS.getPreferredName(), numOuputDocuments);
|
||||||
|
builder.field(NUM_INVOCATIONS.getPreferredName(), numInvocations);
|
||||||
|
builder.field(INDEX_TIME_IN_MS.getPreferredName(), indexTime);
|
||||||
|
builder.field(INDEX_TOTAL.getPreferredName(), indexTotal);
|
||||||
|
builder.field(INDEX_FAILURES.getPreferredName(), indexFailures);
|
||||||
|
builder.field(SEARCH_TIME_IN_MS.getPreferredName(), searchTime);
|
||||||
|
builder.field(SEARCH_TOTAL.getPreferredName(), searchTotal);
|
||||||
|
builder.field(SEARCH_FAILURES.getPreferredName(), searchFailures);
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameIndexerTransformStats merge(DataFrameIndexerTransformStats other) {
|
||||||
|
numPages += other.numPages;
|
||||||
|
numInputDocuments += other.numInputDocuments;
|
||||||
|
numOuputDocuments += other.numOuputDocuments;
|
||||||
|
numInvocations += other.numInvocations;
|
||||||
|
indexTime += other.indexTime;
|
||||||
|
searchTime += other.searchTime;
|
||||||
|
indexTotal += other.indexTotal;
|
||||||
|
searchTotal += other.searchTotal;
|
||||||
|
indexFailures += other.indexFailures;
|
||||||
|
searchFailures += other.searchFailures;
|
||||||
|
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DataFrameIndexerTransformStats fromXContent(XContentParser parser) {
|
||||||
|
try {
|
||||||
|
return PARSER.parse(parser, null);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,149 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.core.dataframe.transform;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.Nullable;
|
||||||
|
import org.elasticsearch.common.ParseField;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||||
|
import org.elasticsearch.common.xcontent.ObjectParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.persistent.PersistentTaskState;
|
||||||
|
import org.elasticsearch.tasks.Task;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.core.indexing.IndexerState;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.SortedMap;
|
||||||
|
import java.util.TreeMap;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg;
|
||||||
|
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg;
|
||||||
|
|
||||||
|
public class DataFrameTransformState implements Task.Status, PersistentTaskState {
|
||||||
|
public static final String NAME = DataFrameField.TASK_NAME;
|
||||||
|
|
||||||
|
private final IndexerState state;
|
||||||
|
private final long generation;
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
private final SortedMap<String, Object> currentPosition;
|
||||||
|
|
||||||
|
private static final ParseField STATE = new ParseField("transform_state");
|
||||||
|
private static final ParseField CURRENT_POSITION = new ParseField("current_position");
|
||||||
|
private static final ParseField GENERATION = new ParseField("generation");
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public static final ConstructingObjectParser<DataFrameTransformState, Void> PARSER = new ConstructingObjectParser<>(NAME,
|
||||||
|
args -> new DataFrameTransformState((IndexerState) args[0], (HashMap<String, Object>) args[1], (long) args[2]));
|
||||||
|
|
||||||
|
static {
|
||||||
|
PARSER.declareField(constructorArg(), p -> {
|
||||||
|
if (p.currentToken() == XContentParser.Token.VALUE_STRING) {
|
||||||
|
return IndexerState.fromString(p.text());
|
||||||
|
}
|
||||||
|
throw new IllegalArgumentException("Unsupported token [" + p.currentToken() + "]");
|
||||||
|
|
||||||
|
}, STATE, ObjectParser.ValueType.STRING);
|
||||||
|
PARSER.declareField(optionalConstructorArg(), p -> {
|
||||||
|
if (p.currentToken() == XContentParser.Token.START_OBJECT) {
|
||||||
|
return p.map();
|
||||||
|
}
|
||||||
|
if (p.currentToken() == XContentParser.Token.VALUE_NULL) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
throw new IllegalArgumentException("Unsupported token [" + p.currentToken() + "]");
|
||||||
|
}, CURRENT_POSITION, ObjectParser.ValueType.VALUE_OBJECT_ARRAY);
|
||||||
|
PARSER.declareLong(ConstructingObjectParser.optionalConstructorArg(), GENERATION);
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameTransformState(IndexerState state, @Nullable Map<String, Object> position, long generation) {
|
||||||
|
this.state = state;
|
||||||
|
this.currentPosition = position == null ? null : Collections.unmodifiableSortedMap(new TreeMap<>(position));
|
||||||
|
this.generation = generation;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameTransformState(StreamInput in) throws IOException {
|
||||||
|
state = IndexerState.fromStream(in);
|
||||||
|
currentPosition = in.readBoolean() ? Collections.unmodifiableSortedMap(new TreeMap<>(in.readMap())) : null;
|
||||||
|
generation = in.readLong();
|
||||||
|
}
|
||||||
|
|
||||||
|
public IndexerState getIndexerState() {
|
||||||
|
return state;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Map<String, Object> getPosition() {
|
||||||
|
return currentPosition;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getGeneration() {
|
||||||
|
return generation;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DataFrameTransformState fromXContent(XContentParser parser) {
|
||||||
|
try {
|
||||||
|
return PARSER.parse(parser, null);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
builder.field(STATE.getPreferredName(), state.value());
|
||||||
|
if (currentPosition != null) {
|
||||||
|
builder.field(CURRENT_POSITION.getPreferredName(), currentPosition);
|
||||||
|
}
|
||||||
|
builder.field(GENERATION.getPreferredName(), generation);
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getWriteableName() {
|
||||||
|
return NAME;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
state.writeTo(out);
|
||||||
|
out.writeBoolean(currentPosition != null);
|
||||||
|
if (currentPosition != null) {
|
||||||
|
out.writeMap(currentPosition);
|
||||||
|
}
|
||||||
|
out.writeLong(generation);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object other) {
|
||||||
|
if (this == other) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (other == null || getClass() != other.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
DataFrameTransformState that = (DataFrameTransformState) other;
|
||||||
|
|
||||||
|
return Objects.equals(this.state, that.state) && Objects.equals(this.currentPosition, that.currentPosition)
|
||||||
|
&& this.generation == that.generation;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(state, currentPosition, generation);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,36 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.core.dataframe;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable.Reader;
|
||||||
|
import org.elasticsearch.test.AbstractWireSerializingTestCase;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameIndexerTransformStatsTests;
|
||||||
|
import org.elasticsearch.xpack.core.indexing.IndexerState;
|
||||||
|
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class DataFrameFeatureSetUsageTests extends AbstractWireSerializingTestCase<DataFrameFeatureSetUsage> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected DataFrameFeatureSetUsage createTestInstance() {
|
||||||
|
Map<String, Long> transformCountByState = new HashMap<>();
|
||||||
|
|
||||||
|
if (randomBoolean()) {
|
||||||
|
transformCountByState.put(randomFrom(IndexerState.values()).toString(), randomLong());
|
||||||
|
}
|
||||||
|
|
||||||
|
return new DataFrameFeatureSetUsage(randomBoolean(), randomBoolean(), transformCountByState,
|
||||||
|
DataFrameIndexerTransformStatsTests.randomStats());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Reader<DataFrameFeatureSetUsage> instanceReader() {
|
||||||
|
return DataFrameFeatureSetUsage::new;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,74 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.core.dataframe;
|
||||||
|
|
||||||
|
import org.elasticsearch.test.ESTestCase;
|
||||||
|
|
||||||
|
import java.lang.reflect.Field;
|
||||||
|
import java.text.MessageFormat;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Locale;
|
||||||
|
|
||||||
|
public class DataFrameMessagesTests extends ESTestCase {
|
||||||
|
|
||||||
|
public void testGetMessage_WithFormatStrings() {
|
||||||
|
String formattedMessage = DataFrameMessages.getMessage(DataFrameMessages.REST_STOP_TRANSFORM_WAIT_FOR_COMPLETION_TIMEOUT, "30s",
|
||||||
|
"my_transform");
|
||||||
|
assertEquals("Timed out after [30s] while waiting for data frame transform [my_transform] to stop", formattedMessage);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testMessageProperFormat() throws IllegalArgumentException, IllegalAccessException {
|
||||||
|
Field[] declaredFields = DataFrameMessages.class.getFields();
|
||||||
|
int checkedMessages = 0;
|
||||||
|
|
||||||
|
for (Field field : declaredFields) {
|
||||||
|
int modifiers = field.getModifiers();
|
||||||
|
if (java.lang.reflect.Modifier.isStatic(modifiers) && java.lang.reflect.Modifier.isFinal(modifiers)
|
||||||
|
&& field.getType().isAssignableFrom(String.class)) {
|
||||||
|
|
||||||
|
assertSingleMessage((String) field.get(DataFrameMessages.class));
|
||||||
|
++checkedMessages;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
assertTrue(checkedMessages > 0);
|
||||||
|
logger.info("Checked {} messages", checkedMessages);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testAssertSingleMessage() {
|
||||||
|
expectThrows(RuntimeException.class, () -> innerAssertSingleMessage("missing zero position {1} {1}"));
|
||||||
|
expectThrows(RuntimeException.class, () -> innerAssertSingleMessage("incomplete {}"));
|
||||||
|
expectThrows(RuntimeException.class, () -> innerAssertSingleMessage("count from 1 {1}"));
|
||||||
|
}
|
||||||
|
|
||||||
|
private void assertSingleMessage(String message) {
|
||||||
|
// for testing the test method, we can not assert directly, but wrap it with an exception, which also
|
||||||
|
// nicely encapsulate parsing errors thrown by MessageFormat itself
|
||||||
|
try {
|
||||||
|
innerAssertSingleMessage(message);
|
||||||
|
} catch (Exception e) {
|
||||||
|
fail(e.getMessage());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void innerAssertSingleMessage(String message) {
|
||||||
|
MessageFormat messageWithNoArguments = new MessageFormat(message, Locale.ROOT);
|
||||||
|
int numberOfArguments = messageWithNoArguments.getFormats().length;
|
||||||
|
|
||||||
|
List<String> args = new ArrayList<>();
|
||||||
|
for (int i = 0; i < numberOfArguments; ++i) {
|
||||||
|
args.add(randomAlphaOfLength(5));
|
||||||
|
}
|
||||||
|
|
||||||
|
String properFormatedMessage = new MessageFormat(message, Locale.ROOT).format(args.toArray(new String[0]));
|
||||||
|
for (String arg : args) {
|
||||||
|
if (properFormatedMessage.contains(arg) == false) {
|
||||||
|
throw new RuntimeException("Message check: [" + message + "] failed, missing argument");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,53 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.core.dataframe.transform;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.test.AbstractSerializingTestCase;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public class DataFrameIndexerTransformStatsTests extends AbstractSerializingTestCase<DataFrameIndexerTransformStats> {
|
||||||
|
@Override
|
||||||
|
protected DataFrameIndexerTransformStats createTestInstance() {
|
||||||
|
return randomStats();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Writeable.Reader<DataFrameIndexerTransformStats> instanceReader() {
|
||||||
|
return DataFrameIndexerTransformStats::new;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected DataFrameIndexerTransformStats doParseInstance(XContentParser parser) {
|
||||||
|
return DataFrameIndexerTransformStats.fromXContent(parser);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DataFrameIndexerTransformStats randomStats() {
|
||||||
|
return new DataFrameIndexerTransformStats(randomLongBetween(0L, 10000L), randomLongBetween(0L, 10000L),
|
||||||
|
randomLongBetween(0L, 10000L), randomLongBetween(0L, 10000L), randomLongBetween(0L, 10000L), randomLongBetween(0L, 10000L),
|
||||||
|
randomLongBetween(0L, 10000L), randomLongBetween(0L, 10000L), randomLongBetween(0L, 10000L), randomLongBetween(0L, 10000L));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testMerge() throws IOException {
|
||||||
|
DataFrameIndexerTransformStats emptyStats = new DataFrameIndexerTransformStats();
|
||||||
|
DataFrameIndexerTransformStats randomStats = randomStats();
|
||||||
|
|
||||||
|
assertEquals(randomStats, emptyStats.merge(randomStats));
|
||||||
|
assertEquals(randomStats, randomStats.merge(emptyStats));
|
||||||
|
|
||||||
|
DataFrameIndexerTransformStats randomStatsClone = copyInstance(randomStats);
|
||||||
|
|
||||||
|
DataFrameIndexerTransformStats trippleRandomStats = new DataFrameIndexerTransformStats(3 * randomStats.getNumPages(),
|
||||||
|
3 * randomStats.getNumDocuments(), 3 * randomStats.getOutputDocuments(), 3 * randomStats.getNumInvocations(),
|
||||||
|
3 * randomStats.getIndexTime(), 3 * randomStats.getSearchTime(), 3 * randomStats.getIndexTotal(),
|
||||||
|
3 * randomStats.getSearchTotal(), 3 * randomStats.getIndexFailures(), 3 * randomStats.getSearchFailures());
|
||||||
|
|
||||||
|
assertEquals(trippleRandomStats, randomStats.merge(randomStatsClone).merge(randomStatsClone));
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,57 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.core.dataframe.transform;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable.Reader;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.test.AbstractSerializingTestCase;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameTransformState;
|
||||||
|
import org.elasticsearch.xpack.core.indexing.IndexerState;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class DataFrameTransformStateTests extends AbstractSerializingTestCase<DataFrameTransformState> {
|
||||||
|
|
||||||
|
public static DataFrameTransformState randomDataFrameTransformState() {
|
||||||
|
return new DataFrameTransformState(randomFrom(IndexerState.values()), randomPosition(), randomLongBetween(0,10));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected DataFrameTransformState doParseInstance(XContentParser parser) throws IOException {
|
||||||
|
return DataFrameTransformState.fromXContent(parser);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected DataFrameTransformState createTestInstance() {
|
||||||
|
return randomDataFrameTransformState();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Reader<DataFrameTransformState> instanceReader() {
|
||||||
|
return DataFrameTransformState::new;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Map<String, Object> randomPosition() {
|
||||||
|
if (randomBoolean()) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
int numFields = randomIntBetween(1, 5);
|
||||||
|
Map<String, Object> position = new HashMap<>();
|
||||||
|
for (int i = 0; i < numFields; i++) {
|
||||||
|
Object value;
|
||||||
|
if (randomBoolean()) {
|
||||||
|
value = randomLong();
|
||||||
|
} else {
|
||||||
|
value = randomAlphaOfLengthBetween(1, 10);
|
||||||
|
}
|
||||||
|
position.put(randomAlphaOfLengthBetween(3, 10), value);
|
||||||
|
}
|
||||||
|
return position;
|
||||||
|
}
|
||||||
|
}
|
37
x-pack/plugin/data-frame/build.gradle
Normal file
37
x-pack/plugin/data-frame/build.gradle
Normal file
|
@ -0,0 +1,37 @@
|
||||||
|
evaluationDependsOn(xpackModule('core'))
|
||||||
|
|
||||||
|
apply plugin: 'elasticsearch.esplugin'
|
||||||
|
esplugin {
|
||||||
|
name 'data-frame'
|
||||||
|
description 'A plugin to build data frames'
|
||||||
|
classname 'org.elasticsearch.xpack.dataframe.DataFrame'
|
||||||
|
extendedPlugins = ['x-pack-core']
|
||||||
|
}
|
||||||
|
|
||||||
|
compileJava.options.compilerArgs << "-Xlint:-rawtypes"
|
||||||
|
compileTestJava.options.compilerArgs << "-Xlint:-rawtypes"
|
||||||
|
|
||||||
|
dependencies {
|
||||||
|
compileOnly "org.elasticsearch:elasticsearch:${version}"
|
||||||
|
|
||||||
|
compileOnly project(path: xpackModule('core'), configuration: 'default')
|
||||||
|
testCompile project(path: xpackModule('core'), configuration: 'testArtifacts')
|
||||||
|
}
|
||||||
|
|
||||||
|
run {
|
||||||
|
plugin xpackModule('core')
|
||||||
|
}
|
||||||
|
|
||||||
|
// xpack modules are installed in real clusters as the meta plugin, so
|
||||||
|
// installing them as individual plugins for integ tests doesn't make sense,
|
||||||
|
// so we disable integ tests
|
||||||
|
integTest.enabled = false
|
||||||
|
|
||||||
|
// add all sub-projects of the qa sub-project
|
||||||
|
gradle.projectsEvaluated {
|
||||||
|
project.subprojects
|
||||||
|
.find { it.path == project.path + ":qa" }
|
||||||
|
.subprojects
|
||||||
|
.findAll { it.path.startsWith(project.path + ":qa") }
|
||||||
|
.each { check.dependsOn it.check }
|
||||||
|
}
|
14
x-pack/plugin/data-frame/qa/build.gradle
Normal file
14
x-pack/plugin/data-frame/qa/build.gradle
Normal file
|
@ -0,0 +1,14 @@
|
||||||
|
/* Remove assemble on all qa projects because we don't need to publish
|
||||||
|
* artifacts for them. */
|
||||||
|
gradle.projectsEvaluated {
|
||||||
|
subprojects {
|
||||||
|
Task assemble = project.tasks.findByName('assemble')
|
||||||
|
if (assemble) {
|
||||||
|
assemble.enabled = false
|
||||||
|
}
|
||||||
|
Task dependenciesInfo = project.tasks.findByName('dependenciesInfo')
|
||||||
|
if (dependenciesInfo) {
|
||||||
|
dependenciesInfo.enabled = false
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
12
x-pack/plugin/data-frame/qa/single-node-tests/build.gradle
Normal file
12
x-pack/plugin/data-frame/qa/single-node-tests/build.gradle
Normal file
|
@ -0,0 +1,12 @@
|
||||||
|
apply plugin: 'elasticsearch.standalone-rest-test'
|
||||||
|
apply plugin: 'elasticsearch.rest-test'
|
||||||
|
|
||||||
|
dependencies {
|
||||||
|
testCompile "org.elasticsearch.plugin:x-pack-core:${version}"
|
||||||
|
testCompile project(path: xpackModule('data-frame'), configuration: 'runtime')
|
||||||
|
}
|
||||||
|
|
||||||
|
integTestCluster {
|
||||||
|
setting 'xpack.security.enabled', 'false'
|
||||||
|
setting 'xpack.license.self_generated.type', 'trial'
|
||||||
|
}
|
|
@ -0,0 +1,58 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.integration;
|
||||||
|
|
||||||
|
import org.apache.http.entity.ContentType;
|
||||||
|
import org.apache.http.entity.StringEntity;
|
||||||
|
import org.elasticsearch.client.Request;
|
||||||
|
import org.elasticsearch.client.Response;
|
||||||
|
import org.elasticsearch.client.ResponseException;
|
||||||
|
import org.elasticsearch.common.Strings;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.dataframe.persistence.DataFrameInternalIndex;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformConfig;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||||
|
|
||||||
|
public class DataFrameConfigurationIndexIT extends DataFrameRestTestCase {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests the corner case that for some reason a transform configuration still exists in the index but
|
||||||
|
* the persistent task disappeared
|
||||||
|
*
|
||||||
|
* test note: {@link DataFrameRestTestCase} checks for an empty index as part of the test case cleanup,
|
||||||
|
* so we do not need to check that the document has been deleted in this place
|
||||||
|
*/
|
||||||
|
public void testDeleteConfigurationLeftOver() throws IOException {
|
||||||
|
String fakeTransformName = randomAlphaOfLengthBetween(5, 20);
|
||||||
|
|
||||||
|
try (XContentBuilder builder = jsonBuilder()) {
|
||||||
|
builder.startObject();
|
||||||
|
{
|
||||||
|
builder.field(DataFrameField.ID.getPreferredName(), fakeTransformName);
|
||||||
|
}
|
||||||
|
builder.endObject();
|
||||||
|
final StringEntity entity = new StringEntity(Strings.toString(builder), ContentType.APPLICATION_JSON);
|
||||||
|
Request req = new Request("PUT",
|
||||||
|
DataFrameInternalIndex.INDEX_NAME + "/_doc/" + DataFrameTransformConfig.documentId(fakeTransformName));
|
||||||
|
req.setEntity(entity);
|
||||||
|
client().performRequest(req);
|
||||||
|
}
|
||||||
|
|
||||||
|
Request deleteRequest = new Request("DELETE", DATAFRAME_ENDPOINT + fakeTransformName);
|
||||||
|
Response deleteResponse = client().performRequest(deleteRequest);
|
||||||
|
assertOK(deleteResponse);
|
||||||
|
assertTrue((boolean)XContentMapValues.extractValue("acknowledged", entityAsMap(deleteResponse)));
|
||||||
|
|
||||||
|
// delete again, should fail
|
||||||
|
expectThrows(ResponseException.class,() -> client().performRequest(deleteRequest));
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,59 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.integration;
|
||||||
|
|
||||||
|
import org.elasticsearch.Version;
|
||||||
|
import org.elasticsearch.client.Request;
|
||||||
|
import org.elasticsearch.client.Response;
|
||||||
|
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||||
|
import org.junit.Before;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class DataFrameMetaDataIT extends DataFrameRestTestCase {
|
||||||
|
|
||||||
|
private boolean indicesCreated = false;
|
||||||
|
|
||||||
|
// preserve indices in order to reuse source indices in several test cases
|
||||||
|
@Override
|
||||||
|
protected boolean preserveIndicesUponCompletion() {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void createIndexes() throws IOException {
|
||||||
|
|
||||||
|
// it's not possible to run it as @BeforeClass as clients aren't initialized then, so we need this little hack
|
||||||
|
if (indicesCreated) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
createReviewsIndex();
|
||||||
|
indicesCreated = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testMetaData() throws IOException {
|
||||||
|
long testStarted = System.currentTimeMillis();
|
||||||
|
createPivotReviewsTransform("test_meta", "pivot_reviews", null);
|
||||||
|
|
||||||
|
Response mappingResponse = client().performRequest(new Request("GET", "pivot_reviews/_mapping"));
|
||||||
|
|
||||||
|
Map<?, ?> mappingAsMap = entityAsMap(mappingResponse);
|
||||||
|
assertEquals(Version.CURRENT.toString(),
|
||||||
|
XContentMapValues.extractValue("pivot_reviews.mappings._meta._data_frame.version.created", mappingAsMap));
|
||||||
|
assertTrue((Long) XContentMapValues.extractValue("pivot_reviews.mappings._meta._data_frame.creation_date_in_millis",
|
||||||
|
mappingAsMap) < System.currentTimeMillis());
|
||||||
|
assertTrue((Long) XContentMapValues.extractValue("pivot_reviews.mappings._meta._data_frame.creation_date_in_millis",
|
||||||
|
mappingAsMap) > testStarted);
|
||||||
|
assertEquals("test_meta",
|
||||||
|
XContentMapValues.extractValue("pivot_reviews.mappings._meta._data_frame.transform", mappingAsMap));
|
||||||
|
assertEquals("data-frame-transform",
|
||||||
|
XContentMapValues.extractValue("pivot_reviews.mappings._meta.created_by", mappingAsMap));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,263 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.integration;
|
||||||
|
|
||||||
|
import org.elasticsearch.client.Request;
|
||||||
|
import org.elasticsearch.client.Response;
|
||||||
|
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||||
|
import org.junit.Before;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
|
||||||
|
public class DataFramePivotRestIT extends DataFrameRestTestCase {
|
||||||
|
|
||||||
|
private static boolean indicesCreated = false;
|
||||||
|
|
||||||
|
// preserve indices in order to reuse source indices in several test cases
|
||||||
|
@Override
|
||||||
|
protected boolean preserveIndicesUponCompletion() {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void createIndexes() throws IOException {
|
||||||
|
|
||||||
|
// it's not possible to run it as @BeforeClass as clients aren't initialized then, so we need this little hack
|
||||||
|
if (indicesCreated) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
createReviewsIndex();
|
||||||
|
indicesCreated = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testSimplePivot() throws Exception {
|
||||||
|
String transformId = "simplePivot";
|
||||||
|
String dataFrameIndex = "pivot_reviews";
|
||||||
|
|
||||||
|
createPivotReviewsTransform(transformId, dataFrameIndex, null);
|
||||||
|
|
||||||
|
startAndWaitForTransform(transformId, dataFrameIndex);
|
||||||
|
|
||||||
|
// we expect 27 documents as there shall be 27 user_id's
|
||||||
|
Map<String, Object> indexStats = getAsMap(dataFrameIndex + "/_stats");
|
||||||
|
assertEquals(27, XContentMapValues.extractValue("_all.total.docs.count", indexStats));
|
||||||
|
|
||||||
|
// get and check some users
|
||||||
|
assertOnePivotValue(dataFrameIndex + "/_search?q=reviewer:user_0", 3.776978417);
|
||||||
|
assertOnePivotValue(dataFrameIndex + "/_search?q=reviewer:user_5", 3.72);
|
||||||
|
assertOnePivotValue(dataFrameIndex + "/_search?q=reviewer:user_11", 3.846153846);
|
||||||
|
assertOnePivotValue(dataFrameIndex + "/_search?q=reviewer:user_20", 3.769230769);
|
||||||
|
assertOnePivotValue(dataFrameIndex + "/_search?q=reviewer:user_26", 3.918918918);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testSimplePivotWithQuery() throws Exception {
|
||||||
|
String transformId = "simplePivotWithQuery";
|
||||||
|
String dataFrameIndex = "pivot_reviews_user_id_above_20";
|
||||||
|
String query = "\"match\": {\"user_id\": \"user_26\"}";
|
||||||
|
|
||||||
|
createPivotReviewsTransform(transformId, dataFrameIndex, query);
|
||||||
|
|
||||||
|
startAndWaitForTransform(transformId, dataFrameIndex);
|
||||||
|
|
||||||
|
// we expect only 1 document due to the query
|
||||||
|
Map<String, Object> indexStats = getAsMap(dataFrameIndex + "/_stats");
|
||||||
|
assertEquals(1, XContentMapValues.extractValue("_all.total.docs.count", indexStats));
|
||||||
|
assertOnePivotValue(dataFrameIndex + "/_search?q=reviewer:user_26", 3.918918918);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testHistogramPivot() throws Exception {
|
||||||
|
String transformId = "simpleHistogramPivot";
|
||||||
|
String dataFrameIndex = "pivot_reviews_via_histogram";
|
||||||
|
|
||||||
|
final Request createDataframeTransformRequest = new Request("PUT", DATAFRAME_ENDPOINT + transformId);
|
||||||
|
|
||||||
|
String config = "{"
|
||||||
|
+ " \"source\": \"reviews\","
|
||||||
|
+ " \"dest\": \"" + dataFrameIndex + "\",";
|
||||||
|
|
||||||
|
|
||||||
|
config += " \"pivot\": {"
|
||||||
|
+ " \"group_by\": {"
|
||||||
|
+ " \"every_2\": {"
|
||||||
|
+ " \"histogram\": {"
|
||||||
|
+ " \"interval\": 2,\"field\":\"stars\""
|
||||||
|
+ " } } },"
|
||||||
|
+ " \"aggregations\": {"
|
||||||
|
+ " \"avg_rating\": {"
|
||||||
|
+ " \"avg\": {"
|
||||||
|
+ " \"field\": \"stars\""
|
||||||
|
+ " } } } }"
|
||||||
|
+ "}";
|
||||||
|
|
||||||
|
|
||||||
|
createDataframeTransformRequest.setJsonEntity(config);
|
||||||
|
Map<String, Object> createDataframeTransformResponse = entityAsMap(client().performRequest(createDataframeTransformRequest));
|
||||||
|
assertThat(createDataframeTransformResponse.get("acknowledged"), equalTo(Boolean.TRUE));
|
||||||
|
assertTrue(indexExists(dataFrameIndex));
|
||||||
|
|
||||||
|
startAndWaitForTransform(transformId, dataFrameIndex);
|
||||||
|
|
||||||
|
// we expect 3 documents as there shall be 5 unique star values and we are bucketing every 2 starting at 0
|
||||||
|
Map<String, Object> indexStats = getAsMap(dataFrameIndex + "/_stats");
|
||||||
|
assertEquals(3, XContentMapValues.extractValue("_all.total.docs.count", indexStats));
|
||||||
|
assertOnePivotValue(dataFrameIndex + "/_search?q=every_2:0.0", 1.0);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testBiggerPivot() throws Exception {
|
||||||
|
String transformId = "biggerPivot";
|
||||||
|
String dataFrameIndex = "bigger_pivot_reviews";
|
||||||
|
|
||||||
|
final Request createDataframeTransformRequest = new Request("PUT", DATAFRAME_ENDPOINT + transformId);
|
||||||
|
|
||||||
|
String config = "{"
|
||||||
|
+ " \"source\": \"reviews\","
|
||||||
|
+ " \"dest\": \"" + dataFrameIndex + "\",";
|
||||||
|
|
||||||
|
|
||||||
|
config += " \"pivot\": {"
|
||||||
|
+ " \"group_by\": {"
|
||||||
|
+ " \"reviewer\": {"
|
||||||
|
+ " \"terms\": {"
|
||||||
|
+ " \"field\": \"user_id\""
|
||||||
|
+ " } } },"
|
||||||
|
+ " \"aggregations\": {"
|
||||||
|
+ " \"avg_rating\": {"
|
||||||
|
+ " \"avg\": {"
|
||||||
|
+ " \"field\": \"stars\""
|
||||||
|
+ " } },"
|
||||||
|
+ " \"sum_rating\": {"
|
||||||
|
+ " \"sum\": {"
|
||||||
|
+ " \"field\": \"stars\""
|
||||||
|
+ " } },"
|
||||||
|
+ " \"cardinality_business\": {"
|
||||||
|
+ " \"cardinality\": {"
|
||||||
|
+ " \"field\": \"business_id\""
|
||||||
|
+ " } },"
|
||||||
|
+ " \"min_rating\": {"
|
||||||
|
+ " \"min\": {"
|
||||||
|
+ " \"field\": \"stars\""
|
||||||
|
+ " } },"
|
||||||
|
+ " \"max_rating\": {"
|
||||||
|
+ " \"max\": {"
|
||||||
|
+ " \"field\": \"stars\""
|
||||||
|
+ " } },"
|
||||||
|
+ " \"count\": {"
|
||||||
|
+ " \"value_count\": {"
|
||||||
|
+ " \"field\": \"business_id\""
|
||||||
|
+ " } }"
|
||||||
|
+ " } }"
|
||||||
|
+ "}";
|
||||||
|
|
||||||
|
createDataframeTransformRequest.setJsonEntity(config);
|
||||||
|
Map<String, Object> createDataframeTransformResponse = entityAsMap(client().performRequest(createDataframeTransformRequest));
|
||||||
|
assertThat(createDataframeTransformResponse.get("acknowledged"), equalTo(Boolean.TRUE));
|
||||||
|
assertTrue(indexExists(dataFrameIndex));
|
||||||
|
|
||||||
|
startAndWaitForTransform(transformId, dataFrameIndex);
|
||||||
|
|
||||||
|
// we expect 27 documents as there shall be 27 user_id's
|
||||||
|
Map<String, Object> indexStats = getAsMap(dataFrameIndex + "/_stats");
|
||||||
|
assertEquals(27, XContentMapValues.extractValue("_all.total.docs.count", indexStats));
|
||||||
|
|
||||||
|
// get and check some users
|
||||||
|
Map<String, Object> searchResult = getAsMap(dataFrameIndex + "/_search?q=reviewer:user_4");
|
||||||
|
|
||||||
|
assertEquals(1, XContentMapValues.extractValue("hits.total.value", searchResult));
|
||||||
|
Number actual = (Number) ((List<?>) XContentMapValues.extractValue("hits.hits._source.avg_rating", searchResult)).get(0);
|
||||||
|
assertEquals(3.878048780, actual.doubleValue(), 0.000001);
|
||||||
|
actual = (Number) ((List<?>) XContentMapValues.extractValue("hits.hits._source.sum_rating", searchResult)).get(0);
|
||||||
|
assertEquals(159, actual.longValue());
|
||||||
|
actual = (Number) ((List<?>) XContentMapValues.extractValue("hits.hits._source.cardinality_business", searchResult)).get(0);
|
||||||
|
assertEquals(6, actual.longValue());
|
||||||
|
actual = (Number) ((List<?>) XContentMapValues.extractValue("hits.hits._source.min_rating", searchResult)).get(0);
|
||||||
|
assertEquals(1, actual.longValue());
|
||||||
|
actual = (Number) ((List<?>) XContentMapValues.extractValue("hits.hits._source.max_rating", searchResult)).get(0);
|
||||||
|
assertEquals(5, actual.longValue());
|
||||||
|
actual = (Number) ((List<?>) XContentMapValues.extractValue("hits.hits._source.count", searchResult)).get(0);
|
||||||
|
assertEquals(41, actual.longValue());
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testDateHistogramPivot() throws Exception {
|
||||||
|
String transformId = "simpleDateHistogramPivot";
|
||||||
|
String dataFrameIndex = "pivot_reviews_via_date_histogram";
|
||||||
|
|
||||||
|
final Request createDataframeTransformRequest = new Request("PUT", DATAFRAME_ENDPOINT + transformId);
|
||||||
|
|
||||||
|
String config = "{"
|
||||||
|
+ " \"source\": \"reviews\","
|
||||||
|
+ " \"dest\": \"" + dataFrameIndex + "\",";
|
||||||
|
|
||||||
|
|
||||||
|
config += " \"pivot\": {"
|
||||||
|
+ " \"group_by\": {"
|
||||||
|
+ " \"by_day\": {"
|
||||||
|
+ " \"date_histogram\": {"
|
||||||
|
+ " \"interval\": \"1d\",\"field\":\"timestamp\",\"format\":\"yyyy-MM-DD\""
|
||||||
|
+ " } } },"
|
||||||
|
+ " \"aggregations\": {"
|
||||||
|
+ " \"avg_rating\": {"
|
||||||
|
+ " \"avg\": {"
|
||||||
|
+ " \"field\": \"stars\""
|
||||||
|
+ " } } } }"
|
||||||
|
+ "}";
|
||||||
|
|
||||||
|
createDataframeTransformRequest.setJsonEntity(config);
|
||||||
|
Map<String, Object> createDataframeTransformResponse = entityAsMap(client().performRequest(createDataframeTransformRequest));
|
||||||
|
assertThat(createDataframeTransformResponse.get("acknowledged"), equalTo(Boolean.TRUE));
|
||||||
|
assertTrue(indexExists(dataFrameIndex));
|
||||||
|
|
||||||
|
startAndWaitForTransform(transformId, dataFrameIndex);
|
||||||
|
|
||||||
|
// we expect 21 documents as there shall be 21 days worth of docs
|
||||||
|
Map<String, Object> indexStats = getAsMap(dataFrameIndex + "/_stats");
|
||||||
|
assertEquals(21, XContentMapValues.extractValue("_all.total.docs.count", indexStats));
|
||||||
|
assertOnePivotValue(dataFrameIndex + "/_search?q=by_day:2017-01-15", 3.82);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void startAndWaitForTransform(String transformId, String dataFrameIndex) throws IOException, Exception {
|
||||||
|
// start the transform
|
||||||
|
final Request startTransformRequest = new Request("POST", DATAFRAME_ENDPOINT + transformId + "/_start");
|
||||||
|
Map<String, Object> startTransformResponse = entityAsMap(client().performRequest(startTransformRequest));
|
||||||
|
assertThat(startTransformResponse.get("started"), equalTo(Boolean.TRUE));
|
||||||
|
|
||||||
|
// wait until the dataframe has been created and all data is available
|
||||||
|
waitForDataFrameGeneration(transformId);
|
||||||
|
refreshIndex(dataFrameIndex);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void waitForDataFrameGeneration(String transformId) throws Exception {
|
||||||
|
assertBusy(() -> {
|
||||||
|
long generation = getDataFrameGeneration(transformId);
|
||||||
|
assertEquals(1, generation);
|
||||||
|
}, 30, TimeUnit.SECONDS);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static int getDataFrameGeneration(String transformId) throws IOException {
|
||||||
|
Response statsResponse = client().performRequest(new Request("GET", DATAFRAME_ENDPOINT + transformId + "/_stats"));
|
||||||
|
|
||||||
|
Map<?, ?> transformStatsAsMap = (Map<?, ?>) ((List<?>) entityAsMap(statsResponse).get("transforms")).get(0);
|
||||||
|
return (int) XContentMapValues.extractValue("state.generation", transformStatsAsMap);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void refreshIndex(String index) throws IOException {
|
||||||
|
assertOK(client().performRequest(new Request("POST", index + "/_refresh")));
|
||||||
|
}
|
||||||
|
|
||||||
|
private void assertOnePivotValue(String query, double expected) throws IOException {
|
||||||
|
Map<String, Object> searchResult = getAsMap(query);
|
||||||
|
|
||||||
|
assertEquals(1, XContentMapValues.extractValue("hits.total.value", searchResult));
|
||||||
|
double actual = (double) ((List<?>) XContentMapValues.extractValue("hits.hits._source.avg_rating", searchResult)).get(0);
|
||||||
|
assertEquals(expected, actual, 0.000001);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,224 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.integration;
|
||||||
|
|
||||||
|
import org.apache.http.entity.ContentType;
|
||||||
|
import org.apache.http.entity.StringEntity;
|
||||||
|
import org.elasticsearch.client.Request;
|
||||||
|
import org.elasticsearch.client.Response;
|
||||||
|
import org.elasticsearch.client.ResponseException;
|
||||||
|
import org.elasticsearch.common.Strings;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||||
|
import org.elasticsearch.test.rest.ESRestTestCase;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.dataframe.persistence.DataFrameInternalIndex;
|
||||||
|
import org.junit.AfterClass;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||||
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
|
||||||
|
public abstract class DataFrameRestTestCase extends ESRestTestCase {
|
||||||
|
|
||||||
|
protected static final String DATAFRAME_ENDPOINT = DataFrameField.REST_BASE_PATH + "transforms/";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a simple dataset for testing with reviewers, ratings and businesses
|
||||||
|
*/
|
||||||
|
protected void createReviewsIndex() throws IOException {
|
||||||
|
int[] distributionTable = {5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 4, 4, 4, 3, 3, 2, 1, 1, 1};
|
||||||
|
|
||||||
|
final int numDocs = 1000;
|
||||||
|
|
||||||
|
// create mapping
|
||||||
|
try (XContentBuilder builder = jsonBuilder()) {
|
||||||
|
builder.startObject();
|
||||||
|
{
|
||||||
|
builder.startObject("mappings")
|
||||||
|
.startObject("properties")
|
||||||
|
.startObject("timestamp")
|
||||||
|
.field("type", "date")
|
||||||
|
.endObject()
|
||||||
|
.startObject("user_id")
|
||||||
|
.field("type", "keyword")
|
||||||
|
.endObject()
|
||||||
|
.startObject("business_id")
|
||||||
|
.field("type", "keyword")
|
||||||
|
.endObject()
|
||||||
|
.startObject("stars")
|
||||||
|
.field("type", "integer")
|
||||||
|
.endObject()
|
||||||
|
.endObject()
|
||||||
|
.endObject();
|
||||||
|
}
|
||||||
|
builder.endObject();
|
||||||
|
final StringEntity entity = new StringEntity(Strings.toString(builder), ContentType.APPLICATION_JSON);
|
||||||
|
Request req = new Request("PUT", "reviews");
|
||||||
|
req.setEntity(entity);
|
||||||
|
client().performRequest(req);
|
||||||
|
}
|
||||||
|
|
||||||
|
// create index
|
||||||
|
final StringBuilder bulk = new StringBuilder();
|
||||||
|
int day = 10;
|
||||||
|
for (int i = 0; i < numDocs; i++) {
|
||||||
|
bulk.append("{\"index\":{\"_index\":\"reviews\"}}\n");
|
||||||
|
long user = Math.round(Math.pow(i * 31 % 1000, distributionTable[i % distributionTable.length]) % 27);
|
||||||
|
int stars = distributionTable[(i * 33) % distributionTable.length];
|
||||||
|
long business = Math.round(Math.pow(user * stars, distributionTable[i % distributionTable.length]) % 13);
|
||||||
|
int hour = randomIntBetween(10, 20);
|
||||||
|
int min = randomIntBetween(30, 59);
|
||||||
|
int sec = randomIntBetween(30, 59);
|
||||||
|
|
||||||
|
String date_string = "2017-01-" + day + "T" + hour + ":" + min + ":" + sec + "Z";
|
||||||
|
bulk.append("{\"user_id\":\"")
|
||||||
|
.append("user_")
|
||||||
|
.append(user)
|
||||||
|
.append("\",\"business_id\":\"")
|
||||||
|
.append("business_")
|
||||||
|
.append(business)
|
||||||
|
.append("\",\"stars\":")
|
||||||
|
.append(stars)
|
||||||
|
.append(",\"timestamp\":\"")
|
||||||
|
.append(date_string)
|
||||||
|
.append("\"}\n");
|
||||||
|
|
||||||
|
if (i % 50 == 0) {
|
||||||
|
bulk.append("\r\n");
|
||||||
|
final Request bulkRequest = new Request("POST", "/_bulk");
|
||||||
|
bulkRequest.addParameter("refresh", "true");
|
||||||
|
bulkRequest.setJsonEntity(bulk.toString());
|
||||||
|
client().performRequest(bulkRequest);
|
||||||
|
// clear the builder
|
||||||
|
bulk.setLength(0);
|
||||||
|
day += 1;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
bulk.append("\r\n");
|
||||||
|
|
||||||
|
final Request bulkRequest = new Request("POST", "/_bulk");
|
||||||
|
bulkRequest.addParameter("refresh", "true");
|
||||||
|
bulkRequest.setJsonEntity(bulk.toString());
|
||||||
|
client().performRequest(bulkRequest);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void createPivotReviewsTransform(String transformId, String dataFrameIndex, String query) throws IOException {
|
||||||
|
final Request createDataframeTransformRequest = new Request("PUT", DATAFRAME_ENDPOINT + transformId);
|
||||||
|
|
||||||
|
String config = "{"
|
||||||
|
+ " \"source\": \"reviews\","
|
||||||
|
+ " \"dest\": \"" + dataFrameIndex + "\",";
|
||||||
|
|
||||||
|
if (query != null) {
|
||||||
|
config += "\"query\": {"
|
||||||
|
+ query
|
||||||
|
+ "},";
|
||||||
|
}
|
||||||
|
|
||||||
|
config += " \"pivot\": {"
|
||||||
|
+ " \"group_by\": {"
|
||||||
|
+ " \"reviewer\": {"
|
||||||
|
+ " \"terms\": {"
|
||||||
|
+ " \"field\": \"user_id\""
|
||||||
|
+ " } } },"
|
||||||
|
+ " \"aggregations\": {"
|
||||||
|
+ " \"avg_rating\": {"
|
||||||
|
+ " \"avg\": {"
|
||||||
|
+ " \"field\": \"stars\""
|
||||||
|
+ " } } } }"
|
||||||
|
+ "}";
|
||||||
|
|
||||||
|
createDataframeTransformRequest.setJsonEntity(config);
|
||||||
|
Map<String, Object> createDataframeTransformResponse = entityAsMap(client().performRequest(createDataframeTransformRequest));
|
||||||
|
assertThat(createDataframeTransformResponse.get("acknowledged"), equalTo(Boolean.TRUE));
|
||||||
|
assertTrue(indexExists(dataFrameIndex));
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
private static List<Map<String, Object>> getDataFrameTransforms() throws IOException {
|
||||||
|
Response response = adminClient().performRequest(new Request("GET", DATAFRAME_ENDPOINT + "_all"));
|
||||||
|
Map<String, Object> transforms = entityAsMap(response);
|
||||||
|
List<Map<String, Object>> transformConfigs = (List<Map<String, Object>>) XContentMapValues.extractValue("transforms", transforms);
|
||||||
|
|
||||||
|
return transformConfigs == null ? Collections.emptyList() : transformConfigs;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static String getDataFrameIndexerState(String transformId) throws IOException {
|
||||||
|
Response statsResponse = client().performRequest(new Request("GET", DATAFRAME_ENDPOINT + transformId + "/_stats"));
|
||||||
|
|
||||||
|
Map<?, ?> transformStatsAsMap = (Map<?, ?>) ((List<?>) entityAsMap(statsResponse).get("transforms")).get(0);
|
||||||
|
return (String) XContentMapValues.extractValue("state.transform_state", transformStatsAsMap);
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterClass
|
||||||
|
public static void removeIndices() throws Exception {
|
||||||
|
wipeDataFrameTransforms();
|
||||||
|
waitForPendingDataFrameTasks();
|
||||||
|
// we might have disabled wiping indices, but now its time to get rid of them
|
||||||
|
// note: can not use super.cleanUpCluster() as this method must be static
|
||||||
|
wipeIndices();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static void wipeDataFrameTransforms() throws IOException, InterruptedException {
|
||||||
|
List<Map<String, Object>> transformConfigs = getDataFrameTransforms();
|
||||||
|
|
||||||
|
for (Map<String, Object> transformConfig : transformConfigs) {
|
||||||
|
String transformId = (String) transformConfig.get("id");
|
||||||
|
Request request = new Request("POST", DATAFRAME_ENDPOINT + transformId + "/_stop");
|
||||||
|
request.addParameter("wait_for_completion", "true");
|
||||||
|
request.addParameter("timeout", "10s");
|
||||||
|
request.addParameter("ignore", "404");
|
||||||
|
adminClient().performRequest(request);
|
||||||
|
assertEquals("stopped", getDataFrameIndexerState(transformId));
|
||||||
|
}
|
||||||
|
|
||||||
|
for (Map<String, Object> transformConfig : transformConfigs) {
|
||||||
|
String transformId = (String) transformConfig.get("id");
|
||||||
|
Request request = new Request("DELETE", DATAFRAME_ENDPOINT + transformId);
|
||||||
|
request.addParameter("ignore", "404"); // Ignore 404s because they imply someone was racing us to delete this
|
||||||
|
adminClient().performRequest(request);
|
||||||
|
}
|
||||||
|
|
||||||
|
// transforms should be all gone
|
||||||
|
transformConfigs = getDataFrameTransforms();
|
||||||
|
assertTrue(transformConfigs.isEmpty());
|
||||||
|
|
||||||
|
// the configuration index should be empty
|
||||||
|
Request request = new Request("GET", DataFrameInternalIndex.INDEX_NAME + "/_search");
|
||||||
|
try {
|
||||||
|
Response searchResponse = adminClient().performRequest(request);
|
||||||
|
Map<String, Object> searchResult = entityAsMap(searchResponse);
|
||||||
|
|
||||||
|
assertEquals(0, XContentMapValues.extractValue("hits.total.value", searchResult));
|
||||||
|
} catch (ResponseException e) {
|
||||||
|
// 404 here just means we had no data frame transforms, true for some tests
|
||||||
|
if (e.getResponse().getStatusLine().getStatusCode() != 404) {
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static void waitForPendingDataFrameTasks() throws Exception {
|
||||||
|
waitForPendingTasks(adminClient(), taskName -> taskName.startsWith(DataFrameField.TASK_NAME) == false);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static void wipeIndices() throws IOException {
|
||||||
|
try {
|
||||||
|
adminClient().performRequest(new Request("DELETE", "*"));
|
||||||
|
} catch (ResponseException e) {
|
||||||
|
// 404 here just means we had no indexes
|
||||||
|
if (e.getResponse().getStatusLine().getStatusCode() != 404) {
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,58 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.integration;
|
||||||
|
|
||||||
|
import org.elasticsearch.client.Request;
|
||||||
|
import org.elasticsearch.client.Response;
|
||||||
|
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||||
|
import org.junit.Before;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class DataFrameUsageIT extends DataFrameRestTestCase {
|
||||||
|
private boolean indicesCreated = false;
|
||||||
|
|
||||||
|
// preserve indices in order to reuse source indices in several test cases
|
||||||
|
@Override
|
||||||
|
protected boolean preserveIndicesUponCompletion() {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void createIndexes() throws IOException {
|
||||||
|
|
||||||
|
// it's not possible to run it as @BeforeClass as clients aren't initialized then, so we need this little hack
|
||||||
|
if (indicesCreated) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
createReviewsIndex();
|
||||||
|
indicesCreated = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testUsage() throws IOException {
|
||||||
|
Response usageResponse = client().performRequest(new Request("GET", "_xpack/usage"));
|
||||||
|
|
||||||
|
Map<?, ?> usageAsMap = entityAsMap(usageResponse);
|
||||||
|
assertTrue((boolean) XContentMapValues.extractValue("data_frame.available", usageAsMap));
|
||||||
|
assertTrue((boolean) XContentMapValues.extractValue("data_frame.enabled", usageAsMap));
|
||||||
|
// no transforms, no stats
|
||||||
|
assertEquals(null, XContentMapValues.extractValue("data_frame.transforms", usageAsMap));
|
||||||
|
assertEquals(null, XContentMapValues.extractValue("data_frame.stats", usageAsMap));
|
||||||
|
|
||||||
|
// create a transform
|
||||||
|
createPivotReviewsTransform("test_usage", "pivot_reviews", null);
|
||||||
|
|
||||||
|
usageResponse = client().performRequest(new Request("GET", "_xpack/usage"));
|
||||||
|
|
||||||
|
usageAsMap = entityAsMap(usageResponse);
|
||||||
|
// we should see some stats
|
||||||
|
assertEquals(1, XContentMapValues.extractValue("data_frame.transforms._all", usageAsMap));
|
||||||
|
assertEquals(0, XContentMapValues.extractValue("data_frame.stats.index_failures", usageAsMap));
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,226 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.apache.lucene.util.SetOnce;
|
||||||
|
import org.elasticsearch.action.ActionRequest;
|
||||||
|
import org.elasticsearch.action.ActionResponse;
|
||||||
|
import org.elasticsearch.client.Client;
|
||||||
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
|
import org.elasticsearch.cluster.metadata.IndexTemplateMetaData;
|
||||||
|
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||||
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
|
import org.elasticsearch.common.ParseField;
|
||||||
|
import org.elasticsearch.common.inject.Module;
|
||||||
|
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||||
|
import org.elasticsearch.common.settings.ClusterSettings;
|
||||||
|
import org.elasticsearch.common.settings.IndexScopedSettings;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.common.settings.SettingsFilter;
|
||||||
|
import org.elasticsearch.common.settings.SettingsModule;
|
||||||
|
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||||
|
import org.elasticsearch.env.Environment;
|
||||||
|
import org.elasticsearch.env.NodeEnvironment;
|
||||||
|
import org.elasticsearch.license.XPackLicenseState;
|
||||||
|
import org.elasticsearch.persistent.PersistentTaskParams;
|
||||||
|
import org.elasticsearch.persistent.PersistentTaskState;
|
||||||
|
import org.elasticsearch.persistent.PersistentTasksExecutor;
|
||||||
|
import org.elasticsearch.plugins.ActionPlugin;
|
||||||
|
import org.elasticsearch.plugins.PersistentTaskPlugin;
|
||||||
|
import org.elasticsearch.plugins.Plugin;
|
||||||
|
import org.elasticsearch.rest.RestController;
|
||||||
|
import org.elasticsearch.rest.RestHandler;
|
||||||
|
import org.elasticsearch.script.ScriptService;
|
||||||
|
import org.elasticsearch.tasks.Task;
|
||||||
|
import org.elasticsearch.threadpool.ExecutorBuilder;
|
||||||
|
import org.elasticsearch.threadpool.FixedExecutorBuilder;
|
||||||
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
import org.elasticsearch.watcher.ResourceWatcherService;
|
||||||
|
import org.elasticsearch.xpack.core.XPackPlugin;
|
||||||
|
import org.elasticsearch.xpack.core.XPackSettings;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameTransformState;
|
||||||
|
import org.elasticsearch.xpack.core.scheduler.SchedulerEngine;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.DeleteDataFrameTransformAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.GetDataFrameTransformsAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.GetDataFrameTransformsStatsAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.PutDataFrameTransformAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.StartDataFrameTransformAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.StopDataFrameTransformAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.TransportDeleteDataFrameTransformAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.TransportGetDataFrameTransformsAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.TransportGetDataFrameTransformsStatsAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.TransportPutDataFrameTransformAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.TransportStartDataFrameTransformAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.TransportStopDataFrameTransformAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.persistence.DataFrameInternalIndex;
|
||||||
|
import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager;
|
||||||
|
import org.elasticsearch.xpack.dataframe.rest.action.RestDeleteDataFrameTransformAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.rest.action.RestGetDataFrameTransformsAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.rest.action.RestGetDataFrameTransformsStatsAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.rest.action.RestPutDataFrameTransformAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.rest.action.RestStartDataFrameTransformAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.rest.action.RestStopDataFrameTransformAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransform;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformPersistentTasksExecutor;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.time.Clock;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.function.Supplier;
|
||||||
|
import java.util.function.UnaryOperator;
|
||||||
|
|
||||||
|
import static java.util.Collections.emptyList;
|
||||||
|
|
||||||
|
public class DataFrame extends Plugin implements ActionPlugin, PersistentTaskPlugin {
|
||||||
|
|
||||||
|
public static final String NAME = "data_frame";
|
||||||
|
public static final String TASK_THREAD_POOL_NAME = "data_frame_indexing";
|
||||||
|
|
||||||
|
// list of headers that will be stored when a transform is created
|
||||||
|
public static final Set<String> HEADER_FILTERS = new HashSet<>(
|
||||||
|
Arrays.asList("es-security-runas-user", "_xpack_security_authentication"));
|
||||||
|
|
||||||
|
private static final Logger logger = LogManager.getLogger(XPackPlugin.class);
|
||||||
|
|
||||||
|
private final boolean enabled;
|
||||||
|
private final Settings settings;
|
||||||
|
private final boolean transportClientMode;
|
||||||
|
private final SetOnce<DataFrameTransformsConfigManager> dataFrameTransformsConfigManager = new SetOnce<>();
|
||||||
|
|
||||||
|
public DataFrame(Settings settings) {
|
||||||
|
this.settings = settings;
|
||||||
|
|
||||||
|
this.enabled = XPackSettings.DATA_FRAME_ENABLED.get(settings);
|
||||||
|
this.transportClientMode = XPackPlugin.transportClientMode(settings);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Collection<Module> createGuiceModules() {
|
||||||
|
List<Module> modules = new ArrayList<>();
|
||||||
|
|
||||||
|
if (transportClientMode) {
|
||||||
|
return modules;
|
||||||
|
}
|
||||||
|
|
||||||
|
modules.add(b -> XPackPlugin.bindFeatureSet(b, DataFrameFeatureSet.class));
|
||||||
|
return modules;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected XPackLicenseState getLicenseState() { return XPackPlugin.getSharedLicenseState(); }
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<RestHandler> getRestHandlers(final Settings settings, final RestController restController,
|
||||||
|
final ClusterSettings clusterSettings, final IndexScopedSettings indexScopedSettings, final SettingsFilter settingsFilter,
|
||||||
|
final IndexNameExpressionResolver indexNameExpressionResolver, final Supplier<DiscoveryNodes> nodesInCluster) {
|
||||||
|
|
||||||
|
if (!enabled) {
|
||||||
|
return emptyList();
|
||||||
|
}
|
||||||
|
|
||||||
|
return Arrays.asList(
|
||||||
|
new RestPutDataFrameTransformAction(settings, restController),
|
||||||
|
new RestStartDataFrameTransformAction(settings, restController),
|
||||||
|
new RestStopDataFrameTransformAction(settings, restController),
|
||||||
|
new RestDeleteDataFrameTransformAction(settings, restController),
|
||||||
|
new RestGetDataFrameTransformsAction(settings, restController),
|
||||||
|
new RestGetDataFrameTransformsStatsAction(settings, restController)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<ActionHandler<? extends ActionRequest, ? extends ActionResponse>> getActions() {
|
||||||
|
if (!enabled) {
|
||||||
|
return emptyList();
|
||||||
|
}
|
||||||
|
|
||||||
|
return Arrays.asList(
|
||||||
|
new ActionHandler<>(PutDataFrameTransformAction.INSTANCE, TransportPutDataFrameTransformAction.class),
|
||||||
|
new ActionHandler<>(StartDataFrameTransformAction.INSTANCE, TransportStartDataFrameTransformAction.class),
|
||||||
|
new ActionHandler<>(StopDataFrameTransformAction.INSTANCE, TransportStopDataFrameTransformAction.class),
|
||||||
|
new ActionHandler<>(DeleteDataFrameTransformAction.INSTANCE, TransportDeleteDataFrameTransformAction.class),
|
||||||
|
new ActionHandler<>(GetDataFrameTransformsAction.INSTANCE, TransportGetDataFrameTransformsAction.class),
|
||||||
|
new ActionHandler<>(GetDataFrameTransformsStatsAction.INSTANCE, TransportGetDataFrameTransformsStatsAction.class)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<ExecutorBuilder<?>> getExecutorBuilders(Settings settings) {
|
||||||
|
if (false == enabled || transportClientMode) {
|
||||||
|
return emptyList();
|
||||||
|
}
|
||||||
|
|
||||||
|
FixedExecutorBuilder indexing = new FixedExecutorBuilder(settings, TASK_THREAD_POOL_NAME, 4, 4,
|
||||||
|
"data_frame.task_thread_pool");
|
||||||
|
|
||||||
|
return Collections.singletonList(indexing);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Collection<Object> createComponents(Client client, ClusterService clusterService, ThreadPool threadPool,
|
||||||
|
ResourceWatcherService resourceWatcherService, ScriptService scriptService, NamedXContentRegistry xContentRegistry,
|
||||||
|
Environment environment, NodeEnvironment nodeEnvironment, NamedWriteableRegistry namedWriteableRegistry) {
|
||||||
|
if (enabled == false || transportClientMode) {
|
||||||
|
return emptyList();
|
||||||
|
}
|
||||||
|
|
||||||
|
dataFrameTransformsConfigManager.set(new DataFrameTransformsConfigManager(client, xContentRegistry));
|
||||||
|
|
||||||
|
return Collections.singletonList(dataFrameTransformsConfigManager.get());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public UnaryOperator<Map<String, IndexTemplateMetaData>> getIndexTemplateMetaDataUpgrader() {
|
||||||
|
return templates -> {
|
||||||
|
try {
|
||||||
|
templates.put(DataFrameInternalIndex.INDEX_TEMPLATE_NAME, DataFrameInternalIndex.getIndexTemplateMetaData());
|
||||||
|
} catch (IOException e) {
|
||||||
|
logger.error("Error creating data frame index template", e);
|
||||||
|
}
|
||||||
|
return templates;
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<PersistentTasksExecutor<?>> getPersistentTasksExecutor(ClusterService clusterService, ThreadPool threadPool,
|
||||||
|
Client client, SettingsModule settingsModule) {
|
||||||
|
if (enabled == false || transportClientMode) {
|
||||||
|
return emptyList();
|
||||||
|
}
|
||||||
|
|
||||||
|
SchedulerEngine schedulerEngine = new SchedulerEngine(settings, Clock.systemUTC());
|
||||||
|
|
||||||
|
// the transforms config manager should have been created
|
||||||
|
assert dataFrameTransformsConfigManager.get() != null;
|
||||||
|
return Collections.singletonList(
|
||||||
|
new DataFrameTransformPersistentTasksExecutor(client, dataFrameTransformsConfigManager.get(), schedulerEngine, threadPool));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<NamedXContentRegistry.Entry> getNamedXContent() {
|
||||||
|
if (enabled == false) {
|
||||||
|
return emptyList();
|
||||||
|
}
|
||||||
|
return Arrays.asList(
|
||||||
|
new NamedXContentRegistry.Entry(PersistentTaskParams.class, new ParseField(DataFrameField.TASK_NAME),
|
||||||
|
DataFrameTransform::fromXContent),
|
||||||
|
new NamedXContentRegistry.Entry(Task.Status.class, new ParseField(DataFrameTransformState.NAME),
|
||||||
|
DataFrameTransformState::fromXContent),
|
||||||
|
new NamedXContentRegistry.Entry(PersistentTaskState.class, new ParseField(DataFrameTransformState.NAME),
|
||||||
|
DataFrameTransformState::fromXContent)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,88 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.client.Client;
|
||||||
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
|
import org.elasticsearch.common.Nullable;
|
||||||
|
import org.elasticsearch.common.inject.Inject;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.license.XPackLicenseState;
|
||||||
|
import org.elasticsearch.xpack.core.XPackFeatureSet;
|
||||||
|
import org.elasticsearch.xpack.core.XPackField;
|
||||||
|
import org.elasticsearch.xpack.core.XPackSettings;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameFeatureSetUsage;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameIndexerTransformStats;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.GetDataFrameTransformsStatsAction;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
public class DataFrameFeatureSet implements XPackFeatureSet {
|
||||||
|
|
||||||
|
private final boolean enabled;
|
||||||
|
private final Client client;
|
||||||
|
private final XPackLicenseState licenseState;
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public DataFrameFeatureSet(Settings settings, Client client, @Nullable XPackLicenseState licenseState) {
|
||||||
|
this.enabled = XPackSettings.DATA_FRAME_ENABLED.get(settings);
|
||||||
|
this.client = Objects.requireNonNull(client);
|
||||||
|
this.licenseState = licenseState;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String name() {
|
||||||
|
return XPackField.DATA_FRAME;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String description() {
|
||||||
|
return "Data Frame for the Elastic Stack";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean available() {
|
||||||
|
return licenseState != null && licenseState.isDataFrameAllowed();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean enabled() {
|
||||||
|
return enabled;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, Object> nativeCodeInfo() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void usage(ActionListener<XPackFeatureSet.Usage> listener) {
|
||||||
|
if (enabled == false) {
|
||||||
|
listener.onResponse(
|
||||||
|
new DataFrameFeatureSetUsage(available(), enabled(), Collections.emptyMap(), new DataFrameIndexerTransformStats()));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
GetDataFrameTransformsStatsAction.Request transformStatsRequest = new GetDataFrameTransformsStatsAction.Request(MetaData.ALL);
|
||||||
|
|
||||||
|
client.execute(GetDataFrameTransformsStatsAction.INSTANCE, transformStatsRequest, ActionListener.wrap(transformStatsResponse -> {
|
||||||
|
Map<String, Long> transformsCountByState = new HashMap<>();
|
||||||
|
DataFrameIndexerTransformStats accumulatedStats = new DataFrameIndexerTransformStats();
|
||||||
|
|
||||||
|
transformStatsResponse.getTransformsStateAndStats().stream().forEach(singleResult -> {
|
||||||
|
transformsCountByState.merge(singleResult.getTransformState().getIndexerState().value(), 1L, Long::sum);
|
||||||
|
accumulatedStats.merge(singleResult.getTransformStats());
|
||||||
|
});
|
||||||
|
|
||||||
|
listener.onResponse(new DataFrameFeatureSetUsage(available(), enabled(), transformsCountByState, accumulatedStats));
|
||||||
|
}, listener::onFailure));
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,103 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.ParseField;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameIndexerTransformStats;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameTransformState;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
public class DataFrameTransformStateAndStats implements Writeable, ToXContentObject {
|
||||||
|
|
||||||
|
public static final ParseField STATE_FIELD = new ParseField("state");
|
||||||
|
|
||||||
|
private final String id;
|
||||||
|
private final DataFrameTransformState transformState;
|
||||||
|
private final DataFrameIndexerTransformStats transformStats;
|
||||||
|
|
||||||
|
public static final ConstructingObjectParser<DataFrameTransformStateAndStats, Void> PARSER = new ConstructingObjectParser<>(
|
||||||
|
GetDataFrameTransformsAction.NAME,
|
||||||
|
a -> new DataFrameTransformStateAndStats((String) a[0], (DataFrameTransformState) a[1], (DataFrameIndexerTransformStats) a[2]));
|
||||||
|
|
||||||
|
static {
|
||||||
|
PARSER.declareString(ConstructingObjectParser.constructorArg(), DataFrameField.ID);
|
||||||
|
PARSER.declareObject(ConstructingObjectParser.constructorArg(), DataFrameTransformState.PARSER::apply, STATE_FIELD);
|
||||||
|
PARSER.declareObject(ConstructingObjectParser.constructorArg(), (p, c) -> DataFrameIndexerTransformStats.fromXContent(p),
|
||||||
|
DataFrameField.STATS_FIELD);
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameTransformStateAndStats(String id, DataFrameTransformState state, DataFrameIndexerTransformStats stats) {
|
||||||
|
this.id = Objects.requireNonNull(id);
|
||||||
|
this.transformState = Objects.requireNonNull(state);
|
||||||
|
this.transformStats = Objects.requireNonNull(stats);
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameTransformStateAndStats(StreamInput in) throws IOException {
|
||||||
|
this.id = in.readString();
|
||||||
|
this.transformState = new DataFrameTransformState(in);
|
||||||
|
this.transformStats = new DataFrameIndexerTransformStats(in);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
builder.field(DataFrameField.ID.getPreferredName(), id);
|
||||||
|
builder.field(STATE_FIELD.getPreferredName(), transformState);
|
||||||
|
builder.field(DataFrameField.STATS_FIELD.getPreferredName(), transformStats);
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeString(id);
|
||||||
|
transformState.writeTo(out);
|
||||||
|
transformStats.writeTo(out);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(id, transformState, transformStats);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object other) {
|
||||||
|
if (this == other) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (other == null || getClass() != other.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
DataFrameTransformStateAndStats that = (DataFrameTransformStateAndStats) other;
|
||||||
|
|
||||||
|
return Objects.equals(this.id, that.id) && Objects.equals(this.transformState, that.transformState)
|
||||||
|
&& Objects.equals(this.transformStats, that.transformStats);
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getId() {
|
||||||
|
return id;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameIndexerTransformStats getTransformStats() {
|
||||||
|
return transformStats;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameTransformState getTransformState() {
|
||||||
|
return transformState;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,175 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.Action;
|
||||||
|
import org.elasticsearch.action.ActionRequestBuilder;
|
||||||
|
import org.elasticsearch.action.ActionRequestValidationException;
|
||||||
|
import org.elasticsearch.action.FailedNodeException;
|
||||||
|
import org.elasticsearch.action.TaskOperationFailure;
|
||||||
|
import org.elasticsearch.action.support.tasks.BaseTasksRequest;
|
||||||
|
import org.elasticsearch.action.support.tasks.BaseTasksResponse;
|
||||||
|
import org.elasticsearch.client.ElasticsearchClient;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentFragment;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.tasks.Task;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
public class DeleteDataFrameTransformAction extends Action<DeleteDataFrameTransformAction.Response> {
|
||||||
|
|
||||||
|
public static final DeleteDataFrameTransformAction INSTANCE = new DeleteDataFrameTransformAction();
|
||||||
|
public static final String NAME = "cluster:admin/data_frame/delete";
|
||||||
|
|
||||||
|
private DeleteDataFrameTransformAction() {
|
||||||
|
super(NAME);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Response newResponse() {
|
||||||
|
return new Response();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Request extends BaseTasksRequest<Request> implements ToXContentFragment {
|
||||||
|
private String id;
|
||||||
|
|
||||||
|
public Request(String id) {
|
||||||
|
this.id = ExceptionsHelper.requireNonNull(id, DataFrameField.ID.getPreferredName());
|
||||||
|
}
|
||||||
|
|
||||||
|
public Request() {
|
||||||
|
}
|
||||||
|
|
||||||
|
public Request(StreamInput in) throws IOException {
|
||||||
|
super(in);
|
||||||
|
id = in.readString();
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getId() {
|
||||||
|
return id;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean match(Task task) {
|
||||||
|
return task.getDescription().equals(DataFrameField.PERSISTENT_TASK_DESCRIPTION_PREFIX + id);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
super.writeTo(out);
|
||||||
|
out.writeString(id);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ActionRequestValidationException validate() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.field(DataFrameField.ID.getPreferredName(), id);
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(id);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object obj) {
|
||||||
|
if (this == obj) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (obj == null || getClass() != obj.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
Request other = (Request) obj;
|
||||||
|
return Objects.equals(id, other.id);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class RequestBuilder
|
||||||
|
extends ActionRequestBuilder<DeleteDataFrameTransformAction.Request, DeleteDataFrameTransformAction.Response> {
|
||||||
|
|
||||||
|
protected RequestBuilder(ElasticsearchClient client, DeleteDataFrameTransformAction action) {
|
||||||
|
super(client, action, new DeleteDataFrameTransformAction.Request());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Response extends BaseTasksResponse implements Writeable, ToXContentObject {
|
||||||
|
private boolean acknowledged;
|
||||||
|
public Response(StreamInput in) throws IOException {
|
||||||
|
super(Collections.emptyList(), Collections.emptyList());
|
||||||
|
readFrom(in);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Response(boolean acknowledged, List<TaskOperationFailure> taskFailures, List<FailedNodeException> nodeFailures) {
|
||||||
|
super(taskFailures, nodeFailures);
|
||||||
|
this.acknowledged = acknowledged;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Response(boolean acknowledged) {
|
||||||
|
this(acknowledged, Collections.emptyList(), Collections.emptyList());
|
||||||
|
}
|
||||||
|
|
||||||
|
public Response() {
|
||||||
|
this(false, Collections.emptyList(), Collections.emptyList());
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isDeleted() {
|
||||||
|
return acknowledged;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
super.readFrom(in);
|
||||||
|
acknowledged = in.readBoolean();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
super.writeTo(out);
|
||||||
|
out.writeBoolean(acknowledged);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
{
|
||||||
|
toXContentCommon(builder, params);
|
||||||
|
builder.field("acknowledged", acknowledged);
|
||||||
|
}
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o)
|
||||||
|
return true;
|
||||||
|
if (o == null || getClass() != o.getClass())
|
||||||
|
return false;
|
||||||
|
DeleteDataFrameTransformAction.Response response = (DeleteDataFrameTransformAction.Response) o;
|
||||||
|
return super.equals(o) && acknowledged == response.acknowledged;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(super.hashCode(), acknowledged);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,223 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.elasticsearch.action.Action;
|
||||||
|
import org.elasticsearch.action.ActionRequestBuilder;
|
||||||
|
import org.elasticsearch.action.ActionRequestValidationException;
|
||||||
|
import org.elasticsearch.action.FailedNodeException;
|
||||||
|
import org.elasticsearch.action.TaskOperationFailure;
|
||||||
|
import org.elasticsearch.action.support.tasks.BaseTasksRequest;
|
||||||
|
import org.elasticsearch.action.support.tasks.BaseTasksResponse;
|
||||||
|
import org.elasticsearch.client.ElasticsearchClient;
|
||||||
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
|
import org.elasticsearch.common.ParseField;
|
||||||
|
import org.elasticsearch.common.Strings;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.common.logging.DeprecationLogger;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.tasks.Task;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformConfig;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
public class GetDataFrameTransformsAction extends Action<GetDataFrameTransformsAction.Response>{
|
||||||
|
|
||||||
|
public static final GetDataFrameTransformsAction INSTANCE = new GetDataFrameTransformsAction();
|
||||||
|
public static final String NAME = "cluster:monitor/data_frame/get";
|
||||||
|
|
||||||
|
private static final DeprecationLogger deprecationLogger = new DeprecationLogger(
|
||||||
|
LogManager.getLogger(GetDataFrameTransformsAction.class));
|
||||||
|
|
||||||
|
private GetDataFrameTransformsAction() {
|
||||||
|
super(NAME);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Response newResponse() {
|
||||||
|
return new Response();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Request extends BaseTasksRequest<Request> implements ToXContent {
|
||||||
|
private String id;
|
||||||
|
|
||||||
|
public Request(String id) {
|
||||||
|
if (Strings.isNullOrEmpty(id) || id.equals("*")) {
|
||||||
|
this.id = MetaData.ALL;
|
||||||
|
} else {
|
||||||
|
this.id = id;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public Request() {}
|
||||||
|
|
||||||
|
public Request(StreamInput in) throws IOException {
|
||||||
|
super(in);
|
||||||
|
id = in.readString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean match(Task task) {
|
||||||
|
// If we are retrieving all the transforms, the task description does not contain the id
|
||||||
|
if (id.equals(MetaData.ALL)) {
|
||||||
|
return task.getDescription().startsWith(DataFrameField.PERSISTENT_TASK_DESCRIPTION_PREFIX);
|
||||||
|
}
|
||||||
|
// Otherwise find the task by ID
|
||||||
|
return task.getDescription().equals(DataFrameField.PERSISTENT_TASK_DESCRIPTION_PREFIX + id);
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getId() {
|
||||||
|
return id;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
super.writeTo(out);
|
||||||
|
out.writeString(id);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ActionRequestValidationException validate() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.field(DataFrameField.ID.getPreferredName(), id);
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(id);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object obj) {
|
||||||
|
if (obj == null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (getClass() != obj.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
Request other = (Request) obj;
|
||||||
|
return Objects.equals(id, other.id);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class RequestBuilder extends ActionRequestBuilder<Request, Response> {
|
||||||
|
|
||||||
|
protected RequestBuilder(ElasticsearchClient client, GetDataFrameTransformsAction action) {
|
||||||
|
super(client, action, new Request());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Response extends BaseTasksResponse implements Writeable, ToXContentObject {
|
||||||
|
|
||||||
|
public static final String INVALID_TRANSFORMS_DEPRECATION_WARNING = "Found [{}] invalid transforms";
|
||||||
|
private static final ParseField INVALID_TRANSFORMS = new ParseField("invalid_transforms");
|
||||||
|
|
||||||
|
private List<DataFrameTransformConfig> transformConfigurations;
|
||||||
|
|
||||||
|
public Response(List<DataFrameTransformConfig> transformConfigs) {
|
||||||
|
super(Collections.emptyList(), Collections.emptyList());
|
||||||
|
this.transformConfigurations = transformConfigs;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Response(List<DataFrameTransformConfig> transformConfigs, List<TaskOperationFailure> taskFailures,
|
||||||
|
List<? extends FailedNodeException> nodeFailures) {
|
||||||
|
super(taskFailures, nodeFailures);
|
||||||
|
this.transformConfigurations = transformConfigs;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Response() {
|
||||||
|
super(Collections.emptyList(), Collections.emptyList());
|
||||||
|
}
|
||||||
|
|
||||||
|
public Response(StreamInput in) throws IOException {
|
||||||
|
super(Collections.emptyList(), Collections.emptyList());
|
||||||
|
readFrom(in);
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<DataFrameTransformConfig> getTransformConfigurations() {
|
||||||
|
return transformConfigurations;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
super.readFrom(in);
|
||||||
|
transformConfigurations = in.readList(DataFrameTransformConfig::new);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
super.writeTo(out);
|
||||||
|
out.writeList(transformConfigurations);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
List<String> invalidTransforms = new ArrayList<>();
|
||||||
|
builder.startObject();
|
||||||
|
builder.field(DataFrameField.COUNT.getPreferredName(), transformConfigurations.size());
|
||||||
|
// XContentBuilder does not support passing the params object for Iterables
|
||||||
|
builder.field(DataFrameField.TRANSFORMS.getPreferredName());
|
||||||
|
builder.startArray();
|
||||||
|
for (DataFrameTransformConfig configResponse : transformConfigurations) {
|
||||||
|
configResponse.toXContent(builder, params);
|
||||||
|
if (configResponse.isValid() == false) {
|
||||||
|
invalidTransforms.add(configResponse.getId());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
builder.endArray();
|
||||||
|
if (invalidTransforms.isEmpty() == false) {
|
||||||
|
builder.startObject(INVALID_TRANSFORMS.getPreferredName());
|
||||||
|
builder.field(DataFrameField.COUNT.getPreferredName(), invalidTransforms.size());
|
||||||
|
builder.field(DataFrameField.TRANSFORMS.getPreferredName(), invalidTransforms);
|
||||||
|
builder.endObject();
|
||||||
|
deprecationLogger.deprecated(INVALID_TRANSFORMS_DEPRECATION_WARNING, invalidTransforms.size());
|
||||||
|
}
|
||||||
|
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(transformConfigurations);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object other) {
|
||||||
|
if (this == other) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (other == null || getClass() != other.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
final Response that = (Response) other;
|
||||||
|
return Objects.equals(this.transformConfigurations, that.transformConfigurations);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final String toString() {
|
||||||
|
return Strings.toString(this);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,193 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.Action;
|
||||||
|
import org.elasticsearch.action.ActionRequestBuilder;
|
||||||
|
import org.elasticsearch.action.ActionRequestValidationException;
|
||||||
|
import org.elasticsearch.action.FailedNodeException;
|
||||||
|
import org.elasticsearch.action.TaskOperationFailure;
|
||||||
|
import org.elasticsearch.action.support.tasks.BaseTasksRequest;
|
||||||
|
import org.elasticsearch.action.support.tasks.BaseTasksResponse;
|
||||||
|
import org.elasticsearch.client.ElasticsearchClient;
|
||||||
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
|
import org.elasticsearch.common.Strings;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.tasks.Task;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
public class GetDataFrameTransformsStatsAction extends Action<GetDataFrameTransformsStatsAction.Response> {
|
||||||
|
|
||||||
|
public static final GetDataFrameTransformsStatsAction INSTANCE = new GetDataFrameTransformsStatsAction();
|
||||||
|
public static final String NAME = "cluster:monitor/data_frame_stats/get";
|
||||||
|
public GetDataFrameTransformsStatsAction() {
|
||||||
|
super(NAME);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Response newResponse() {
|
||||||
|
return new Response();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Request extends BaseTasksRequest<Request> implements ToXContent {
|
||||||
|
private String id;
|
||||||
|
|
||||||
|
public Request(String id) {
|
||||||
|
if (Strings.isNullOrEmpty(id) || id.equals("*")) {
|
||||||
|
this.id = MetaData.ALL;
|
||||||
|
} else {
|
||||||
|
this.id = id;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public Request() {}
|
||||||
|
|
||||||
|
public Request(StreamInput in) throws IOException {
|
||||||
|
super(in);
|
||||||
|
id = in.readString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean match(Task task) {
|
||||||
|
// If we are retrieving all the transforms, the task description does not contain the id
|
||||||
|
if (id.equals(MetaData.ALL)) {
|
||||||
|
return task.getDescription().startsWith(DataFrameField.PERSISTENT_TASK_DESCRIPTION_PREFIX);
|
||||||
|
}
|
||||||
|
// Otherwise find the task by ID
|
||||||
|
return task.getDescription().equals(DataFrameField.PERSISTENT_TASK_DESCRIPTION_PREFIX + id);
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getId() {
|
||||||
|
return id;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
super.writeTo(out);
|
||||||
|
out.writeString(id);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ActionRequestValidationException validate() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.field(DataFrameField.ID.getPreferredName(), id);
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(id);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object obj) {
|
||||||
|
if (obj == null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (getClass() != obj.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
Request other = (Request) obj;
|
||||||
|
return Objects.equals(id, other.id);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class RequestBuilder extends ActionRequestBuilder<Request, Response> {
|
||||||
|
|
||||||
|
protected RequestBuilder(ElasticsearchClient client, GetDataFrameTransformsStatsAction action) {
|
||||||
|
super(client, action, new Request());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Response extends BaseTasksResponse implements Writeable, ToXContentObject {
|
||||||
|
private List<DataFrameTransformStateAndStats> transformsStateAndStats;
|
||||||
|
|
||||||
|
public Response(List<DataFrameTransformStateAndStats> transformsStateAndStats) {
|
||||||
|
super(Collections.emptyList(), Collections.emptyList());
|
||||||
|
this.transformsStateAndStats = transformsStateAndStats;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Response(List<DataFrameTransformStateAndStats> transformsStateAndStats, List<TaskOperationFailure> taskFailures,
|
||||||
|
List<? extends FailedNodeException> nodeFailures) {
|
||||||
|
super(taskFailures, nodeFailures);
|
||||||
|
this.transformsStateAndStats = transformsStateAndStats;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Response() {
|
||||||
|
super(Collections.emptyList(), Collections.emptyList());
|
||||||
|
this.transformsStateAndStats = Collections.emptyList();
|
||||||
|
}
|
||||||
|
|
||||||
|
public Response(StreamInput in) throws IOException {
|
||||||
|
super(Collections.emptyList(), Collections.emptyList());
|
||||||
|
readFrom(in);
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<DataFrameTransformStateAndStats> getTransformsStateAndStats() {
|
||||||
|
return transformsStateAndStats;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
super.readFrom(in);
|
||||||
|
transformsStateAndStats = in.readList(DataFrameTransformStateAndStats::new);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
super.writeTo(out);
|
||||||
|
out.writeList(transformsStateAndStats);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
builder.field(DataFrameField.COUNT.getPreferredName(), transformsStateAndStats.size());
|
||||||
|
builder.field(DataFrameField.TRANSFORMS.getPreferredName(), transformsStateAndStats);
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(transformsStateAndStats);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object other) {
|
||||||
|
if (this == other) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (other == null || getClass() != other.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
final Response that = (Response) other;
|
||||||
|
return Objects.equals(this.transformsStateAndStats, that.transformsStateAndStats);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final String toString() {
|
||||||
|
return Strings.toString(this);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,119 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.Action;
|
||||||
|
import org.elasticsearch.action.ActionRequestValidationException;
|
||||||
|
import org.elasticsearch.action.support.master.AcknowledgedRequest;
|
||||||
|
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
||||||
|
import org.elasticsearch.action.support.master.MasterNodeOperationRequestBuilder;
|
||||||
|
import org.elasticsearch.client.ElasticsearchClient;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformConfig;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
public class PutDataFrameTransformAction extends Action<PutDataFrameTransformAction.Response> {
|
||||||
|
|
||||||
|
public static final PutDataFrameTransformAction INSTANCE = new PutDataFrameTransformAction();
|
||||||
|
public static final String NAME = "cluster:admin/data_frame/put";
|
||||||
|
|
||||||
|
private PutDataFrameTransformAction() {
|
||||||
|
super(NAME);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Response newResponse() {
|
||||||
|
return new Response();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Request extends AcknowledgedRequest<Request> implements ToXContentObject {
|
||||||
|
|
||||||
|
private DataFrameTransformConfig config;
|
||||||
|
|
||||||
|
public Request(DataFrameTransformConfig config) {
|
||||||
|
this.setConfig(config);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Request() {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Request fromXContent(final XContentParser parser, final String id) throws IOException {
|
||||||
|
return new Request(DataFrameTransformConfig.fromXContent(parser, id, false));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ActionRequestValidationException validate() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
return this.config.toXContent(builder, params);
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameTransformConfig getConfig() {
|
||||||
|
return config;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setConfig(DataFrameTransformConfig config) {
|
||||||
|
this.config = config;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
super.readFrom(in);
|
||||||
|
this.config = new DataFrameTransformConfig(in);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
super.writeTo(out);
|
||||||
|
this.config.writeTo(out);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(config);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object obj) {
|
||||||
|
if (obj == null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (getClass() != obj.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
Request other = (Request) obj;
|
||||||
|
return Objects.equals(config, other.config);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class RequestBuilder extends MasterNodeOperationRequestBuilder<Request, Response, RequestBuilder> {
|
||||||
|
|
||||||
|
protected RequestBuilder(ElasticsearchClient client, PutDataFrameTransformAction action) {
|
||||||
|
super(client, action, new Request());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Response extends AcknowledgedResponse {
|
||||||
|
public Response() {
|
||||||
|
super();
|
||||||
|
}
|
||||||
|
|
||||||
|
public Response(boolean acknowledged) {
|
||||||
|
super(acknowledged);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,162 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.Action;
|
||||||
|
import org.elasticsearch.action.ActionRequestBuilder;
|
||||||
|
import org.elasticsearch.action.ActionRequestValidationException;
|
||||||
|
import org.elasticsearch.action.support.tasks.BaseTasksRequest;
|
||||||
|
import org.elasticsearch.action.support.tasks.BaseTasksResponse;
|
||||||
|
import org.elasticsearch.client.ElasticsearchClient;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
public class StartDataFrameTransformAction extends Action<StartDataFrameTransformAction.Response> {
|
||||||
|
|
||||||
|
public static final StartDataFrameTransformAction INSTANCE = new StartDataFrameTransformAction();
|
||||||
|
public static final String NAME = "cluster:admin/data_frame/start";
|
||||||
|
|
||||||
|
private StartDataFrameTransformAction() {
|
||||||
|
super(NAME);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Response newResponse() {
|
||||||
|
return new Response();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Request extends BaseTasksRequest<Request> implements ToXContent {
|
||||||
|
private String id;
|
||||||
|
|
||||||
|
public Request(String id) {
|
||||||
|
this.id = ExceptionsHelper.requireNonNull(id, DataFrameField.ID.getPreferredName());
|
||||||
|
}
|
||||||
|
|
||||||
|
public Request() {
|
||||||
|
}
|
||||||
|
|
||||||
|
public Request(StreamInput in) throws IOException {
|
||||||
|
super(in);
|
||||||
|
id = in.readString();
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getId() {
|
||||||
|
return id;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
super.writeTo(out);
|
||||||
|
out.writeString(id);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ActionRequestValidationException validate() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.field(DataFrameField.ID.getPreferredName(), id);
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(id);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object obj) {
|
||||||
|
if (obj == null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (getClass() != obj.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
Request other = (Request) obj;
|
||||||
|
return Objects.equals(id, other.id);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class RequestBuilder extends ActionRequestBuilder<Request, Response> {
|
||||||
|
|
||||||
|
protected RequestBuilder(ElasticsearchClient client, StartDataFrameTransformAction action) {
|
||||||
|
super(client, action, new Request());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Response extends BaseTasksResponse implements Writeable, ToXContentObject {
|
||||||
|
private boolean started;
|
||||||
|
|
||||||
|
public Response() {
|
||||||
|
super(Collections.emptyList(), Collections.emptyList());
|
||||||
|
}
|
||||||
|
|
||||||
|
public Response(StreamInput in) throws IOException {
|
||||||
|
super(Collections.emptyList(), Collections.emptyList());
|
||||||
|
readFrom(in);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Response(boolean started) {
|
||||||
|
super(Collections.emptyList(), Collections.emptyList());
|
||||||
|
this.started = started;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isStarted() {
|
||||||
|
return started;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
super.readFrom(in);
|
||||||
|
started = in.readBoolean();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
super.writeTo(out);
|
||||||
|
out.writeBoolean(started);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
builder.field("started", started);
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object obj) {
|
||||||
|
if (this == obj) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (obj == null || getClass() != obj.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
Response response = (Response) obj;
|
||||||
|
return started == response.started;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(started);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,200 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.Action;
|
||||||
|
import org.elasticsearch.action.ActionRequestBuilder;
|
||||||
|
import org.elasticsearch.action.ActionRequestValidationException;
|
||||||
|
import org.elasticsearch.action.support.tasks.BaseTasksRequest;
|
||||||
|
import org.elasticsearch.action.support.tasks.BaseTasksResponse;
|
||||||
|
import org.elasticsearch.client.ElasticsearchClient;
|
||||||
|
import org.elasticsearch.common.Nullable;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.tasks.Task;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
|
public class StopDataFrameTransformAction extends Action<StopDataFrameTransformAction.Response> {
|
||||||
|
|
||||||
|
public static final StopDataFrameTransformAction INSTANCE = new StopDataFrameTransformAction();
|
||||||
|
public static final String NAME = "cluster:admin/data_frame/stop";
|
||||||
|
|
||||||
|
public static final TimeValue DEFAULT_TIMEOUT = new TimeValue(30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
|
private StopDataFrameTransformAction() {
|
||||||
|
super(NAME);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Response newResponse() {
|
||||||
|
return new Response();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Request extends BaseTasksRequest<Request> implements ToXContent {
|
||||||
|
private String id;
|
||||||
|
private final boolean waitForCompletion;
|
||||||
|
|
||||||
|
public Request(String id, boolean waitForCompletion, @Nullable TimeValue timeout) {
|
||||||
|
this.id = ExceptionsHelper.requireNonNull(id, DataFrameField.ID.getPreferredName());
|
||||||
|
this.waitForCompletion = waitForCompletion;
|
||||||
|
|
||||||
|
// use the timeout value already present in BaseTasksRequest
|
||||||
|
this.setTimeout(timeout == null ? DEFAULT_TIMEOUT : timeout);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Request() {
|
||||||
|
this(null, false, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Request(StreamInput in) throws IOException {
|
||||||
|
super(in);
|
||||||
|
id = in.readString();
|
||||||
|
waitForCompletion = in.readBoolean();
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getId() {
|
||||||
|
return id;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setId(String id) {
|
||||||
|
this.id = id;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean waitForCompletion() {
|
||||||
|
return waitForCompletion;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
super.writeTo(out);
|
||||||
|
out.writeString(id);
|
||||||
|
out.writeBoolean(waitForCompletion);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ActionRequestValidationException validate() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.field(DataFrameField.ID.getPreferredName(), id);
|
||||||
|
builder.field(DataFrameField.WAIT_FOR_COMPLETION.getPreferredName(), waitForCompletion);
|
||||||
|
if (this.getTimeout() != null) {
|
||||||
|
builder.field(DataFrameField.TIMEOUT.getPreferredName(), this.getTimeout());
|
||||||
|
}
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
// the base class does not implement hashCode, therefore we need to hash timeout ourselves
|
||||||
|
return Objects.hash(id, waitForCompletion, this.getTimeout());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object obj) {
|
||||||
|
if (this == obj) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (obj == null || getClass() != obj.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
Request other = (Request) obj;
|
||||||
|
|
||||||
|
// the base class does not implement equals, therefore we need to compare timeout ourselves
|
||||||
|
if (Objects.equals(this.getTimeout(), other.getTimeout()) == false) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
return Objects.equals(id, other.id) && Objects.equals(waitForCompletion, other.waitForCompletion);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean match(Task task) {
|
||||||
|
String expectedDescription = DataFrameField.PERSISTENT_TASK_DESCRIPTION_PREFIX + id;
|
||||||
|
|
||||||
|
return task.getDescription().equals(expectedDescription);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class RequestBuilder extends ActionRequestBuilder<Request, Response> {
|
||||||
|
|
||||||
|
protected RequestBuilder(ElasticsearchClient client, StopDataFrameTransformAction action) {
|
||||||
|
super(client, action, new Request());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Response extends BaseTasksResponse implements Writeable, ToXContentObject {
|
||||||
|
|
||||||
|
private boolean stopped;
|
||||||
|
|
||||||
|
public Response() {
|
||||||
|
super(Collections.emptyList(), Collections.emptyList());
|
||||||
|
}
|
||||||
|
|
||||||
|
public Response(StreamInput in) throws IOException {
|
||||||
|
super(Collections.emptyList(), Collections.emptyList());
|
||||||
|
readFrom(in);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Response(boolean stopped) {
|
||||||
|
super(Collections.emptyList(), Collections.emptyList());
|
||||||
|
this.stopped = stopped;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isStopped() {
|
||||||
|
return stopped;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
super.readFrom(in);
|
||||||
|
stopped = in.readBoolean();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
super.writeTo(out);
|
||||||
|
out.writeBoolean(stopped);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
builder.field("stopped", stopped);
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o)
|
||||||
|
return true;
|
||||||
|
if (o == null || getClass() != o.getClass())
|
||||||
|
return false;
|
||||||
|
Response response = (Response) o;
|
||||||
|
return stopped == response.stopped;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(stopped);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.ActionListenerResponseHandler;
|
||||||
|
import org.elasticsearch.action.FailedNodeException;
|
||||||
|
import org.elasticsearch.action.TaskOperationFailure;
|
||||||
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
|
import org.elasticsearch.action.support.tasks.TransportTasksAction;
|
||||||
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
|
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||||
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
|
import org.elasticsearch.common.inject.Inject;
|
||||||
|
import org.elasticsearch.discovery.MasterNotDiscoveredException;
|
||||||
|
import org.elasticsearch.persistent.PersistentTasksCustomMetaData;
|
||||||
|
import org.elasticsearch.persistent.PersistentTasksService;
|
||||||
|
import org.elasticsearch.tasks.Task;
|
||||||
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
import org.elasticsearch.transport.TransportService;
|
||||||
|
import org.elasticsearch.xpack.core.indexing.IndexerState;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.DeleteDataFrameTransformAction.Request;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.DeleteDataFrameTransformAction.Response;
|
||||||
|
import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformTask;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public class TransportDeleteDataFrameTransformAction extends TransportTasksAction<DataFrameTransformTask, Request, Response, Response> {
|
||||||
|
|
||||||
|
private final DataFrameTransformsConfigManager transformsConfigManager;
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public TransportDeleteDataFrameTransformAction(TransportService transportService, ThreadPool threadPool, ActionFilters actionFilters,
|
||||||
|
IndexNameExpressionResolver indexNameExpressionResolver, PersistentTasksService persistentTasksService,
|
||||||
|
ClusterService clusterService, DataFrameTransformsConfigManager transformsConfigManager) {
|
||||||
|
super(DeleteDataFrameTransformAction.NAME, clusterService, transportService, actionFilters, Request::new, Response::new,
|
||||||
|
Response::new, ThreadPool.Names.SAME);
|
||||||
|
this.transformsConfigManager = transformsConfigManager;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Response newResponse(Request request, List<Response> tasks, List<TaskOperationFailure> taskOperationFailures,
|
||||||
|
List<FailedNodeException> failedNodeExceptions) {
|
||||||
|
assert tasks.size() + taskOperationFailures.size() == 1;
|
||||||
|
boolean cancelled = tasks.size() > 0 && tasks.stream().allMatch(Response::isDeleted);
|
||||||
|
|
||||||
|
return new Response(cancelled, taskOperationFailures, failedNodeExceptions);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void taskOperation(Request request, DataFrameTransformTask task, ActionListener<Response> listener) {
|
||||||
|
assert task.getTransformId().equals(request.getId());
|
||||||
|
IndexerState state = task.getState().getIndexerState();
|
||||||
|
if (state.equals(IndexerState.STOPPED)) {
|
||||||
|
task.onCancelled();
|
||||||
|
transformsConfigManager.deleteTransformConfiguration(request.getId(), ActionListener.wrap(r -> {
|
||||||
|
listener.onResponse(new Response(true));
|
||||||
|
}, listener::onFailure));
|
||||||
|
} else {
|
||||||
|
listener.onFailure(new IllegalStateException("Could not delete transform [" + request.getId() + "] because "
|
||||||
|
+ "indexer state is [" + state + "]. Transform must be [" + IndexerState.STOPPED + "] before deletion."));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void doExecute(Task task, Request request, ActionListener<Response> listener) {
|
||||||
|
final ClusterState state = clusterService.state();
|
||||||
|
final DiscoveryNodes nodes = state.nodes();
|
||||||
|
if (nodes.isLocalNodeElectedMaster()) {
|
||||||
|
PersistentTasksCustomMetaData pTasksMeta = state.getMetaData().custom(PersistentTasksCustomMetaData.TYPE);
|
||||||
|
if (pTasksMeta != null && pTasksMeta.getTask(request.getId()) != null) {
|
||||||
|
super.doExecute(task, request, listener);
|
||||||
|
} else {
|
||||||
|
// we couldn't find the transform in the persistent task CS, but maybe the transform exists in the configuration index,
|
||||||
|
// if so delete the orphaned document and do not throw (for the normal case we want to stop the task first,
|
||||||
|
// than delete the configuration document if and only if the data frame transform is in stopped state)
|
||||||
|
transformsConfigManager.deleteTransformConfiguration(request.getId(), ActionListener.wrap(r -> {
|
||||||
|
listener.onResponse(new Response(true));
|
||||||
|
return;
|
||||||
|
}, listener::onFailure));
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// Delegates DeleteTransform to elected master node, so it becomes the coordinating node.
|
||||||
|
// Non-master nodes may have a stale cluster state that shows transforms which are cancelled
|
||||||
|
// on the master, which makes testing difficult.
|
||||||
|
if (nodes.getMasterNode() == null) {
|
||||||
|
listener.onFailure(new MasterNotDiscoveredException("no known master nodes"));
|
||||||
|
} else {
|
||||||
|
transportService.sendRequest(nodes.getMasterNode(), actionName, request,
|
||||||
|
new ActionListenerResponseHandler<>(listener, Response::new));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,101 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.ActionListenerResponseHandler;
|
||||||
|
import org.elasticsearch.action.FailedNodeException;
|
||||||
|
import org.elasticsearch.action.TaskOperationFailure;
|
||||||
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
|
import org.elasticsearch.action.support.tasks.TransportTasksAction;
|
||||||
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
|
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||||
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
|
import org.elasticsearch.common.inject.Inject;
|
||||||
|
import org.elasticsearch.discovery.MasterNotDiscoveredException;
|
||||||
|
import org.elasticsearch.tasks.Task;
|
||||||
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
import org.elasticsearch.transport.TransportService;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.GetDataFrameTransformsAction.Request;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.GetDataFrameTransformsAction.Response;
|
||||||
|
import org.elasticsearch.xpack.dataframe.persistence.DataFramePersistentTaskUtils;
|
||||||
|
import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformConfig;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformTask;
|
||||||
|
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
public class TransportGetDataFrameTransformsAction extends
|
||||||
|
TransportTasksAction<DataFrameTransformTask,
|
||||||
|
GetDataFrameTransformsAction.Request,
|
||||||
|
GetDataFrameTransformsAction.Response,
|
||||||
|
GetDataFrameTransformsAction.Response> {
|
||||||
|
|
||||||
|
private final DataFrameTransformsConfigManager transformsConfigManager;
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public TransportGetDataFrameTransformsAction(TransportService transportService, ActionFilters actionFilters,
|
||||||
|
ClusterService clusterService, DataFrameTransformsConfigManager transformsConfigManager) {
|
||||||
|
super(GetDataFrameTransformsAction.NAME, clusterService, transportService, actionFilters, GetDataFrameTransformsAction.Request::new,
|
||||||
|
GetDataFrameTransformsAction.Response::new, GetDataFrameTransformsAction.Response::new, ThreadPool.Names.SAME);
|
||||||
|
this.transformsConfigManager = transformsConfigManager;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Response newResponse(Request request, List<Response> tasks, List<TaskOperationFailure> taskOperationFailures,
|
||||||
|
List<FailedNodeException> failedNodeExceptions) {
|
||||||
|
List<DataFrameTransformConfig> configs = tasks.stream().map(GetDataFrameTransformsAction.Response::getTransformConfigurations)
|
||||||
|
.flatMap(Collection::stream).collect(Collectors.toList());
|
||||||
|
return new Response(configs, taskOperationFailures, failedNodeExceptions);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void taskOperation(Request request, DataFrameTransformTask task, ActionListener<Response> listener) {
|
||||||
|
assert task.getTransformId().equals(request.getId()) || request.getId().equals(MetaData.ALL);
|
||||||
|
// Little extra insurance, make sure we only return transforms that aren't cancelled
|
||||||
|
if (task.isCancelled() == false) {
|
||||||
|
transformsConfigManager.getTransformConfiguration(task.getTransformId(), ActionListener.wrap(config -> {
|
||||||
|
listener.onResponse(new Response(Collections.singletonList(config)));
|
||||||
|
}, e -> {
|
||||||
|
listener.onFailure(new RuntimeException("failed to retrieve...", e));
|
||||||
|
}));
|
||||||
|
} else {
|
||||||
|
listener.onResponse(new Response(Collections.emptyList()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void doExecute(Task task, Request request, ActionListener<Response> listener) {
|
||||||
|
final ClusterState state = clusterService.state();
|
||||||
|
final DiscoveryNodes nodes = state.nodes();
|
||||||
|
|
||||||
|
if (nodes.isLocalNodeElectedMaster()) {
|
||||||
|
if (DataFramePersistentTaskUtils.stateHasDataFrameTransforms(request.getId(), state)) {
|
||||||
|
super.doExecute(task, request, listener);
|
||||||
|
} else {
|
||||||
|
// If we couldn't find the transform in the persistent task CS, it means it was deleted prior to this GET
|
||||||
|
// and we can just send an empty response, no need to go looking for the allocated task
|
||||||
|
listener.onResponse(new Response(Collections.emptyList()));
|
||||||
|
}
|
||||||
|
|
||||||
|
} else {
|
||||||
|
// Delegates GetTransforms to elected master node, so it becomes the coordinating node.
|
||||||
|
// Non-master nodes may have a stale cluster state that shows transforms which are cancelled
|
||||||
|
// on the master, which makes testing difficult.
|
||||||
|
if (nodes.getMasterNode() == null) {
|
||||||
|
listener.onFailure(new MasterNotDiscoveredException("no known master nodes"));
|
||||||
|
} else {
|
||||||
|
transportService.sendRequest(nodes.getMasterNode(), actionName, request,
|
||||||
|
new ActionListenerResponseHandler<>(listener, Response::new));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,98 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.ActionListenerResponseHandler;
|
||||||
|
import org.elasticsearch.action.FailedNodeException;
|
||||||
|
import org.elasticsearch.action.TaskOperationFailure;
|
||||||
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
|
import org.elasticsearch.action.support.tasks.TransportTasksAction;
|
||||||
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
|
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||||
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
|
import org.elasticsearch.common.inject.Inject;
|
||||||
|
import org.elasticsearch.discovery.MasterNotDiscoveredException;
|
||||||
|
import org.elasticsearch.tasks.Task;
|
||||||
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
import org.elasticsearch.transport.TransportService;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.GetDataFrameTransformsStatsAction.Request;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.GetDataFrameTransformsStatsAction.Response;
|
||||||
|
import org.elasticsearch.xpack.dataframe.persistence.DataFramePersistentTaskUtils;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformTask;
|
||||||
|
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
public class TransportGetDataFrameTransformsStatsAction extends
|
||||||
|
TransportTasksAction<DataFrameTransformTask,
|
||||||
|
GetDataFrameTransformsStatsAction.Request,
|
||||||
|
GetDataFrameTransformsStatsAction.Response,
|
||||||
|
GetDataFrameTransformsStatsAction.Response> {
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public TransportGetDataFrameTransformsStatsAction(TransportService transportService, ActionFilters actionFilters,
|
||||||
|
ClusterService clusterService) {
|
||||||
|
super(GetDataFrameTransformsStatsAction.NAME, clusterService, transportService, actionFilters, Request::new, Response::new,
|
||||||
|
Response::new, ThreadPool.Names.SAME);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Response newResponse(Request request, List<Response> tasks, List<TaskOperationFailure> taskOperationFailures,
|
||||||
|
List<FailedNodeException> failedNodeExceptions) {
|
||||||
|
List<DataFrameTransformStateAndStats> responses = tasks.stream()
|
||||||
|
.map(GetDataFrameTransformsStatsAction.Response::getTransformsStateAndStats).flatMap(Collection::stream)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
return new Response(responses, taskOperationFailures, failedNodeExceptions);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void taskOperation(Request request, DataFrameTransformTask task, ActionListener<Response> listener) {
|
||||||
|
List<DataFrameTransformStateAndStats> transformsStateAndStats = Collections.emptyList();
|
||||||
|
|
||||||
|
assert task.getTransformId().equals(request.getId()) || request.getId().equals(MetaData.ALL);
|
||||||
|
|
||||||
|
// Little extra insurance, make sure we only return transforms that aren't cancelled
|
||||||
|
if (task.isCancelled() == false) {
|
||||||
|
DataFrameTransformStateAndStats transformStateAndStats = new DataFrameTransformStateAndStats(task.getTransformId(),
|
||||||
|
task.getState(), task.getStats());
|
||||||
|
transformsStateAndStats = Collections.singletonList(transformStateAndStats);
|
||||||
|
}
|
||||||
|
|
||||||
|
listener.onResponse(new Response(transformsStateAndStats));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void doExecute(Task task, Request request, ActionListener<Response> listener) {
|
||||||
|
final ClusterState state = clusterService.state();
|
||||||
|
final DiscoveryNodes nodes = state.nodes();
|
||||||
|
|
||||||
|
if (nodes.isLocalNodeElectedMaster()) {
|
||||||
|
if (DataFramePersistentTaskUtils.stateHasDataFrameTransforms(request.getId(), state)) {
|
||||||
|
super.doExecute(task, request, listener);
|
||||||
|
} else {
|
||||||
|
// If we couldn't find the transform in the persistent task CS, it means it was deleted prior to this GET
|
||||||
|
// and we can just send an empty response, no need to go looking for the allocated task
|
||||||
|
listener.onResponse(new Response(Collections.emptyList()));
|
||||||
|
}
|
||||||
|
|
||||||
|
} else {
|
||||||
|
// Delegates GetTransforms to elected master node, so it becomes the coordinating node.
|
||||||
|
// Non-master nodes may have a stale cluster state that shows transforms which are cancelled
|
||||||
|
// on the master, which makes testing difficult.
|
||||||
|
if (nodes.getMasterNode() == null) {
|
||||||
|
listener.onFailure(new MasterNotDiscoveredException("no known master nodes"));
|
||||||
|
} else {
|
||||||
|
transportService.sendRequest(nodes.getMasterNode(), actionName, request,
|
||||||
|
new ActionListenerResponseHandler<>(listener, Response::new));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,148 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.elasticsearch.ResourceAlreadyExistsException;
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
|
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
|
||||||
|
import org.elasticsearch.client.Client;
|
||||||
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
|
import org.elasticsearch.cluster.block.ClusterBlockException;
|
||||||
|
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||||
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
|
import org.elasticsearch.common.inject.Inject;
|
||||||
|
import org.elasticsearch.index.query.MatchAllQueryBuilder;
|
||||||
|
import org.elasticsearch.license.LicenseUtils;
|
||||||
|
import org.elasticsearch.license.XPackLicenseState;
|
||||||
|
import org.elasticsearch.persistent.PersistentTasksCustomMetaData;
|
||||||
|
import org.elasticsearch.persistent.PersistentTasksService;
|
||||||
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
import org.elasticsearch.transport.TransportService;
|
||||||
|
import org.elasticsearch.xpack.core.XPackField;
|
||||||
|
import org.elasticsearch.xpack.core.XPackPlugin;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameMessages;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.PutDataFrameTransformAction.Request;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.PutDataFrameTransformAction.Response;
|
||||||
|
import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager;
|
||||||
|
import org.elasticsearch.xpack.dataframe.persistence.DataframeIndex;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransform;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.pivot.Pivot;
|
||||||
|
|
||||||
|
public class TransportPutDataFrameTransformAction
|
||||||
|
extends TransportMasterNodeAction<PutDataFrameTransformAction.Request, PutDataFrameTransformAction.Response> {
|
||||||
|
|
||||||
|
private static final Logger logger = LogManager.getLogger(TransportPutDataFrameTransformAction.class);
|
||||||
|
|
||||||
|
private final XPackLicenseState licenseState;
|
||||||
|
private final PersistentTasksService persistentTasksService;
|
||||||
|
private final Client client;
|
||||||
|
private final DataFrameTransformsConfigManager dataFrameTransformsConfigManager;
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public TransportPutDataFrameTransformAction(TransportService transportService, ThreadPool threadPool, ActionFilters actionFilters,
|
||||||
|
IndexNameExpressionResolver indexNameExpressionResolver, ClusterService clusterService, XPackLicenseState licenseState,
|
||||||
|
PersistentTasksService persistentTasksService, DataFrameTransformsConfigManager dataFrameTransformsConfigManager,
|
||||||
|
Client client) {
|
||||||
|
super(PutDataFrameTransformAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver,
|
||||||
|
PutDataFrameTransformAction.Request::new);
|
||||||
|
this.licenseState = licenseState;
|
||||||
|
this.persistentTasksService = persistentTasksService;
|
||||||
|
this.client = client;
|
||||||
|
this.dataFrameTransformsConfigManager = dataFrameTransformsConfigManager;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String executor() {
|
||||||
|
return ThreadPool.Names.SAME;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected PutDataFrameTransformAction.Response newResponse() {
|
||||||
|
return new PutDataFrameTransformAction.Response();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void masterOperation(Request request, ClusterState clusterState, ActionListener<Response> listener) throws Exception {
|
||||||
|
|
||||||
|
if (!licenseState.isDataFrameAllowed()) {
|
||||||
|
listener.onFailure(LicenseUtils.newComplianceException(XPackField.DATA_FRAME));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
XPackPlugin.checkReadyForXPackCustomMetadata(clusterState);
|
||||||
|
|
||||||
|
String transformId = request.getConfig().getId();
|
||||||
|
// quick check whether a transform has already been created under that name
|
||||||
|
if (PersistentTasksCustomMetaData.getTaskWithId(clusterState, transformId) != null) {
|
||||||
|
listener.onFailure(new ResourceAlreadyExistsException(
|
||||||
|
DataFrameMessages.getMessage(DataFrameMessages.REST_PUT_DATA_FRAME_TRANSFORM_EXISTS, transformId)));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
// create the transform, for now we only have pivot and no support for custom queries
|
||||||
|
Pivot pivot = new Pivot(request.getConfig().getSource(), new MatchAllQueryBuilder(), request.getConfig().getPivotConfig());
|
||||||
|
|
||||||
|
// the non-state creating steps are done first, so we minimize the chance to end up with orphaned state transform validation
|
||||||
|
pivot.validate(client, ActionListener.wrap(validationResult -> {
|
||||||
|
// deduce target mappings
|
||||||
|
pivot.deduceMappings(client, ActionListener.wrap(mappings -> {
|
||||||
|
// create the destination index
|
||||||
|
DataframeIndex.createDestinationIndex(client, request.getConfig(), mappings, ActionListener.wrap(createIndexResult -> {
|
||||||
|
DataFrameTransform transform = createDataFrameTransform(transformId, threadPool);
|
||||||
|
// create the transform configuration and store it in the internal index
|
||||||
|
dataFrameTransformsConfigManager.putTransformConfiguration(request.getConfig(), ActionListener.wrap(r -> {
|
||||||
|
// finally start the persistent task
|
||||||
|
persistentTasksService.sendStartRequest(transform.getId(), DataFrameTransform.NAME, transform,
|
||||||
|
ActionListener.wrap(persistentTask -> {
|
||||||
|
listener.onResponse(new PutDataFrameTransformAction.Response(true));
|
||||||
|
}, startPersistentTaskException -> {
|
||||||
|
// delete the otherwise orphaned transform configuration, for now we do not delete the destination index
|
||||||
|
dataFrameTransformsConfigManager.deleteTransformConfiguration(transformId, ActionListener.wrap(r2 -> {
|
||||||
|
logger.debug("Deleted data frame transform [{}] configuration from data frame configuration index",
|
||||||
|
transformId);
|
||||||
|
listener.onFailure(
|
||||||
|
new RuntimeException(
|
||||||
|
DataFrameMessages.getMessage(
|
||||||
|
DataFrameMessages.REST_PUT_DATA_FRAME_FAILED_TO_START_PERSISTENT_TASK, r2),
|
||||||
|
startPersistentTaskException));
|
||||||
|
}, deleteTransformFromIndexException -> {
|
||||||
|
logger.error("Failed to cleanup orphaned data frame transform [{}] configuration", transformId);
|
||||||
|
listener.onFailure(
|
||||||
|
new RuntimeException(
|
||||||
|
DataFrameMessages.getMessage(
|
||||||
|
DataFrameMessages.REST_PUT_DATA_FRAME_FAILED_TO_START_PERSISTENT_TASK, false),
|
||||||
|
startPersistentTaskException));
|
||||||
|
}));
|
||||||
|
}));
|
||||||
|
}, listener::onFailure));
|
||||||
|
}, createDestinationIndexException -> {
|
||||||
|
listener.onFailure(new RuntimeException(DataFrameMessages.REST_PUT_DATA_FRAME_FAILED_TO_CREATE_TARGET_INDEX,
|
||||||
|
createDestinationIndexException));
|
||||||
|
}));
|
||||||
|
}, deduceTargetMappingsException -> {
|
||||||
|
listener.onFailure(new RuntimeException(DataFrameMessages.REST_PUT_DATA_FRAME_FAILED_TO_DEDUCE_TARGET_MAPPINGS,
|
||||||
|
deduceTargetMappingsException));
|
||||||
|
}));
|
||||||
|
}, validationException -> {
|
||||||
|
listener.onFailure(new RuntimeException(DataFrameMessages.REST_PUT_DATA_FRAME_FAILED_TO_VALIDATE_DATA_FRAME_CONFIGURATION,
|
||||||
|
validationException));
|
||||||
|
}));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static DataFrameTransform createDataFrameTransform(String transformId, ThreadPool threadPool) {
|
||||||
|
return new DataFrameTransform(transformId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected ClusterBlockException checkBlock(PutDataFrameTransformAction.Request request, ClusterState state) {
|
||||||
|
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,110 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.ResourceNotFoundException;
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.FailedNodeException;
|
||||||
|
import org.elasticsearch.action.TaskOperationFailure;
|
||||||
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
|
import org.elasticsearch.action.support.tasks.TransportTasksAction;
|
||||||
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
|
import org.elasticsearch.common.inject.Inject;
|
||||||
|
import org.elasticsearch.license.LicenseUtils;
|
||||||
|
import org.elasticsearch.license.XPackLicenseState;
|
||||||
|
import org.elasticsearch.tasks.Task;
|
||||||
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
import org.elasticsearch.transport.TransportService;
|
||||||
|
import org.elasticsearch.xpack.core.XPackField;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformTask;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.function.Consumer;
|
||||||
|
|
||||||
|
public class TransportStartDataFrameTransformAction extends
|
||||||
|
TransportTasksAction<DataFrameTransformTask, StartDataFrameTransformAction.Request,
|
||||||
|
StartDataFrameTransformAction.Response, StartDataFrameTransformAction.Response> {
|
||||||
|
|
||||||
|
private final XPackLicenseState licenseState;
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public TransportStartDataFrameTransformAction(TransportService transportService, ActionFilters actionFilters,
|
||||||
|
ClusterService clusterService, XPackLicenseState licenseState) {
|
||||||
|
super(StartDataFrameTransformAction.NAME, clusterService, transportService, actionFilters,
|
||||||
|
StartDataFrameTransformAction.Request::new, StartDataFrameTransformAction.Response::new,
|
||||||
|
StartDataFrameTransformAction.Response::new, ThreadPool.Names.SAME);
|
||||||
|
this.licenseState = licenseState;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void processTasks(StartDataFrameTransformAction.Request request, Consumer<DataFrameTransformTask> operation) {
|
||||||
|
DataFrameTransformTask matchingTask = null;
|
||||||
|
|
||||||
|
// todo: re-factor, see rollup TransportTaskHelper
|
||||||
|
for (Task task : taskManager.getTasks().values()) {
|
||||||
|
if (task instanceof DataFrameTransformTask
|
||||||
|
&& ((DataFrameTransformTask) task).getTransformId().equals(request.getId())) {
|
||||||
|
if (matchingTask != null) {
|
||||||
|
throw new IllegalArgumentException("Found more than one matching task for data frame transform [" + request.getId()
|
||||||
|
+ "] when " + "there should only be one.");
|
||||||
|
}
|
||||||
|
matchingTask = (DataFrameTransformTask) task;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (matchingTask != null) {
|
||||||
|
operation.accept(matchingTask);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void doExecute(Task task, StartDataFrameTransformAction.Request request,
|
||||||
|
ActionListener<StartDataFrameTransformAction.Response> listener) {
|
||||||
|
|
||||||
|
if (!licenseState.isDataFrameAllowed()) {
|
||||||
|
listener.onFailure(LicenseUtils.newComplianceException(XPackField.DATA_FRAME));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
super.doExecute(task, request, listener);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void taskOperation(StartDataFrameTransformAction.Request request, DataFrameTransformTask transformTask,
|
||||||
|
ActionListener<StartDataFrameTransformAction.Response> listener) {
|
||||||
|
if (transformTask.getTransformId().equals(request.getId())) {
|
||||||
|
transformTask.start(listener);
|
||||||
|
} else {
|
||||||
|
listener.onFailure(new RuntimeException("ID of data frame transform task [" + transformTask.getTransformId()
|
||||||
|
+ "] does not match request's ID [" + request.getId() + "]"));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected StartDataFrameTransformAction.Response newResponse(StartDataFrameTransformAction.Request request,
|
||||||
|
List<StartDataFrameTransformAction.Response> tasks, List<TaskOperationFailure> taskOperationFailures,
|
||||||
|
List<FailedNodeException> failedNodeExceptions) {
|
||||||
|
|
||||||
|
if (taskOperationFailures.isEmpty() == false) {
|
||||||
|
throw org.elasticsearch.ExceptionsHelper.convertToElastic(taskOperationFailures.get(0).getCause());
|
||||||
|
} else if (failedNodeExceptions.isEmpty() == false) {
|
||||||
|
throw org.elasticsearch.ExceptionsHelper.convertToElastic(failedNodeExceptions.get(0));
|
||||||
|
}
|
||||||
|
|
||||||
|
// Either the transform doesn't exist (the user didn't create it yet) or was deleted
|
||||||
|
// after the StartAPI executed.
|
||||||
|
// In either case, let the user know
|
||||||
|
if (tasks.size() == 0) {
|
||||||
|
throw new ResourceNotFoundException("Task for data frame transform [" + request.getId() + "] not found");
|
||||||
|
}
|
||||||
|
|
||||||
|
assert tasks.size() == 1;
|
||||||
|
|
||||||
|
boolean allStarted = tasks.stream().allMatch(StartDataFrameTransformAction.Response::isStarted);
|
||||||
|
return new StartDataFrameTransformAction.Response(allStarted);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,120 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.ElasticsearchException;
|
||||||
|
import org.elasticsearch.ElasticsearchTimeoutException;
|
||||||
|
import org.elasticsearch.ExceptionsHelper;
|
||||||
|
import org.elasticsearch.ResourceNotFoundException;
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.FailedNodeException;
|
||||||
|
import org.elasticsearch.action.TaskOperationFailure;
|
||||||
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
|
import org.elasticsearch.action.support.tasks.TransportTasksAction;
|
||||||
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
|
import org.elasticsearch.common.inject.Inject;
|
||||||
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
|
import org.elasticsearch.tasks.Task;
|
||||||
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
import org.elasticsearch.transport.TransportService;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameMessages;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformTask;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
|
||||||
|
|
||||||
|
public class TransportStopDataFrameTransformAction extends
|
||||||
|
TransportTasksAction<DataFrameTransformTask, StopDataFrameTransformAction.Request,
|
||||||
|
StopDataFrameTransformAction.Response, StopDataFrameTransformAction.Response> {
|
||||||
|
|
||||||
|
private static final TimeValue WAIT_FOR_COMPLETION_POLL = timeValueMillis(100);
|
||||||
|
private final ThreadPool threadPool;
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public TransportStopDataFrameTransformAction(TransportService transportService, ActionFilters actionFilters,
|
||||||
|
ClusterService clusterService, ThreadPool threadPool) {
|
||||||
|
super(StopDataFrameTransformAction.NAME, clusterService, transportService, actionFilters, StopDataFrameTransformAction.Request::new,
|
||||||
|
StopDataFrameTransformAction.Response::new, StopDataFrameTransformAction.Response::new, ThreadPool.Names.SAME);
|
||||||
|
this.threadPool = threadPool;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void doExecute(Task task, StopDataFrameTransformAction.Request request,
|
||||||
|
ActionListener<StopDataFrameTransformAction.Response> listener) {
|
||||||
|
super.doExecute(task, request, listener);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void taskOperation(StopDataFrameTransformAction.Request request, DataFrameTransformTask transformTask,
|
||||||
|
ActionListener<StopDataFrameTransformAction.Response> listener) {
|
||||||
|
if (transformTask.getTransformId().equals(request.getId())) {
|
||||||
|
if (request.waitForCompletion() == false) {
|
||||||
|
transformTask.stop(listener);
|
||||||
|
} else {
|
||||||
|
ActionListener<StopDataFrameTransformAction.Response> blockingListener = ActionListener.wrap(response -> {
|
||||||
|
if (response.isStopped()) {
|
||||||
|
// The Task acknowledged that it is stopped/stopping... wait until the status actually
|
||||||
|
// changes over before returning. Switch over to Generic threadpool so
|
||||||
|
// we don't block the network thread
|
||||||
|
threadPool.generic().execute(() -> {
|
||||||
|
try {
|
||||||
|
long untilInNanos = System.nanoTime() + request.getTimeout().getNanos();
|
||||||
|
|
||||||
|
while (System.nanoTime() - untilInNanos < 0) {
|
||||||
|
if (transformTask.isStopped()) {
|
||||||
|
listener.onResponse(response);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
Thread.sleep(WAIT_FOR_COMPLETION_POLL.millis());
|
||||||
|
}
|
||||||
|
// ran out of time
|
||||||
|
listener.onFailure(new ElasticsearchTimeoutException(
|
||||||
|
DataFrameMessages.getMessage(DataFrameMessages.REST_STOP_TRANSFORM_WAIT_FOR_COMPLETION_TIMEOUT,
|
||||||
|
request.getTimeout().getStringRep(), request.getId())));
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
listener.onFailure(new ElasticsearchException(DataFrameMessages.getMessage(
|
||||||
|
DataFrameMessages.REST_STOP_TRANSFORM_WAIT_FOR_COMPLETION_INTERRUPT, request.getId()), e));
|
||||||
|
}
|
||||||
|
});
|
||||||
|
} else {
|
||||||
|
// Did not acknowledge stop, just return the response
|
||||||
|
listener.onResponse(response);
|
||||||
|
}
|
||||||
|
}, listener::onFailure);
|
||||||
|
|
||||||
|
transformTask.stop(blockingListener);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
listener.onFailure(new RuntimeException("ID of data frame indexer task [" + transformTask.getTransformId()
|
||||||
|
+ "] does not match request's ID [" + request.getId() + "]"));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected StopDataFrameTransformAction.Response newResponse(StopDataFrameTransformAction.Request request,
|
||||||
|
List<StopDataFrameTransformAction.Response> tasks, List<TaskOperationFailure> taskOperationFailures,
|
||||||
|
List<FailedNodeException> failedNodeExceptions) {
|
||||||
|
|
||||||
|
if (taskOperationFailures.isEmpty() == false) {
|
||||||
|
throw ExceptionsHelper.convertToElastic(taskOperationFailures.get(0).getCause());
|
||||||
|
} else if (failedNodeExceptions.isEmpty() == false) {
|
||||||
|
throw ExceptionsHelper.convertToElastic(failedNodeExceptions.get(0));
|
||||||
|
}
|
||||||
|
|
||||||
|
// Either the transform doesn't exist (the user didn't create it yet) or was deleted
|
||||||
|
// after the Stop API executed.
|
||||||
|
// In either case, let the user know
|
||||||
|
if (tasks.size() == 0) {
|
||||||
|
throw new ResourceNotFoundException("Task for Data Frame transform [" + request.getId() + "] not found");
|
||||||
|
}
|
||||||
|
|
||||||
|
assert tasks.size() == 1;
|
||||||
|
|
||||||
|
boolean allStopped = tasks.stream().allMatch(StopDataFrameTransformAction.Response::isStopped);
|
||||||
|
return new StopDataFrameTransformAction.Response(allStopped);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,107 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.persistence;
|
||||||
|
|
||||||
|
import org.elasticsearch.Version;
|
||||||
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
|
import org.elasticsearch.cluster.metadata.IndexTemplateMetaData;
|
||||||
|
import org.elasticsearch.common.Strings;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.index.mapper.MapperService;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collections;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||||
|
|
||||||
|
public final class DataFrameInternalIndex {
|
||||||
|
|
||||||
|
// constants for the index
|
||||||
|
public static final String INDEX_TEMPLATE_VERSION = "1";
|
||||||
|
public static final String INDEX_TEMPLATE_PATTERN = ".data-frame-internal-";
|
||||||
|
public static final String INDEX_TEMPLATE_NAME = INDEX_TEMPLATE_PATTERN + INDEX_TEMPLATE_VERSION;
|
||||||
|
public static final String INDEX_NAME = INDEX_TEMPLATE_NAME;
|
||||||
|
|
||||||
|
// constants for mappings
|
||||||
|
public static final String ENABLED = "enabled";
|
||||||
|
public static final String DYNAMIC = "dynamic";
|
||||||
|
public static final String PROPERTIES = "properties";
|
||||||
|
public static final String TYPE = "type";
|
||||||
|
|
||||||
|
// data types
|
||||||
|
public static final String DOUBLE = "double";
|
||||||
|
public static final String KEYWORD = "keyword";
|
||||||
|
|
||||||
|
// internal document types, e.g. "transform_config"
|
||||||
|
public static final String DOC_TYPE = "doc_type";
|
||||||
|
|
||||||
|
public static IndexTemplateMetaData getIndexTemplateMetaData() throws IOException {
|
||||||
|
IndexTemplateMetaData dataFrameTemplate = IndexTemplateMetaData.builder(INDEX_TEMPLATE_NAME)
|
||||||
|
.patterns(Collections.singletonList(INDEX_TEMPLATE_NAME))
|
||||||
|
.version(Version.CURRENT.id)
|
||||||
|
.settings(Settings.builder()
|
||||||
|
// the configurations are expected to be small
|
||||||
|
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
|
||||||
|
.put(IndexMetaData.SETTING_AUTO_EXPAND_REPLICAS, "0-1"))
|
||||||
|
// todo: remove type
|
||||||
|
.putMapping(MapperService.SINGLE_MAPPING_NAME, Strings.toString(mappings()))
|
||||||
|
.build();
|
||||||
|
return dataFrameTemplate;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static XContentBuilder mappings() throws IOException {
|
||||||
|
XContentBuilder builder = jsonBuilder();
|
||||||
|
builder.startObject();
|
||||||
|
|
||||||
|
builder.startObject(MapperService.SINGLE_MAPPING_NAME);
|
||||||
|
addMetaInformation(builder);
|
||||||
|
|
||||||
|
// no need to analyze anything, we use the config index as key value store, revisit if we decide to search on it
|
||||||
|
builder.field(ENABLED, false);
|
||||||
|
// do not allow anything outside of the defined schema
|
||||||
|
builder.field(DYNAMIC, "strict");
|
||||||
|
// the schema definitions
|
||||||
|
builder.startObject(PROPERTIES);
|
||||||
|
// overall doc type
|
||||||
|
builder.startObject(DOC_TYPE).field(TYPE, KEYWORD).endObject();
|
||||||
|
// add the schema for transform configurations
|
||||||
|
addDataFrameTransformsConfigMappings(builder);
|
||||||
|
|
||||||
|
// end type
|
||||||
|
builder.endObject();
|
||||||
|
// end properties
|
||||||
|
builder.endObject();
|
||||||
|
// end mapping
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static XContentBuilder addDataFrameTransformsConfigMappings(XContentBuilder builder) throws IOException {
|
||||||
|
return builder
|
||||||
|
.startObject(DataFrameField.ID.getPreferredName())
|
||||||
|
.field(TYPE, KEYWORD)
|
||||||
|
.endObject();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Inserts "_meta" containing useful information like the version into the mapping
|
||||||
|
* template.
|
||||||
|
*
|
||||||
|
* @param builder The builder for the mappings
|
||||||
|
* @throws IOException On write error
|
||||||
|
*/
|
||||||
|
private static XContentBuilder addMetaInformation(XContentBuilder builder) throws IOException {
|
||||||
|
return builder.startObject("_meta")
|
||||||
|
.field("version", Version.CURRENT)
|
||||||
|
.endObject();
|
||||||
|
}
|
||||||
|
|
||||||
|
private DataFrameInternalIndex() {
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,41 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.persistence;
|
||||||
|
|
||||||
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
|
import org.elasticsearch.persistent.PersistentTasksCustomMetaData;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
|
||||||
|
public final class DataFramePersistentTaskUtils {
|
||||||
|
|
||||||
|
private DataFramePersistentTaskUtils() {
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check to see if the PersistentTask's cluster state contains the data frame transform(s) we
|
||||||
|
* are interested in
|
||||||
|
*/
|
||||||
|
public static boolean stateHasDataFrameTransforms(String id, ClusterState state) {
|
||||||
|
boolean hasTransforms = false;
|
||||||
|
PersistentTasksCustomMetaData pTasksMeta = state.getMetaData().custom(PersistentTasksCustomMetaData.TYPE);
|
||||||
|
|
||||||
|
if (pTasksMeta != null) {
|
||||||
|
// If the request was for _all transforms, we need to look through the list of
|
||||||
|
// persistent tasks and see if at least one is a data frame task
|
||||||
|
if (id.equals(MetaData.ALL)) {
|
||||||
|
hasTransforms = pTasksMeta.tasks().stream()
|
||||||
|
.anyMatch(persistentTask -> persistentTask.getTaskName().equals(DataFrameField.TASK_NAME));
|
||||||
|
|
||||||
|
} else if (pTasksMeta.getTask(id) != null) {
|
||||||
|
// If we're looking for a single transform, we can just check directly
|
||||||
|
hasTransforms = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return hasTransforms;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,151 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.persistence;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.elasticsearch.ElasticsearchParseException;
|
||||||
|
import org.elasticsearch.ResourceAlreadyExistsException;
|
||||||
|
import org.elasticsearch.ResourceNotFoundException;
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.DocWriteRequest;
|
||||||
|
import org.elasticsearch.action.DocWriteResponse;
|
||||||
|
import org.elasticsearch.action.delete.DeleteAction;
|
||||||
|
import org.elasticsearch.action.delete.DeleteRequest;
|
||||||
|
import org.elasticsearch.action.get.GetAction;
|
||||||
|
import org.elasticsearch.action.get.GetRequest;
|
||||||
|
import org.elasticsearch.action.index.IndexAction;
|
||||||
|
import org.elasticsearch.action.index.IndexRequest;
|
||||||
|
import org.elasticsearch.action.support.WriteRequest;
|
||||||
|
import org.elasticsearch.client.Client;
|
||||||
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
|
import org.elasticsearch.common.xcontent.LoggingDeprecationHandler;
|
||||||
|
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.index.IndexNotFoundException;
|
||||||
|
import org.elasticsearch.index.engine.VersionConflictEngineException;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameMessages;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformConfig;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InputStream;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import static org.elasticsearch.xpack.core.ClientHelper.DATA_FRAME_ORIGIN;
|
||||||
|
import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin;
|
||||||
|
|
||||||
|
public class DataFrameTransformsConfigManager {
|
||||||
|
|
||||||
|
private static final Logger logger = LogManager.getLogger(DataFrameTransformsConfigManager.class);
|
||||||
|
|
||||||
|
public static final Map<String, String> TO_XCONTENT_PARAMS;
|
||||||
|
static {
|
||||||
|
Map<String, String> modifiable = new HashMap<>();
|
||||||
|
modifiable.put("for_internal_storage", "true");
|
||||||
|
TO_XCONTENT_PARAMS = Collections.unmodifiableMap(modifiable);
|
||||||
|
}
|
||||||
|
|
||||||
|
private final Client client;
|
||||||
|
private final NamedXContentRegistry xContentRegistry;
|
||||||
|
|
||||||
|
public DataFrameTransformsConfigManager(Client client, NamedXContentRegistry xContentRegistry) {
|
||||||
|
this.client = client;
|
||||||
|
this.xContentRegistry = xContentRegistry;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void putTransformConfiguration(DataFrameTransformConfig transformConfig, ActionListener<Boolean> listener) {
|
||||||
|
try (XContentBuilder builder = XContentFactory.jsonBuilder()) {
|
||||||
|
XContentBuilder source = transformConfig.toXContent(builder, new ToXContent.MapParams(TO_XCONTENT_PARAMS));
|
||||||
|
|
||||||
|
IndexRequest indexRequest = new IndexRequest(DataFrameInternalIndex.INDEX_NAME)
|
||||||
|
.opType(DocWriteRequest.OpType.CREATE)
|
||||||
|
.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
|
||||||
|
.id(DataFrameTransformConfig.documentId(transformConfig.getId()))
|
||||||
|
.source(source);
|
||||||
|
|
||||||
|
executeAsyncWithOrigin(client, DATA_FRAME_ORIGIN, IndexAction.INSTANCE, indexRequest, ActionListener.wrap(r -> {
|
||||||
|
listener.onResponse(true);
|
||||||
|
}, e -> {
|
||||||
|
if (e instanceof VersionConflictEngineException) {
|
||||||
|
// the transform already exists
|
||||||
|
listener.onFailure(new ResourceAlreadyExistsException(
|
||||||
|
DataFrameMessages.getMessage(DataFrameMessages.REST_PUT_DATA_FRAME_TRANSFORM_EXISTS,
|
||||||
|
transformConfig.getId())));
|
||||||
|
} else {
|
||||||
|
listener.onFailure(
|
||||||
|
new RuntimeException(DataFrameMessages.REST_PUT_DATA_FRAME_FAILED_PERSIST_TRANSFORM_CONFIGURATION, e));
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
} catch (IOException e) {
|
||||||
|
// not expected to happen but for the sake of completeness
|
||||||
|
listener.onFailure(new ElasticsearchParseException(
|
||||||
|
DataFrameMessages.getMessage(DataFrameMessages.REST_DATA_FRAME_FAILED_TO_SERIALIZE_TRANSFORM, transformConfig.getId()),
|
||||||
|
e));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void getTransformConfiguration(String transformId, ActionListener<DataFrameTransformConfig> resultListener) {
|
||||||
|
GetRequest getRequest = new GetRequest(DataFrameInternalIndex.INDEX_NAME, DataFrameTransformConfig.documentId(transformId));
|
||||||
|
executeAsyncWithOrigin(client, DATA_FRAME_ORIGIN, GetAction.INSTANCE, getRequest, ActionListener.wrap(getResponse -> {
|
||||||
|
|
||||||
|
if (getResponse.isExists() == false) {
|
||||||
|
resultListener.onFailure(new ResourceNotFoundException(
|
||||||
|
DataFrameMessages.getMessage(DataFrameMessages.REST_DATA_FRAME_UNKNOWN_TRANSFORM, transformId)));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
BytesReference source = getResponse.getSourceAsBytesRef();
|
||||||
|
parseTransformLenientlyFromSource(source, transformId, resultListener);
|
||||||
|
}, e -> {
|
||||||
|
if (e.getClass() == IndexNotFoundException.class) {
|
||||||
|
resultListener.onFailure(new ResourceNotFoundException(
|
||||||
|
DataFrameMessages.getMessage(DataFrameMessages.REST_DATA_FRAME_UNKNOWN_TRANSFORM, transformId)));
|
||||||
|
} else {
|
||||||
|
resultListener.onFailure(e);
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void deleteTransformConfiguration(String transformId, ActionListener<Boolean> listener) {
|
||||||
|
DeleteRequest request = new DeleteRequest(DataFrameInternalIndex.INDEX_NAME, DataFrameTransformConfig.documentId(transformId));
|
||||||
|
request.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
|
||||||
|
|
||||||
|
executeAsyncWithOrigin(client, DATA_FRAME_ORIGIN, DeleteAction.INSTANCE, request, ActionListener.wrap(deleteResponse -> {
|
||||||
|
|
||||||
|
if (deleteResponse.getResult() == DocWriteResponse.Result.NOT_FOUND) {
|
||||||
|
listener.onFailure(new ResourceNotFoundException(
|
||||||
|
DataFrameMessages.getMessage(DataFrameMessages.REST_DATA_FRAME_UNKNOWN_TRANSFORM, transformId)));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
listener.onResponse(true);
|
||||||
|
}, e -> {
|
||||||
|
if (e.getClass() == IndexNotFoundException.class) {
|
||||||
|
listener.onFailure(new ResourceNotFoundException(
|
||||||
|
DataFrameMessages.getMessage(DataFrameMessages.REST_DATA_FRAME_UNKNOWN_TRANSFORM, transformId)));
|
||||||
|
} else {
|
||||||
|
listener.onFailure(e);
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
}
|
||||||
|
|
||||||
|
private void parseTransformLenientlyFromSource(BytesReference source, String transformId,
|
||||||
|
ActionListener<DataFrameTransformConfig> transformListener) {
|
||||||
|
try (InputStream stream = source.streamInput();
|
||||||
|
XContentParser parser = XContentFactory.xContent(XContentType.JSON)
|
||||||
|
.createParser(xContentRegistry, LoggingDeprecationHandler.INSTANCE, stream)) {
|
||||||
|
transformListener.onResponse(DataFrameTransformConfig.fromXContent(parser, transformId, true));
|
||||||
|
} catch (Exception e) {
|
||||||
|
logger.error(DataFrameMessages.getMessage(DataFrameMessages.FAILED_TO_PARSE_TRANSFORM_CONFIGURATION, transformId), e);
|
||||||
|
transformListener.onFailure(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,89 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.persistence;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.elasticsearch.Version;
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.admin.indices.create.CreateIndexAction;
|
||||||
|
import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
|
||||||
|
import org.elasticsearch.client.Client;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameMessages;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformConfig;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Map.Entry;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||||
|
|
||||||
|
public final class DataframeIndex {
|
||||||
|
private static final Logger logger = LogManager.getLogger(DataframeIndex.class);
|
||||||
|
|
||||||
|
public static final String DOC_TYPE = "_doc";
|
||||||
|
private static final String PROPERTIES = "properties";
|
||||||
|
private static final String TYPE = "type";
|
||||||
|
private static final String META = "_meta";
|
||||||
|
|
||||||
|
private DataframeIndex() {
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void createDestinationIndex(Client client, DataFrameTransformConfig transformConfig, Map<String, String> mappings,
|
||||||
|
final ActionListener<Boolean> listener) {
|
||||||
|
CreateIndexRequest request = new CreateIndexRequest(transformConfig.getDestination());
|
||||||
|
|
||||||
|
// TODO: revisit number of shards, number of replicas
|
||||||
|
request.settings(Settings.builder() // <1>
|
||||||
|
.put("index.number_of_shards", 1).put("index.number_of_replicas", 0));
|
||||||
|
|
||||||
|
request.mapping(DOC_TYPE, createMappingXContent(mappings, transformConfig.getId()));
|
||||||
|
|
||||||
|
client.execute(CreateIndexAction.INSTANCE, request, ActionListener.wrap(createIndexResponse -> {
|
||||||
|
listener.onResponse(true);
|
||||||
|
}, e -> {
|
||||||
|
String message = DataFrameMessages.getMessage(DataFrameMessages.FAILED_TO_CREATE_DESTINATION_INDEX,
|
||||||
|
transformConfig.getDestination(), transformConfig.getId());
|
||||||
|
logger.error(message);
|
||||||
|
listener.onFailure(new RuntimeException(message, e));
|
||||||
|
}));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static XContentBuilder createMappingXContent(Map<String, String> mappings, String id) {
|
||||||
|
try {
|
||||||
|
XContentBuilder builder = jsonBuilder().startObject();
|
||||||
|
builder.startObject(DOC_TYPE);
|
||||||
|
addMetaData(builder, id);
|
||||||
|
builder.startObject(PROPERTIES);
|
||||||
|
for (Entry<String, String> field : mappings.entrySet()) {
|
||||||
|
builder.startObject(field.getKey()).field(TYPE, field.getValue()).endObject();
|
||||||
|
}
|
||||||
|
builder.endObject(); // properties
|
||||||
|
builder.endObject(); // doc_type
|
||||||
|
return builder.endObject();
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static XContentBuilder addMetaData(XContentBuilder builder, String id) throws IOException {
|
||||||
|
builder.startObject(META);
|
||||||
|
builder.field(DataFrameField.CREATED_BY, DataFrameField.DATA_FRAME_SIGNATURE);
|
||||||
|
builder.startObject(DataFrameField.META_FIELDNAME);
|
||||||
|
builder.field(DataFrameField.CREATION_DATE_MILLIS, System.currentTimeMillis());
|
||||||
|
builder.startObject(DataFrameField.VERSION);
|
||||||
|
builder.field(DataFrameField.CREATED, Version.CURRENT);
|
||||||
|
builder.endObject();
|
||||||
|
builder.field(DataFrameField.TRANSFORM, id);
|
||||||
|
builder.endObject(); // META_FIELDNAME
|
||||||
|
builder.endObject(); // META
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,49 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.xpack.dataframe.rest.action;
|
||||||
|
|
||||||
|
|
||||||
|
import org.elasticsearch.client.node.NodeClient;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.rest.BaseRestHandler;
|
||||||
|
import org.elasticsearch.rest.RestController;
|
||||||
|
import org.elasticsearch.rest.RestRequest;
|
||||||
|
import org.elasticsearch.rest.RestStatus;
|
||||||
|
import org.elasticsearch.rest.action.RestToXContentListener;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.DeleteDataFrameTransformAction;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public class RestDeleteDataFrameTransformAction extends BaseRestHandler {
|
||||||
|
|
||||||
|
public RestDeleteDataFrameTransformAction(Settings settings, RestController controller) {
|
||||||
|
super(settings);
|
||||||
|
controller.registerHandler(RestRequest.Method.DELETE, DataFrameField.REST_BASE_PATH_TRANSFORMS_BY_ID, this);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException {
|
||||||
|
String id = restRequest.param(DataFrameField.ID.getPreferredName());
|
||||||
|
DeleteDataFrameTransformAction.Request request = new DeleteDataFrameTransformAction.Request(id);
|
||||||
|
|
||||||
|
return channel -> client.execute(DeleteDataFrameTransformAction.INSTANCE, request,
|
||||||
|
new RestToXContentListener<DeleteDataFrameTransformAction.Response>(channel) {
|
||||||
|
@Override
|
||||||
|
protected RestStatus getStatus(DeleteDataFrameTransformAction.Response response) {
|
||||||
|
if (response.getNodeFailures().size() > 0 || response.getTaskFailures().size() > 0) {
|
||||||
|
return RestStatus.INTERNAL_SERVER_ERROR;
|
||||||
|
}
|
||||||
|
return RestStatus.OK;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getName() {
|
||||||
|
return "data_frame_delete_transform_action";
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,36 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.rest.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.client.node.NodeClient;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.rest.BaseRestHandler;
|
||||||
|
import org.elasticsearch.rest.RestController;
|
||||||
|
import org.elasticsearch.rest.RestRequest;
|
||||||
|
import org.elasticsearch.rest.action.RestToXContentListener;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.GetDataFrameTransformsAction;
|
||||||
|
|
||||||
|
public class RestGetDataFrameTransformsAction extends BaseRestHandler {
|
||||||
|
|
||||||
|
public RestGetDataFrameTransformsAction(Settings settings, RestController controller) {
|
||||||
|
super(settings);
|
||||||
|
controller.registerHandler(RestRequest.Method.GET, DataFrameField.REST_BASE_PATH_TRANSFORMS_BY_ID, this);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) {
|
||||||
|
String id = restRequest.param(DataFrameField.ID.getPreferredName());
|
||||||
|
GetDataFrameTransformsAction.Request request = new GetDataFrameTransformsAction.Request(id);
|
||||||
|
return channel -> client.execute(GetDataFrameTransformsAction.INSTANCE, request, new RestToXContentListener<>(channel));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getName() {
|
||||||
|
return "data_frame_get_transforms_action";
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,36 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.rest.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.client.node.NodeClient;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.rest.BaseRestHandler;
|
||||||
|
import org.elasticsearch.rest.RestController;
|
||||||
|
import org.elasticsearch.rest.RestRequest;
|
||||||
|
import org.elasticsearch.rest.action.RestToXContentListener;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.GetDataFrameTransformsStatsAction;
|
||||||
|
|
||||||
|
public class RestGetDataFrameTransformsStatsAction extends BaseRestHandler {
|
||||||
|
|
||||||
|
public RestGetDataFrameTransformsStatsAction(Settings settings, RestController controller) {
|
||||||
|
super(settings);
|
||||||
|
controller.registerHandler(RestRequest.Method.GET, DataFrameField.REST_BASE_PATH_TRANSFORMS_BY_ID + "_stats", this);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) {
|
||||||
|
String id = restRequest.param(DataFrameField.ID.getPreferredName());
|
||||||
|
GetDataFrameTransformsStatsAction.Request request = new GetDataFrameTransformsStatsAction.Request(id);
|
||||||
|
return channel -> client.execute(GetDataFrameTransformsStatsAction.INSTANCE, request, new RestToXContentListener<>(channel));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getName() {
|
||||||
|
return "data_frame_get_transforms_stats_action";
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,42 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.rest.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.client.node.NodeClient;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.rest.BaseRestHandler;
|
||||||
|
import org.elasticsearch.rest.RestController;
|
||||||
|
import org.elasticsearch.rest.RestRequest;
|
||||||
|
import org.elasticsearch.rest.action.RestToXContentListener;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.PutDataFrameTransformAction;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public class RestPutDataFrameTransformAction extends BaseRestHandler {
|
||||||
|
|
||||||
|
public RestPutDataFrameTransformAction(Settings settings, RestController controller) {
|
||||||
|
super(settings);
|
||||||
|
controller.registerHandler(RestRequest.Method.PUT, DataFrameField.REST_BASE_PATH_TRANSFORMS_BY_ID, this);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getName() {
|
||||||
|
return "data_frame_put_transform_action";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException {
|
||||||
|
String id = restRequest.param(DataFrameField.ID.getPreferredName());
|
||||||
|
XContentParser parser = restRequest.contentParser();
|
||||||
|
|
||||||
|
PutDataFrameTransformAction.Request request = PutDataFrameTransformAction.Request.fromXContent(parser, id);
|
||||||
|
|
||||||
|
return channel -> client.execute(PutDataFrameTransformAction.INSTANCE, request, new RestToXContentListener<>(channel));
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,40 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.rest.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.client.node.NodeClient;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.rest.BaseRestHandler;
|
||||||
|
import org.elasticsearch.rest.RestController;
|
||||||
|
import org.elasticsearch.rest.RestRequest;
|
||||||
|
import org.elasticsearch.rest.action.RestToXContentListener;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.core.rollup.RollupField;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.StartDataFrameTransformAction;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public class RestStartDataFrameTransformAction extends BaseRestHandler {
|
||||||
|
|
||||||
|
public RestStartDataFrameTransformAction(Settings settings, RestController controller) {
|
||||||
|
super(settings);
|
||||||
|
controller.registerHandler(RestRequest.Method.POST, DataFrameField.REST_BASE_PATH_TRANSFORMS_BY_ID + "_start", this);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException {
|
||||||
|
String id = restRequest.param(RollupField.ID.getPreferredName());
|
||||||
|
StartDataFrameTransformAction.Request request = new StartDataFrameTransformAction.Request(id);
|
||||||
|
|
||||||
|
return channel -> client.execute(StartDataFrameTransformAction.INSTANCE, request, new RestToXContentListener<>(channel));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getName() {
|
||||||
|
return "data_frame_start_transform_action";
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.xpack.dataframe.rest.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.client.node.NodeClient;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
|
import org.elasticsearch.rest.BaseRestHandler;
|
||||||
|
import org.elasticsearch.rest.RestController;
|
||||||
|
import org.elasticsearch.rest.RestRequest;
|
||||||
|
import org.elasticsearch.rest.action.RestToXContentListener;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.StopDataFrameTransformAction;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public class RestStopDataFrameTransformAction extends BaseRestHandler {
|
||||||
|
|
||||||
|
public RestStopDataFrameTransformAction(Settings settings, RestController controller) {
|
||||||
|
super(settings);
|
||||||
|
controller.registerHandler(RestRequest.Method.POST, DataFrameField.REST_BASE_PATH_TRANSFORMS_BY_ID + "_stop", this);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException {
|
||||||
|
String id = restRequest.param(DataFrameField.ID.getPreferredName());
|
||||||
|
TimeValue timeout = restRequest.paramAsTime(DataFrameField.TIMEOUT.getPreferredName(),
|
||||||
|
StopDataFrameTransformAction.DEFAULT_TIMEOUT);
|
||||||
|
boolean waitForCompletion = restRequest.paramAsBoolean(DataFrameField.WAIT_FOR_COMPLETION.getPreferredName(), false);
|
||||||
|
|
||||||
|
StopDataFrameTransformAction.Request request = new StopDataFrameTransformAction.Request(id, waitForCompletion, timeout);
|
||||||
|
|
||||||
|
return channel -> client.execute(StopDataFrameTransformAction.INSTANCE, request, new RestToXContentListener<>(channel));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getName() {
|
||||||
|
return "data_frame_stop_transform_action";
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,90 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.elasticsearch.action.index.IndexRequest;
|
||||||
|
import org.elasticsearch.action.search.SearchRequest;
|
||||||
|
import org.elasticsearch.action.search.SearchResponse;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.index.query.QueryBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameIndexerTransformStats;
|
||||||
|
import org.elasticsearch.xpack.core.indexing.AsyncTwoPhaseIndexer;
|
||||||
|
import org.elasticsearch.xpack.core.indexing.IndexerState;
|
||||||
|
import org.elasticsearch.xpack.core.indexing.IterationResult;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.pivot.Pivot;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.UncheckedIOException;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.Executor;
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||||
|
|
||||||
|
public abstract class DataFrameIndexer extends AsyncTwoPhaseIndexer<Map<String, Object>, DataFrameIndexerTransformStats> {
|
||||||
|
|
||||||
|
private static final String COMPOSITE_AGGREGATION_NAME = "_data_frame";
|
||||||
|
private static final Logger logger = LogManager.getLogger(DataFrameIndexer.class);
|
||||||
|
|
||||||
|
private Pivot pivot;
|
||||||
|
|
||||||
|
public DataFrameIndexer(Executor executor, AtomicReference<IndexerState> initialState, Map<String, Object> initialPosition) {
|
||||||
|
super(executor, initialState, initialPosition, new DataFrameIndexerTransformStats());
|
||||||
|
}
|
||||||
|
|
||||||
|
protected abstract DataFrameTransformConfig getConfig();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void onStartJob(long now) {
|
||||||
|
QueryBuilder queryBuilder = getConfig().getQueryConfig().getQuery();
|
||||||
|
|
||||||
|
pivot = new Pivot(getConfig().getSource(), queryBuilder, getConfig().getPivotConfig());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected IterationResult<Map<String, Object>> doProcess(SearchResponse searchResponse) {
|
||||||
|
final CompositeAggregation agg = searchResponse.getAggregations().get(COMPOSITE_AGGREGATION_NAME);
|
||||||
|
return new IterationResult<>(processBucketsToIndexRequests(agg).collect(Collectors.toList()), agg.afterKey(),
|
||||||
|
agg.getBuckets().isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Parses the result and creates a stream of indexable documents
|
||||||
|
*
|
||||||
|
* Implementation decisions:
|
||||||
|
*
|
||||||
|
* Extraction uses generic maps as intermediate exchange format in order to hook in ingest pipelines/processors
|
||||||
|
* in later versions, see {@link IngestDocument).
|
||||||
|
*/
|
||||||
|
private Stream<IndexRequest> processBucketsToIndexRequests(CompositeAggregation agg) {
|
||||||
|
final DataFrameTransformConfig transformConfig = getConfig();
|
||||||
|
String indexName = transformConfig.getDestination();
|
||||||
|
|
||||||
|
return pivot.extractResults(agg, getStats()).map(document -> {
|
||||||
|
XContentBuilder builder;
|
||||||
|
try {
|
||||||
|
builder = jsonBuilder();
|
||||||
|
builder.map(document);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new UncheckedIOException(e);
|
||||||
|
}
|
||||||
|
|
||||||
|
IndexRequest request = new IndexRequest(indexName).source(builder);
|
||||||
|
return request;
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected SearchRequest buildSearchRequest() {
|
||||||
|
return pivot.buildSearchRequest(getPosition());
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,100 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms;
|
||||||
|
|
||||||
|
import org.elasticsearch.Version;
|
||||||
|
import org.elasticsearch.cluster.AbstractDiffable;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.xpack.core.XPackPlugin;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
public class DataFrameTransform extends AbstractDiffable<DataFrameTransform> implements XPackPlugin.XPackPersistentTaskParams {
|
||||||
|
|
||||||
|
public static final String NAME = DataFrameField.TASK_NAME;
|
||||||
|
|
||||||
|
private final String transformId;
|
||||||
|
|
||||||
|
public static final ConstructingObjectParser<DataFrameTransform, Void> PARSER = new ConstructingObjectParser<>(NAME,
|
||||||
|
a -> new DataFrameTransform((String) a[0]));
|
||||||
|
|
||||||
|
static {
|
||||||
|
PARSER.declareString(ConstructingObjectParser.constructorArg(), DataFrameField.ID);
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameTransform(String transformId) {
|
||||||
|
this.transformId = transformId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameTransform(StreamInput in) throws IOException {
|
||||||
|
this.transformId = in.readString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getWriteableName() {
|
||||||
|
return NAME;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Version getMinimalSupportedVersion() {
|
||||||
|
// TODO: to be changed once target version has been defined
|
||||||
|
return Version.CURRENT;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeString(transformId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
builder.field(DataFrameField.ID.getPreferredName(), transformId);
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getId() {
|
||||||
|
return transformId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DataFrameTransform fromXContent(XContentParser parser) throws IOException {
|
||||||
|
return PARSER.parse(parser, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object other) {
|
||||||
|
if (this == other) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (other == null || getClass() != other.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
DataFrameTransform that = (DataFrameTransform) other;
|
||||||
|
|
||||||
|
return Objects.equals(this.transformId, that.transformId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(transformId);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Map<String, String> getHeaders() {
|
||||||
|
return Collections.emptyMap();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,210 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms;
|
||||||
|
|
||||||
|
import org.elasticsearch.cluster.AbstractDiffable;
|
||||||
|
import org.elasticsearch.common.Nullable;
|
||||||
|
import org.elasticsearch.common.ParseField;
|
||||||
|
import org.elasticsearch.common.Strings;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.index.query.MatchAllQueryBuilder;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameMessages;
|
||||||
|
import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.pivot.PivotConfig;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg;
|
||||||
|
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This class holds the configuration details of a data frame transform
|
||||||
|
*/
|
||||||
|
public class DataFrameTransformConfig extends AbstractDiffable<DataFrameTransformConfig> implements Writeable, ToXContentObject {
|
||||||
|
|
||||||
|
private static final String NAME = "data_frame_transforms";
|
||||||
|
private static final ParseField SOURCE = new ParseField("source");
|
||||||
|
private static final ParseField DESTINATION = new ParseField("dest");
|
||||||
|
private static final ParseField QUERY = new ParseField("query");
|
||||||
|
|
||||||
|
// types of transforms
|
||||||
|
private static final ParseField PIVOT_TRANSFORM = new ParseField("pivot");
|
||||||
|
|
||||||
|
private static final ConstructingObjectParser<DataFrameTransformConfig, String> STRICT_PARSER = createParser(false);
|
||||||
|
private static final ConstructingObjectParser<DataFrameTransformConfig, String> LENIENT_PARSER = createParser(true);
|
||||||
|
|
||||||
|
private final String id;
|
||||||
|
private final String source;
|
||||||
|
private final String dest;
|
||||||
|
|
||||||
|
private final QueryConfig queryConfig;
|
||||||
|
private final PivotConfig pivotConfig;
|
||||||
|
|
||||||
|
private static ConstructingObjectParser<DataFrameTransformConfig, String> createParser(boolean lenient) {
|
||||||
|
ConstructingObjectParser<DataFrameTransformConfig, String> parser = new ConstructingObjectParser<>(NAME, lenient,
|
||||||
|
(args, optionalId) -> {
|
||||||
|
String id = args[0] != null ? (String) args[0] : optionalId;
|
||||||
|
String source = (String) args[1];
|
||||||
|
String dest = (String) args[2];
|
||||||
|
|
||||||
|
// default handling: if the user does not specify a query, we default to match_all
|
||||||
|
QueryConfig queryConfig = null;
|
||||||
|
if (args[3] == null) {
|
||||||
|
queryConfig = new QueryConfig(Collections.singletonMap(MatchAllQueryBuilder.NAME, Collections.emptyMap()),
|
||||||
|
new MatchAllQueryBuilder());
|
||||||
|
} else {
|
||||||
|
queryConfig = (QueryConfig) args[3];
|
||||||
|
}
|
||||||
|
|
||||||
|
PivotConfig pivotConfig = (PivotConfig) args[4];
|
||||||
|
return new DataFrameTransformConfig(id, source, dest, queryConfig, pivotConfig);
|
||||||
|
});
|
||||||
|
|
||||||
|
parser.declareString(optionalConstructorArg(), DataFrameField.ID);
|
||||||
|
parser.declareString(constructorArg(), SOURCE);
|
||||||
|
parser.declareString(constructorArg(), DESTINATION);
|
||||||
|
parser.declareObject(optionalConstructorArg(), (p, c) -> QueryConfig.fromXContent(p, lenient), QUERY);
|
||||||
|
parser.declareObject(optionalConstructorArg(), (p, c) -> PivotConfig.fromXContent(p, lenient), PIVOT_TRANSFORM);
|
||||||
|
|
||||||
|
return parser;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static String documentId(String transformId) {
|
||||||
|
return "data_frame-" + transformId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameTransformConfig(final String id,
|
||||||
|
final String source,
|
||||||
|
final String dest,
|
||||||
|
final QueryConfig queryConfig,
|
||||||
|
final PivotConfig pivotConfig) {
|
||||||
|
this.id = ExceptionsHelper.requireNonNull(id, DataFrameField.ID.getPreferredName());
|
||||||
|
this.source = ExceptionsHelper.requireNonNull(source, SOURCE.getPreferredName());
|
||||||
|
this.dest = ExceptionsHelper.requireNonNull(dest, DESTINATION.getPreferredName());
|
||||||
|
this.queryConfig = ExceptionsHelper.requireNonNull(queryConfig, QUERY.getPreferredName());
|
||||||
|
this.pivotConfig = pivotConfig;
|
||||||
|
|
||||||
|
// at least one function must be defined
|
||||||
|
if (this.pivotConfig == null) {
|
||||||
|
throw new IllegalArgumentException(DataFrameMessages.DATA_FRAME_TRANSFORM_CONFIGURATION_NO_TRANSFORM);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameTransformConfig(final StreamInput in) throws IOException {
|
||||||
|
id = in.readString();
|
||||||
|
source = in.readString();
|
||||||
|
dest = in.readString();
|
||||||
|
queryConfig = in.readOptionalWriteable(QueryConfig::new);
|
||||||
|
pivotConfig = in.readOptionalWriteable(PivotConfig::new);
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getId() {
|
||||||
|
return id;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getCron() {
|
||||||
|
return "*";
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getSource() {
|
||||||
|
return source;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getDestination() {
|
||||||
|
return dest;
|
||||||
|
}
|
||||||
|
|
||||||
|
public PivotConfig getPivotConfig() {
|
||||||
|
return pivotConfig;
|
||||||
|
}
|
||||||
|
|
||||||
|
public QueryConfig getQueryConfig() {
|
||||||
|
return queryConfig;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isValid() {
|
||||||
|
// collect validation results from all child objects
|
||||||
|
if (queryConfig != null && queryConfig.isValid() == false) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (pivotConfig != null && pivotConfig.isValid() == false) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(final StreamOutput out) throws IOException {
|
||||||
|
out.writeString(id);
|
||||||
|
out.writeString(source);
|
||||||
|
out.writeString(dest);
|
||||||
|
out.writeOptionalWriteable(queryConfig);
|
||||||
|
out.writeOptionalWriteable(pivotConfig);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(final XContentBuilder builder, final Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
builder.field(DataFrameField.ID.getPreferredName(), id);
|
||||||
|
builder.field(SOURCE.getPreferredName(), source);
|
||||||
|
builder.field(DESTINATION.getPreferredName(), dest);
|
||||||
|
if (queryConfig != null) {
|
||||||
|
builder.field(QUERY.getPreferredName(), queryConfig);
|
||||||
|
}
|
||||||
|
if (pivotConfig != null) {
|
||||||
|
builder.field(PIVOT_TRANSFORM.getPreferredName(), pivotConfig);
|
||||||
|
}
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object other) {
|
||||||
|
if (this == other) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (other == null || getClass() != other.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
final DataFrameTransformConfig that = (DataFrameTransformConfig) other;
|
||||||
|
|
||||||
|
return Objects.equals(this.id, that.id)
|
||||||
|
&& Objects.equals(this.source, that.source)
|
||||||
|
&& Objects.equals(this.dest, that.dest)
|
||||||
|
&& Objects.equals(this.queryConfig, that.queryConfig)
|
||||||
|
&& Objects.equals(this.pivotConfig, that.pivotConfig);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(id, source, dest, queryConfig, pivotConfig);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return Strings.toString(this, true, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DataFrameTransformConfig fromXContent(final XContentParser parser, @Nullable final String optionalTransformId,
|
||||||
|
boolean lenient) throws IOException {
|
||||||
|
|
||||||
|
return lenient ? LENIENT_PARSER.apply(parser, optionalTransformId) : STRICT_PARSER.apply(parser, optionalTransformId);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,71 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.elasticsearch.client.Client;
|
||||||
|
import org.elasticsearch.common.Nullable;
|
||||||
|
import org.elasticsearch.persistent.AllocatedPersistentTask;
|
||||||
|
import org.elasticsearch.persistent.PersistentTaskState;
|
||||||
|
import org.elasticsearch.persistent.PersistentTasksCustomMetaData;
|
||||||
|
import org.elasticsearch.persistent.PersistentTasksExecutor;
|
||||||
|
import org.elasticsearch.tasks.TaskId;
|
||||||
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameTransformState;
|
||||||
|
import org.elasticsearch.xpack.core.scheduler.SchedulerEngine;
|
||||||
|
import org.elasticsearch.xpack.dataframe.DataFrame;
|
||||||
|
import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class DataFrameTransformPersistentTasksExecutor extends PersistentTasksExecutor<DataFrameTransform> {
|
||||||
|
|
||||||
|
private static final Logger logger = LogManager.getLogger(DataFrameTransformPersistentTasksExecutor.class);
|
||||||
|
|
||||||
|
private final Client client;
|
||||||
|
private final DataFrameTransformsConfigManager transformsConfigManager;
|
||||||
|
private final SchedulerEngine schedulerEngine;
|
||||||
|
private final ThreadPool threadPool;
|
||||||
|
|
||||||
|
public DataFrameTransformPersistentTasksExecutor(Client client, DataFrameTransformsConfigManager transformsConfigManager,
|
||||||
|
SchedulerEngine schedulerEngine, ThreadPool threadPool) {
|
||||||
|
super(DataFrameField.TASK_NAME, DataFrame.TASK_THREAD_POOL_NAME);
|
||||||
|
this.client = client;
|
||||||
|
this.transformsConfigManager = transformsConfigManager;
|
||||||
|
this.schedulerEngine = schedulerEngine;
|
||||||
|
this.threadPool = threadPool;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void nodeOperation(AllocatedPersistentTask task, @Nullable DataFrameTransform params, PersistentTaskState state) {
|
||||||
|
DataFrameTransformTask buildTask = (DataFrameTransformTask) task;
|
||||||
|
SchedulerEngine.Job schedulerJob = new SchedulerEngine.Job(
|
||||||
|
DataFrameTransformTask.SCHEDULE_NAME + "_" + params.getId(), next());
|
||||||
|
|
||||||
|
// Note that while the task is added to the scheduler here, the internal state will prevent
|
||||||
|
// it from doing any work until the task is "started" via the StartTransform api
|
||||||
|
schedulerEngine.register(buildTask);
|
||||||
|
schedulerEngine.add(schedulerJob);
|
||||||
|
|
||||||
|
logger.info("Data frame transform [" + params.getId() + "] created.");
|
||||||
|
}
|
||||||
|
|
||||||
|
static SchedulerEngine.Schedule next() {
|
||||||
|
return (startTime, now) -> {
|
||||||
|
return now + 1000; // to be fixed, hardcode something
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected AllocatedPersistentTask createTask(long id, String type, String action, TaskId parentTaskId,
|
||||||
|
PersistentTasksCustomMetaData.PersistentTask<DataFrameTransform> persistentTask, Map<String, String> headers) {
|
||||||
|
return new DataFrameTransformTask(id, type, action, parentTaskId, persistentTask.getParams(),
|
||||||
|
(DataFrameTransformState) persistentTask.getState(), client, transformsConfigManager, schedulerEngine, threadPool, headers);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,328 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.elasticsearch.ElasticsearchException;
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.LatchedActionListener;
|
||||||
|
import org.elasticsearch.action.bulk.BulkAction;
|
||||||
|
import org.elasticsearch.action.bulk.BulkRequest;
|
||||||
|
import org.elasticsearch.action.bulk.BulkResponse;
|
||||||
|
import org.elasticsearch.action.search.SearchAction;
|
||||||
|
import org.elasticsearch.action.search.SearchRequest;
|
||||||
|
import org.elasticsearch.action.search.SearchResponse;
|
||||||
|
import org.elasticsearch.client.Client;
|
||||||
|
import org.elasticsearch.persistent.AllocatedPersistentTask;
|
||||||
|
import org.elasticsearch.tasks.TaskId;
|
||||||
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
import org.elasticsearch.xpack.core.ClientHelper;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameMessages;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameIndexerTransformStats;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameTransformState;
|
||||||
|
import org.elasticsearch.xpack.core.indexing.IndexerState;
|
||||||
|
import org.elasticsearch.xpack.core.scheduler.SchedulerEngine;
|
||||||
|
import org.elasticsearch.xpack.core.scheduler.SchedulerEngine.Event;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.StartDataFrameTransformAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.StartDataFrameTransformAction.Response;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.StopDataFrameTransformAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
|
public class DataFrameTransformTask extends AllocatedPersistentTask implements SchedulerEngine.Listener {
|
||||||
|
|
||||||
|
private static final Logger logger = LogManager.getLogger(DataFrameTransformTask.class);
|
||||||
|
public static final String SCHEDULE_NAME = DataFrameField.TASK_NAME + "/schedule";
|
||||||
|
|
||||||
|
private final DataFrameTransform transform;
|
||||||
|
private final SchedulerEngine schedulerEngine;
|
||||||
|
private final ThreadPool threadPool;
|
||||||
|
private final DataFrameIndexer indexer;
|
||||||
|
|
||||||
|
// the generation of this data frame, for v1 there will be only
|
||||||
|
// 0: data frame not created or still indexing
|
||||||
|
// 1: data frame complete, all data has been indexed
|
||||||
|
private final AtomicReference<Long> generation;
|
||||||
|
|
||||||
|
public DataFrameTransformTask(long id, String type, String action, TaskId parentTask, DataFrameTransform transform,
|
||||||
|
DataFrameTransformState state, Client client, DataFrameTransformsConfigManager transformsConfigManager,
|
||||||
|
SchedulerEngine schedulerEngine, ThreadPool threadPool, Map<String, String> headers) {
|
||||||
|
super(id, type, action, DataFrameField.PERSISTENT_TASK_DESCRIPTION_PREFIX + transform.getId(), parentTask, headers);
|
||||||
|
this.transform = transform;
|
||||||
|
this.schedulerEngine = schedulerEngine;
|
||||||
|
this.threadPool = threadPool;
|
||||||
|
IndexerState initialState = IndexerState.STOPPED;
|
||||||
|
long initialGeneration = 0;
|
||||||
|
Map<String, Object> initialPosition = null;
|
||||||
|
logger.info("[{}] init, got state: [{}]", transform.getId(), state != null);
|
||||||
|
if (state != null) {
|
||||||
|
final IndexerState existingState = state.getIndexerState();
|
||||||
|
logger.info("[{}] Loading existing state: [{}], position [{}]", transform.getId(), existingState, state.getPosition());
|
||||||
|
if (existingState.equals(IndexerState.INDEXING)) {
|
||||||
|
// reset to started as no indexer is running
|
||||||
|
initialState = IndexerState.STARTED;
|
||||||
|
} else if (existingState.equals(IndexerState.ABORTING) || existingState.equals(IndexerState.STOPPING)) {
|
||||||
|
// reset to stopped as something bad happened
|
||||||
|
initialState = IndexerState.STOPPED;
|
||||||
|
} else {
|
||||||
|
initialState = existingState;
|
||||||
|
}
|
||||||
|
initialPosition = state.getPosition();
|
||||||
|
initialGeneration = state.getGeneration();
|
||||||
|
}
|
||||||
|
|
||||||
|
this.indexer = new ClientDataFrameIndexer(transform.getId(), transformsConfigManager, new AtomicReference<>(initialState),
|
||||||
|
initialPosition, client);
|
||||||
|
this.generation = new AtomicReference<Long>(initialGeneration);
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getTransformId() {
|
||||||
|
return transform.getId();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Enable Task API to return detailed status information
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public Status getStatus() {
|
||||||
|
return getState();
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameTransformState getState() {
|
||||||
|
return new DataFrameTransformState(indexer.getState(), indexer.getPosition(), generation.get());
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataFrameIndexerTransformStats getStats() {
|
||||||
|
return indexer.getStats();
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getGeneration() {
|
||||||
|
return generation.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isStopped() {
|
||||||
|
return indexer.getState().equals(IndexerState.STOPPED);
|
||||||
|
}
|
||||||
|
|
||||||
|
public synchronized void start(ActionListener<Response> listener) {
|
||||||
|
final IndexerState prevState = indexer.getState();
|
||||||
|
if (prevState != IndexerState.STOPPED) {
|
||||||
|
// fails if the task is not STOPPED
|
||||||
|
listener.onFailure(new ElasticsearchException("Cannot start task for data frame transform [{}], because state was [{}]",
|
||||||
|
transform.getId(), prevState));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
final IndexerState newState = indexer.start();
|
||||||
|
if (newState != IndexerState.STARTED) {
|
||||||
|
listener.onFailure(new ElasticsearchException("Cannot start task for data frame transform [{}], because state was [{}]",
|
||||||
|
transform.getId(), newState));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
final DataFrameTransformState state = new DataFrameTransformState(IndexerState.STOPPED, indexer.getPosition(), generation.get());
|
||||||
|
|
||||||
|
logger.debug("Updating state for data frame transform [{}] to [{}][{}]", transform.getId(), state.getIndexerState(),
|
||||||
|
state.getPosition());
|
||||||
|
updatePersistentTaskState(state,
|
||||||
|
ActionListener.wrap(
|
||||||
|
(task) -> {
|
||||||
|
logger.debug("Successfully updated state for data frame transform [" + transform.getId() + "] to ["
|
||||||
|
+ state.getIndexerState() + "][" + state.getPosition() + "]");
|
||||||
|
listener.onResponse(new StartDataFrameTransformAction.Response(true));
|
||||||
|
}, (exc) -> {
|
||||||
|
// We were unable to update the persistent status, so we need to shutdown the indexer too.
|
||||||
|
indexer.stop();
|
||||||
|
listener.onFailure(new ElasticsearchException("Error while updating state for data frame transform ["
|
||||||
|
+ transform.getId() + "] to [" + state.getIndexerState() + "].", exc));
|
||||||
|
})
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
public synchronized void stop(ActionListener<StopDataFrameTransformAction.Response> listener) {
|
||||||
|
final IndexerState newState = indexer.stop();
|
||||||
|
switch (newState) {
|
||||||
|
case STOPPED:
|
||||||
|
listener.onResponse(new StopDataFrameTransformAction.Response(true));
|
||||||
|
break;
|
||||||
|
|
||||||
|
case STOPPING:
|
||||||
|
// update the persistent state to STOPPED. There are two scenarios and both are safe:
|
||||||
|
// 1. we persist STOPPED now, indexer continues a bit then sees the flag and checkpoints another STOPPED with the more recent
|
||||||
|
// position.
|
||||||
|
// 2. we persist STOPPED now, indexer continues a bit but then dies. When/if we resume we'll pick up at last checkpoint,
|
||||||
|
// overwrite some docs and eventually checkpoint.
|
||||||
|
DataFrameTransformState state = new DataFrameTransformState(IndexerState.STOPPED, indexer.getPosition(), generation.get());
|
||||||
|
updatePersistentTaskState(state, ActionListener.wrap((task) -> {
|
||||||
|
logger.debug("Successfully updated state for data frame transform [{}] to [{}]", transform.getId(),
|
||||||
|
state.getIndexerState());
|
||||||
|
listener.onResponse(new StopDataFrameTransformAction.Response(true));
|
||||||
|
}, (exc) -> {
|
||||||
|
listener.onFailure(new ElasticsearchException("Error while updating state for data frame transform [{}] to [{}]", exc,
|
||||||
|
transform.getId(), state.getIndexerState()));
|
||||||
|
}));
|
||||||
|
break;
|
||||||
|
|
||||||
|
default:
|
||||||
|
listener.onFailure(new ElasticsearchException("Cannot stop task for data frame transform [{}], because state was [{}]",
|
||||||
|
transform.getId(), newState));
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized void triggered(Event event) {
|
||||||
|
if (generation.get() == 0 && event.getJobName().equals(SCHEDULE_NAME + "_" + transform.getId())) {
|
||||||
|
logger.debug("Data frame indexer [" + event.getJobName() + "] schedule has triggered, state: [" + indexer.getState() + "]");
|
||||||
|
indexer.maybeTriggerAsyncJob(System.currentTimeMillis());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Attempt to gracefully cleanup the data frame transform so it can be terminated.
|
||||||
|
* This tries to remove the job from the scheduler, and potentially any other
|
||||||
|
* cleanup operations in the future
|
||||||
|
*/
|
||||||
|
synchronized void shutdown() {
|
||||||
|
try {
|
||||||
|
logger.info("Data frame indexer [" + transform.getId() + "] received abort request, stopping indexer.");
|
||||||
|
schedulerEngine.remove(SCHEDULE_NAME + "_" + transform.getId());
|
||||||
|
schedulerEngine.unregister(this);
|
||||||
|
} catch (Exception e) {
|
||||||
|
markAsFailed(e);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
markAsCompleted();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This is called when the persistent task signals that the allocated task should be terminated.
|
||||||
|
* Termination in the task framework is essentially voluntary, as the allocated task can only be
|
||||||
|
* shut down from the inside.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public synchronized void onCancelled() {
|
||||||
|
logger.info(
|
||||||
|
"Received cancellation request for data frame transform [" + transform.getId() + "], state: [" + indexer.getState() + "]");
|
||||||
|
if (indexer.abort()) {
|
||||||
|
// there is no background transform running, we can shutdown safely
|
||||||
|
shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected class ClientDataFrameIndexer extends DataFrameIndexer {
|
||||||
|
private static final int LOAD_TRANSFORM_TIMEOUT_IN_SECONDS = 30;
|
||||||
|
private final Client client;
|
||||||
|
private final DataFrameTransformsConfigManager transformsConfigManager;
|
||||||
|
private final String transformId;
|
||||||
|
|
||||||
|
private DataFrameTransformConfig transformConfig = null;
|
||||||
|
|
||||||
|
public ClientDataFrameIndexer(String transformId, DataFrameTransformsConfigManager transformsConfigManager,
|
||||||
|
AtomicReference<IndexerState> initialState, Map<String, Object> initialPosition, Client client) {
|
||||||
|
super(threadPool.executor(ThreadPool.Names.GENERIC), initialState, initialPosition);
|
||||||
|
this.transformId = transformId;
|
||||||
|
this.transformsConfigManager = transformsConfigManager;
|
||||||
|
this.client = client;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected DataFrameTransformConfig getConfig() {
|
||||||
|
return transformConfig;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getJobId() {
|
||||||
|
return transformId;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized boolean maybeTriggerAsyncJob(long now) {
|
||||||
|
if (transformConfig == null) {
|
||||||
|
CountDownLatch latch = new CountDownLatch(1);
|
||||||
|
|
||||||
|
transformsConfigManager.getTransformConfiguration(transformId, new LatchedActionListener<>(ActionListener.wrap(config -> {
|
||||||
|
transformConfig = config;
|
||||||
|
}, e -> {
|
||||||
|
throw new RuntimeException(
|
||||||
|
DataFrameMessages.getMessage(DataFrameMessages.FAILED_TO_LOAD_TRANSFORM_CONFIGURATION, transformId), e);
|
||||||
|
}), latch));
|
||||||
|
|
||||||
|
try {
|
||||||
|
latch.await(LOAD_TRANSFORM_TIMEOUT_IN_SECONDS, TimeUnit.SECONDS);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
throw new RuntimeException(
|
||||||
|
DataFrameMessages.getMessage(DataFrameMessages.FAILED_TO_LOAD_TRANSFORM_CONFIGURATION, transformId), e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// todo: set job into failed state
|
||||||
|
if (transformConfig.isValid() == false) {
|
||||||
|
throw new RuntimeException(
|
||||||
|
DataFrameMessages.getMessage(DataFrameMessages.DATA_FRAME_TRANSFORM_CONFIGURATION_INVALID, transformId));
|
||||||
|
}
|
||||||
|
|
||||||
|
return super.maybeTriggerAsyncJob(now);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void doNextSearch(SearchRequest request, ActionListener<SearchResponse> nextPhase) {
|
||||||
|
ClientHelper.executeWithHeadersAsync(transform.getHeaders(), ClientHelper.DATA_FRAME_ORIGIN, client, SearchAction.INSTANCE,
|
||||||
|
request, nextPhase);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void doNextBulk(BulkRequest request, ActionListener<BulkResponse> nextPhase) {
|
||||||
|
ClientHelper.executeWithHeadersAsync(transform.getHeaders(), ClientHelper.DATA_FRAME_ORIGIN, client, BulkAction.INSTANCE,
|
||||||
|
request, nextPhase);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void doSaveState(IndexerState indexerState, Map<String, Object> position, Runnable next) {
|
||||||
|
if (indexerState.equals(IndexerState.ABORTING)) {
|
||||||
|
// If we're aborting, just invoke `next` (which is likely an onFailure handler)
|
||||||
|
next.run();
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if(indexerState.equals(IndexerState.STARTED)) {
|
||||||
|
// if the indexer resets the state to started, it means it is done, so increment the generation
|
||||||
|
generation.compareAndSet(0L, 1L);
|
||||||
|
}
|
||||||
|
|
||||||
|
final DataFrameTransformState state = new DataFrameTransformState(indexerState, getPosition(), generation.get());
|
||||||
|
logger.info("Updating persistent state of transform [" + transform.getId() + "] to [" + state.toString() + "]");
|
||||||
|
|
||||||
|
updatePersistentTaskState(state, ActionListener.wrap(task -> next.run(), exc -> {
|
||||||
|
logger.error("Updating persistent state of transform [" + transform.getId() + "] failed", exc);
|
||||||
|
next.run();
|
||||||
|
}));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void onFailure(Exception exc) {
|
||||||
|
logger.warn("Data frame transform [" + transform.getId() + "] failed with an exception: ", exc);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void onFinish() {
|
||||||
|
logger.info("Finished indexing for data frame transform [" + transform.getId() + "]");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void onAbort() {
|
||||||
|
logger.info("Data frame transform [" + transform.getId() + "] received abort request, stopping indexer");
|
||||||
|
shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,113 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.elasticsearch.cluster.AbstractDiffable;
|
||||||
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.common.xcontent.LoggingDeprecationHandler;
|
||||||
|
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.index.query.AbstractQueryBuilder;
|
||||||
|
import org.elasticsearch.index.query.QueryBuilder;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameMessages;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
public class QueryConfig extends AbstractDiffable<QueryConfig> implements Writeable, ToXContentObject {
|
||||||
|
private static final Logger logger = LogManager.getLogger(QueryConfig.class);
|
||||||
|
|
||||||
|
// we store the query in 2 formats: the raw format and the parsed format, because:
|
||||||
|
// - the parsed format adds defaults, which were not part of the original and looks odd on XContent retrieval
|
||||||
|
// - if parsing fails (e.g. query uses removed functionality), the source can be retrieved
|
||||||
|
private final Map<String, Object> source;
|
||||||
|
private final QueryBuilder query;
|
||||||
|
|
||||||
|
public QueryConfig(final Map<String, Object> source, final QueryBuilder query) {
|
||||||
|
this.source = Objects.requireNonNull(source);
|
||||||
|
this.query = query;
|
||||||
|
}
|
||||||
|
|
||||||
|
public QueryConfig(final StreamInput in) throws IOException {
|
||||||
|
this.source = in.readMap();
|
||||||
|
this.query = in.readOptionalNamedWriteable(QueryBuilder.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.map(source);
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeMap(source);
|
||||||
|
out.writeOptionalNamedWriteable(query);
|
||||||
|
}
|
||||||
|
|
||||||
|
public QueryBuilder getQuery() {
|
||||||
|
return query;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static QueryConfig fromXContent(final XContentParser parser, boolean lenient) throws IOException {
|
||||||
|
// we need 2 passes, but the parser can not be cloned, so we parse 1st into a map and then re-parse that for syntax checking
|
||||||
|
|
||||||
|
// remember the registry, needed for the 2nd pass
|
||||||
|
NamedXContentRegistry registry = parser.getXContentRegistry();
|
||||||
|
|
||||||
|
Map<String, Object> source = parser.mapOrdered();
|
||||||
|
QueryBuilder query = null;
|
||||||
|
|
||||||
|
try (XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().map(source);
|
||||||
|
XContentParser sourceParser = XContentType.JSON.xContent().createParser(registry, LoggingDeprecationHandler.INSTANCE,
|
||||||
|
BytesReference.bytes(xContentBuilder).streamInput())) {
|
||||||
|
query = AbstractQueryBuilder.parseInnerQueryBuilder(sourceParser);
|
||||||
|
} catch (Exception e) {
|
||||||
|
if (lenient) {
|
||||||
|
logger.warn(DataFrameMessages.LOG_DATA_FRAME_TRANSFORM_CONFIGURATION_BAD_QUERY, e);
|
||||||
|
} else {
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return new QueryConfig(source, query);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(source, query);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object other) {
|
||||||
|
if (this == other) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (other == null || getClass() != other.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
final QueryConfig that = (QueryConfig) other;
|
||||||
|
|
||||||
|
return Objects.equals(this.source, that.source) && Objects.equals(this.query, that.query);
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isValid() {
|
||||||
|
return this.query != null;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,120 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.common.xcontent.LoggingDeprecationHandler;
|
||||||
|
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameMessages;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Wrapper for the aggregations config part of a composite aggregation.
|
||||||
|
*
|
||||||
|
* For now just wraps aggregations from composite aggs.
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class AggregationConfig implements Writeable, ToXContentObject {
|
||||||
|
private static final Logger logger = LogManager.getLogger(AggregationConfig.class);
|
||||||
|
|
||||||
|
// we store the query in 2 formats: the raw format and the parsed format
|
||||||
|
private final Map<String, Object> source;
|
||||||
|
private final AggregatorFactories.Builder aggregations;
|
||||||
|
|
||||||
|
public AggregationConfig(final Map<String, Object> source, AggregatorFactories.Builder aggregations) {
|
||||||
|
this.source = source;
|
||||||
|
this.aggregations = aggregations;
|
||||||
|
}
|
||||||
|
|
||||||
|
public AggregationConfig(final StreamInput in) throws IOException {
|
||||||
|
source = in.readMap();
|
||||||
|
aggregations = in.readOptionalWriteable(AggregatorFactories.Builder::new);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
return builder.map(source);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeMap(source);
|
||||||
|
out.writeOptionalWriteable(aggregations);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Collection<AggregationBuilder> getAggregatorFactories() {
|
||||||
|
return aggregations.getAggregatorFactories();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static AggregationConfig fromXContent(final XContentParser parser, boolean lenient) throws IOException {
|
||||||
|
NamedXContentRegistry registry = parser.getXContentRegistry();
|
||||||
|
Map<String, Object> source = parser.mapOrdered();
|
||||||
|
AggregatorFactories.Builder aggregations = null;
|
||||||
|
|
||||||
|
if (source.isEmpty()) {
|
||||||
|
if (lenient) {
|
||||||
|
logger.warn(DataFrameMessages.DATA_FRAME_TRANSFORM_CONFIGURATION_PIVOT_NO_AGGREGATION);
|
||||||
|
} else {
|
||||||
|
throw new IllegalArgumentException(DataFrameMessages.DATA_FRAME_TRANSFORM_CONFIGURATION_PIVOT_NO_AGGREGATION);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
try (XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().map(source);
|
||||||
|
XContentParser sourceParser = XContentType.JSON.xContent().createParser(registry, LoggingDeprecationHandler.INSTANCE,
|
||||||
|
BytesReference.bytes(xContentBuilder).streamInput())) {
|
||||||
|
sourceParser.nextToken();
|
||||||
|
aggregations = AggregatorFactories.parseAggregators(sourceParser);
|
||||||
|
} catch (Exception e) {
|
||||||
|
if (lenient) {
|
||||||
|
logger.warn(DataFrameMessages.LOG_DATA_FRAME_TRANSFORM_CONFIGURATION_BAD_AGGREGATION, e);
|
||||||
|
} else {
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return new AggregationConfig(source, aggregations);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(source, aggregations);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object other) {
|
||||||
|
if (this == other) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (other == null || getClass() != other.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
final AggregationConfig that = (AggregationConfig) other;
|
||||||
|
|
||||||
|
return Objects.equals(this.source, that.source) && Objects.equals(this.aggregations, that.aggregations);
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isValid() {
|
||||||
|
return this.aggregations != null;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,66 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregation;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregation;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregation.SingleValue;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameIndexerTransformStats;
|
||||||
|
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
final class AggregationResultUtils {
|
||||||
|
private static final Logger logger = LogManager.getLogger(AggregationResultUtils.class);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Extracts aggregation results from a composite aggregation and puts it into a map.
|
||||||
|
*
|
||||||
|
* @param agg The aggregation result
|
||||||
|
* @param groups The original groupings used for querying
|
||||||
|
* @param aggregationBuilders the aggregation used for querying
|
||||||
|
* @param dataFrameIndexerTransformStats stats collector
|
||||||
|
* @return a map containing the results of the aggregation in a consumable way
|
||||||
|
*/
|
||||||
|
public static Stream<Map<String, Object>> extractCompositeAggregationResults(CompositeAggregation agg,
|
||||||
|
GroupConfig groups, Collection<AggregationBuilder> aggregationBuilders,
|
||||||
|
DataFrameIndexerTransformStats dataFrameIndexerTransformStats) {
|
||||||
|
return agg.getBuckets().stream().map(bucket -> {
|
||||||
|
dataFrameIndexerTransformStats.incrementNumDocuments(bucket.getDocCount());
|
||||||
|
|
||||||
|
Map<String, Object> document = new HashMap<>();
|
||||||
|
groups.getGroups().keySet().forEach(destinationFieldName -> {
|
||||||
|
document.put(destinationFieldName, bucket.getKey().get(destinationFieldName));
|
||||||
|
});
|
||||||
|
|
||||||
|
for (AggregationBuilder aggregationBuilder : aggregationBuilders) {
|
||||||
|
String aggName = aggregationBuilder.getName();
|
||||||
|
|
||||||
|
// TODO: support other aggregation types
|
||||||
|
Aggregation aggResult = bucket.getAggregations().get(aggName);
|
||||||
|
|
||||||
|
if (aggResult instanceof NumericMetricsAggregation.SingleValue) {
|
||||||
|
NumericMetricsAggregation.SingleValue aggResultSingleValue = (SingleValue) aggResult;
|
||||||
|
document.put(aggName, aggResultSingleValue.value());
|
||||||
|
} else {
|
||||||
|
// Execution should never reach this point!
|
||||||
|
// Creating transforms with unsupported aggregations shall not be possible
|
||||||
|
logger.error("Dataframe Internal Error: unsupported aggregation ["+ aggResult.getName() +"], ignoring");
|
||||||
|
assert false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return document;
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,62 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import java.util.Locale;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
public final class Aggregations {
|
||||||
|
private Aggregations() {}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Supported aggregation by dataframe and corresponding meta information.
|
||||||
|
*
|
||||||
|
* aggregationType - the name of the aggregation as returned by
|
||||||
|
* {@link org.elasticsearch.search.aggregations.BaseAggregationBuilder#getType()}}
|
||||||
|
*
|
||||||
|
* targetMapping - the field type for the output, if null, the source type should be used
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
enum AggregationType {
|
||||||
|
AVG("avg", "double"),
|
||||||
|
CARDINALITY("cardinality", "long"),
|
||||||
|
VALUE_COUNT("value_count", "long"),
|
||||||
|
MAX("max", null),
|
||||||
|
MIN("min", null),
|
||||||
|
SUM("sum", null);
|
||||||
|
|
||||||
|
private final String aggregationType;
|
||||||
|
private final String targetMapping;
|
||||||
|
|
||||||
|
AggregationType(String name, String targetMapping) {
|
||||||
|
this.aggregationType = name;
|
||||||
|
this.targetMapping = targetMapping;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getName() {
|
||||||
|
return aggregationType;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getTargetMapping() {
|
||||||
|
return targetMapping;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Set<String> aggregationSupported = Stream.of(AggregationType.values()).map(AggregationType::name)
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
|
||||||
|
public static boolean isSupportedByDataframe(String aggregationType) {
|
||||||
|
return aggregationSupported.contains(aggregationType.toUpperCase(Locale.ROOT));
|
||||||
|
}
|
||||||
|
|
||||||
|
public static String resolveTargetMapping(String aggregationType, String sourceType) {
|
||||||
|
AggregationType agg = AggregationType.valueOf(aggregationType.toUpperCase(Locale.ROOT));
|
||||||
|
return agg.getTargetMapping() == null ? sourceType : agg.getTargetMapping();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,182 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.ParseField;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||||
|
import org.elasticsearch.common.xcontent.ObjectParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.time.ZoneId;
|
||||||
|
import java.time.ZoneOffset;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
|
||||||
|
public class DateHistogramGroupSource extends SingleGroupSource<DateHistogramGroupSource> {
|
||||||
|
|
||||||
|
private static final String NAME = "data_frame_date_histogram_group";
|
||||||
|
private static final ParseField TIME_ZONE = new ParseField("time_zone");
|
||||||
|
private static final ParseField FORMAT = new ParseField("format");
|
||||||
|
|
||||||
|
private static final ConstructingObjectParser<DateHistogramGroupSource, Void> STRICT_PARSER = createParser(false);
|
||||||
|
private static final ConstructingObjectParser<DateHistogramGroupSource, Void> LENIENT_PARSER = createParser(true);
|
||||||
|
private long interval = 0;
|
||||||
|
private DateHistogramInterval dateHistogramInterval;
|
||||||
|
private String format;
|
||||||
|
private ZoneId timeZone;
|
||||||
|
|
||||||
|
public DateHistogramGroupSource(String field) {
|
||||||
|
super(field);
|
||||||
|
}
|
||||||
|
|
||||||
|
public DateHistogramGroupSource(StreamInput in) throws IOException {
|
||||||
|
super(in);
|
||||||
|
this.interval = in.readLong();
|
||||||
|
this.dateHistogramInterval = in.readOptionalWriteable(DateHistogramInterval::new);
|
||||||
|
this.timeZone = in.readOptionalZoneId();
|
||||||
|
this.format = in.readOptionalString();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static ConstructingObjectParser<DateHistogramGroupSource, Void> createParser(boolean lenient) {
|
||||||
|
ConstructingObjectParser<DateHistogramGroupSource, Void> parser = new ConstructingObjectParser<>(NAME, lenient, (args) -> {
|
||||||
|
String field = (String) args[0];
|
||||||
|
return new DateHistogramGroupSource(field);
|
||||||
|
});
|
||||||
|
|
||||||
|
SingleGroupSource.declareValuesSourceFields(parser, null);
|
||||||
|
|
||||||
|
parser.declareField((histogram, interval) -> {
|
||||||
|
if (interval instanceof Long) {
|
||||||
|
histogram.setInterval((long) interval);
|
||||||
|
} else {
|
||||||
|
histogram.setDateHistogramInterval((DateHistogramInterval) interval);
|
||||||
|
}
|
||||||
|
}, p -> {
|
||||||
|
if (p.currentToken() == XContentParser.Token.VALUE_NUMBER) {
|
||||||
|
return p.longValue();
|
||||||
|
} else {
|
||||||
|
return new DateHistogramInterval(p.text());
|
||||||
|
}
|
||||||
|
}, HistogramGroupSource.INTERVAL, ObjectParser.ValueType.LONG);
|
||||||
|
|
||||||
|
parser.declareField(DateHistogramGroupSource::setTimeZone, p -> {
|
||||||
|
if (p.currentToken() == XContentParser.Token.VALUE_STRING) {
|
||||||
|
return ZoneId.of(p.text());
|
||||||
|
} else {
|
||||||
|
return ZoneOffset.ofHours(p.intValue());
|
||||||
|
}
|
||||||
|
}, TIME_ZONE, ObjectParser.ValueType.LONG);
|
||||||
|
|
||||||
|
parser.declareString(DateHistogramGroupSource::setFormat, FORMAT);
|
||||||
|
return parser;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DateHistogramGroupSource fromXContent(final XContentParser parser, boolean lenient) throws IOException {
|
||||||
|
return lenient ? LENIENT_PARSER.apply(parser, null) : STRICT_PARSER.apply(parser, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Type getType() {
|
||||||
|
return Type.DATE_HISTOGRAM;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getInterval() {
|
||||||
|
return interval;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setInterval(long interval) {
|
||||||
|
if (interval < 1) {
|
||||||
|
throw new IllegalArgumentException("[interval] must be greater than or equal to 1.");
|
||||||
|
}
|
||||||
|
this.interval = interval;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DateHistogramInterval getDateHistogramInterval() {
|
||||||
|
return dateHistogramInterval;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setDateHistogramInterval(DateHistogramInterval dateHistogramInterval) {
|
||||||
|
if (dateHistogramInterval == null) {
|
||||||
|
throw new IllegalArgumentException("[dateHistogramInterval] must not be null");
|
||||||
|
}
|
||||||
|
this.dateHistogramInterval = dateHistogramInterval;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getFormat() {
|
||||||
|
return format;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setFormat(String format) {
|
||||||
|
this.format = format;
|
||||||
|
}
|
||||||
|
|
||||||
|
public ZoneId getTimeZone() {
|
||||||
|
return timeZone;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setTimeZone(ZoneId timeZone) {
|
||||||
|
this.timeZone = timeZone;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeOptionalString(field);
|
||||||
|
out.writeLong(interval);
|
||||||
|
out.writeOptionalWriteable(dateHistogramInterval);
|
||||||
|
out.writeOptionalZoneId(timeZone);
|
||||||
|
out.writeOptionalString(format);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
if (field != null) {
|
||||||
|
builder.field(FIELD.getPreferredName(), field);
|
||||||
|
}
|
||||||
|
if (dateHistogramInterval == null) {
|
||||||
|
builder.field(HistogramGroupSource.INTERVAL.getPreferredName(), interval);
|
||||||
|
} else {
|
||||||
|
builder.field(HistogramGroupSource.INTERVAL.getPreferredName(), dateHistogramInterval.toString());
|
||||||
|
}
|
||||||
|
if (timeZone != null) {
|
||||||
|
builder.field(TIME_ZONE.getPreferredName(), timeZone.toString());
|
||||||
|
}
|
||||||
|
if (format != null) {
|
||||||
|
builder.field(FORMAT.getPreferredName(), format);
|
||||||
|
}
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object other) {
|
||||||
|
if (this == other) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (other == null || getClass() != other.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
final DateHistogramGroupSource that = (DateHistogramGroupSource) other;
|
||||||
|
|
||||||
|
return Objects.equals(this.field, that.field) &&
|
||||||
|
Objects.equals(interval, that.interval) &&
|
||||||
|
Objects.equals(dateHistogramInterval, that.dateHistogramInterval) &&
|
||||||
|
Objects.equals(timeZone, that.timeZone) &&
|
||||||
|
Objects.equals(format, that.format);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(field, interval, dateHistogramInterval, timeZone, format);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,185 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.elasticsearch.common.ParsingException;
|
||||||
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.common.xcontent.LoggingDeprecationHandler;
|
||||||
|
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameMessages;
|
||||||
|
import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.pivot.SingleGroupSource.Type;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
|
import java.util.Locale;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Wraps a single group for groupby
|
||||||
|
*/
|
||||||
|
public class GroupConfig implements Writeable, ToXContentObject {
|
||||||
|
|
||||||
|
private static final Logger logger = LogManager.getLogger(GroupConfig.class);
|
||||||
|
|
||||||
|
private final Map<String, Object> source;
|
||||||
|
private final Map<String, SingleGroupSource<?>> groups;
|
||||||
|
|
||||||
|
public GroupConfig(final Map<String, Object> source, final Map<String, SingleGroupSource<?>> groups) {
|
||||||
|
this.source = ExceptionsHelper.requireNonNull(source, DataFrameField.GROUP_BY.getPreferredName());
|
||||||
|
this.groups = groups;
|
||||||
|
}
|
||||||
|
|
||||||
|
public GroupConfig(StreamInput in) throws IOException {
|
||||||
|
source = in.readMap();
|
||||||
|
groups = in.readMap(StreamInput::readString, (stream) -> {
|
||||||
|
Type groupType = Type.fromId(stream.readByte());
|
||||||
|
switch (groupType) {
|
||||||
|
case TERMS:
|
||||||
|
return new TermsGroupSource(stream);
|
||||||
|
case HISTOGRAM:
|
||||||
|
return new HistogramGroupSource(stream);
|
||||||
|
case DATE_HISTOGRAM:
|
||||||
|
return new DateHistogramGroupSource(stream);
|
||||||
|
default:
|
||||||
|
throw new IOException("Unknown group type");
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
public Map <String, SingleGroupSource<?>> getGroups() {
|
||||||
|
return groups;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isValid() {
|
||||||
|
return this.groups != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeMap(source);
|
||||||
|
out.writeMap(groups, StreamOutput::writeString, (stream, value) -> {
|
||||||
|
stream.writeByte(value.getType().getId());
|
||||||
|
value.writeTo(stream);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
return builder.map(source);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object other) {
|
||||||
|
if (this == other) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (other == null || getClass() != other.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
final GroupConfig that = (GroupConfig) other;
|
||||||
|
|
||||||
|
return Objects.equals(this.source, that.source) && Objects.equals(this.groups, that.groups);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(source, groups);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static GroupConfig fromXContent(final XContentParser parser, boolean lenient) throws IOException {
|
||||||
|
NamedXContentRegistry registry = parser.getXContentRegistry();
|
||||||
|
Map<String, Object> source = parser.mapOrdered();
|
||||||
|
Map<String, SingleGroupSource<?>> groups = null;
|
||||||
|
|
||||||
|
if (source.isEmpty()) {
|
||||||
|
if (lenient) {
|
||||||
|
logger.warn(DataFrameMessages.DATA_FRAME_TRANSFORM_CONFIGURATION_PIVOT_NO_GROUP_BY);
|
||||||
|
} else {
|
||||||
|
throw new IllegalArgumentException(DataFrameMessages.DATA_FRAME_TRANSFORM_CONFIGURATION_PIVOT_NO_GROUP_BY);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
try (XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().map(source);
|
||||||
|
XContentParser sourceParser = XContentType.JSON.xContent().createParser(registry, LoggingDeprecationHandler.INSTANCE,
|
||||||
|
BytesReference.bytes(xContentBuilder).streamInput())) {
|
||||||
|
groups = parseGroupConfig(sourceParser, lenient);
|
||||||
|
} catch (Exception e) {
|
||||||
|
if (lenient) {
|
||||||
|
logger.warn(DataFrameMessages.LOG_DATA_FRAME_TRANSFORM_CONFIGURATION_BAD_GROUP_BY, e);
|
||||||
|
} else {
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return new GroupConfig(source, groups);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Map<String, SingleGroupSource<?>> parseGroupConfig(final XContentParser parser,
|
||||||
|
boolean lenient) throws IOException {
|
||||||
|
LinkedHashMap<String, SingleGroupSource<?>> groups = new LinkedHashMap<>();
|
||||||
|
|
||||||
|
// be parsing friendly, whether the token needs to be advanced or not (similar to what ObjectParser does)
|
||||||
|
XContentParser.Token token;
|
||||||
|
if (parser.currentToken() == XContentParser.Token.START_OBJECT) {
|
||||||
|
token = parser.currentToken();
|
||||||
|
} else {
|
||||||
|
token = parser.nextToken();
|
||||||
|
if (token != XContentParser.Token.START_OBJECT) {
|
||||||
|
throw new ParsingException(parser.getTokenLocation(), "Failed to parse object: Expected START_OBJECT but was: " + token);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||||
|
|
||||||
|
ensureExpectedToken(XContentParser.Token.FIELD_NAME, token, parser::getTokenLocation);
|
||||||
|
String destinationFieldName = parser.currentName();
|
||||||
|
token = parser.nextToken();
|
||||||
|
ensureExpectedToken(XContentParser.Token.START_OBJECT, token, parser::getTokenLocation);
|
||||||
|
token = parser.nextToken();
|
||||||
|
ensureExpectedToken(XContentParser.Token.FIELD_NAME, token, parser::getTokenLocation);
|
||||||
|
Type groupType = SingleGroupSource.Type.valueOf(parser.currentName().toUpperCase(Locale.ROOT));
|
||||||
|
|
||||||
|
token = parser.nextToken();
|
||||||
|
ensureExpectedToken(XContentParser.Token.START_OBJECT, token, parser::getTokenLocation);
|
||||||
|
SingleGroupSource<?> groupSource;
|
||||||
|
switch (groupType) {
|
||||||
|
case TERMS:
|
||||||
|
groupSource = TermsGroupSource.fromXContent(parser, lenient);
|
||||||
|
break;
|
||||||
|
case HISTOGRAM:
|
||||||
|
groupSource = HistogramGroupSource.fromXContent(parser, lenient);
|
||||||
|
break;
|
||||||
|
case DATE_HISTOGRAM:
|
||||||
|
groupSource = DateHistogramGroupSource.fromXContent(parser, lenient);
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new ParsingException(parser.getTokenLocation(), "invalid grouping type: " + groupType);
|
||||||
|
}
|
||||||
|
|
||||||
|
parser.nextToken();
|
||||||
|
|
||||||
|
groups.put(destinationFieldName, groupSource);
|
||||||
|
}
|
||||||
|
return groups;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,102 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.ParseField;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg;
|
||||||
|
|
||||||
|
public class HistogramGroupSource extends SingleGroupSource<HistogramGroupSource> {
|
||||||
|
|
||||||
|
static final ParseField INTERVAL = new ParseField("interval");
|
||||||
|
private static final String NAME = "data_frame_histogram_group";
|
||||||
|
private static final ConstructingObjectParser<HistogramGroupSource, Void> STRICT_PARSER = createParser(false);
|
||||||
|
private static final ConstructingObjectParser<HistogramGroupSource, Void> LENIENT_PARSER = createParser(true);
|
||||||
|
private final double interval;
|
||||||
|
|
||||||
|
public HistogramGroupSource(String field, double interval) {
|
||||||
|
super(field);
|
||||||
|
if (interval <= 0) {
|
||||||
|
throw new IllegalArgumentException("[interval] must be greater than 0.");
|
||||||
|
}
|
||||||
|
this.interval = interval;
|
||||||
|
}
|
||||||
|
|
||||||
|
public HistogramGroupSource(StreamInput in) throws IOException {
|
||||||
|
super(in);
|
||||||
|
interval = in.readDouble();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static ConstructingObjectParser<HistogramGroupSource, Void> createParser(boolean lenient) {
|
||||||
|
ConstructingObjectParser<HistogramGroupSource, Void> parser = new ConstructingObjectParser<>(NAME, lenient, (args) -> {
|
||||||
|
String field = (String) args[0];
|
||||||
|
double interval = (double) args[1];
|
||||||
|
return new HistogramGroupSource(field, interval);
|
||||||
|
});
|
||||||
|
SingleGroupSource.declareValuesSourceFields(parser, null);
|
||||||
|
parser.declareDouble(optionalConstructorArg(), INTERVAL);
|
||||||
|
return parser;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Type getType() {
|
||||||
|
return Type.HISTOGRAM;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static HistogramGroupSource fromXContent(final XContentParser parser, boolean lenient) throws IOException {
|
||||||
|
return lenient ? LENIENT_PARSER.apply(parser, null) : STRICT_PARSER.apply(parser, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeOptionalString(field);
|
||||||
|
out.writeDouble(interval);
|
||||||
|
}
|
||||||
|
|
||||||
|
public double getInterval() {
|
||||||
|
return interval;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
if (field != null) {
|
||||||
|
builder.field(FIELD.getPreferredName(), field);
|
||||||
|
}
|
||||||
|
builder.field(INTERVAL.getPreferredName(), interval);
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object other) {
|
||||||
|
if (this == other) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (other == null || getClass() != other.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
final HistogramGroupSource that = (HistogramGroupSource) other;
|
||||||
|
|
||||||
|
return Objects.equals(this.field, that.field) &&
|
||||||
|
Objects.equals(this.interval, that.interval);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(field, interval);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,130 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.search.SearchAction;
|
||||||
|
import org.elasticsearch.action.search.SearchRequest;
|
||||||
|
import org.elasticsearch.client.Client;
|
||||||
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
|
import org.elasticsearch.common.xcontent.LoggingDeprecationHandler;
|
||||||
|
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.index.query.QueryBuilder;
|
||||||
|
import org.elasticsearch.rest.RestStatus;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregationBuilder;
|
||||||
|
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameMessages;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameIndexerTransformStats;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||||
|
|
||||||
|
public class Pivot {
|
||||||
|
private static final String COMPOSITE_AGGREGATION_NAME = "_data_frame";
|
||||||
|
|
||||||
|
private final PivotConfig config;
|
||||||
|
private final String source;
|
||||||
|
|
||||||
|
// objects for re-using
|
||||||
|
private final CompositeAggregationBuilder cachedCompositeAggregation;
|
||||||
|
private final SearchRequest cachedSearchRequest;
|
||||||
|
|
||||||
|
public Pivot(String source, QueryBuilder query, PivotConfig config) {
|
||||||
|
this.source = source;
|
||||||
|
this.config = config;
|
||||||
|
this.cachedCompositeAggregation = createCompositeAggregation(config);
|
||||||
|
this.cachedSearchRequest = createSearchRequest(source, query, cachedCompositeAggregation);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void validate(Client client, final ActionListener<Boolean> listener) {
|
||||||
|
// step 1: check if used aggregations are supported
|
||||||
|
for (AggregationBuilder agg : config.getAggregationConfig().getAggregatorFactories()) {
|
||||||
|
if (Aggregations.isSupportedByDataframe(agg.getType()) == false) {
|
||||||
|
listener.onFailure(new RuntimeException("Unsupported aggregation type [" + agg.getType() + "]"));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// step 2: run a query to validate that config is valid
|
||||||
|
runTestQuery(client, listener);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void deduceMappings(Client client, final ActionListener<Map<String, String>> listener) {
|
||||||
|
SchemaUtil.deduceMappings(client, config, source, listener);
|
||||||
|
}
|
||||||
|
|
||||||
|
public SearchRequest buildSearchRequest(Map<String, Object> position) {
|
||||||
|
if (position != null) {
|
||||||
|
cachedCompositeAggregation.aggregateAfter(position);
|
||||||
|
}
|
||||||
|
|
||||||
|
return cachedSearchRequest;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Stream<Map<String, Object>> extractResults(CompositeAggregation agg,
|
||||||
|
DataFrameIndexerTransformStats dataFrameIndexerTransformStats) {
|
||||||
|
|
||||||
|
GroupConfig groups = config.getGroupConfig();
|
||||||
|
Collection<AggregationBuilder> aggregationBuilders = config.getAggregationConfig().getAggregatorFactories();
|
||||||
|
|
||||||
|
return AggregationResultUtils.extractCompositeAggregationResults(agg, groups, aggregationBuilders, dataFrameIndexerTransformStats);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void runTestQuery(Client client, final ActionListener<Boolean> listener) {
|
||||||
|
// no after key
|
||||||
|
cachedCompositeAggregation.aggregateAfter(null);
|
||||||
|
client.execute(SearchAction.INSTANCE, cachedSearchRequest, ActionListener.wrap(response -> {
|
||||||
|
if (response == null) {
|
||||||
|
listener.onFailure(new RuntimeException("Unexpected null response from test query"));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (response.status() != RestStatus.OK) {
|
||||||
|
listener.onFailure(new RuntimeException("Unexpected status from response of test query: " + response.status()));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
listener.onResponse(true);
|
||||||
|
}, e->{
|
||||||
|
listener.onFailure(new RuntimeException("Failed to test query",e));
|
||||||
|
}));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static SearchRequest createSearchRequest(String index, QueryBuilder query, CompositeAggregationBuilder compositeAggregation) {
|
||||||
|
SearchRequest searchRequest = new SearchRequest(index);
|
||||||
|
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder();
|
||||||
|
sourceBuilder.aggregation(compositeAggregation);
|
||||||
|
sourceBuilder.size(0);
|
||||||
|
sourceBuilder.query(query);
|
||||||
|
searchRequest.source(sourceBuilder);
|
||||||
|
return searchRequest;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static CompositeAggregationBuilder createCompositeAggregation(PivotConfig config) {
|
||||||
|
CompositeAggregationBuilder compositeAggregation;
|
||||||
|
|
||||||
|
try (XContentBuilder builder = jsonBuilder()) {
|
||||||
|
// write configuration for composite aggs into builder
|
||||||
|
config.toCompositeAggXContent(builder, ToXContentObject.EMPTY_PARAMS);
|
||||||
|
XContentParser parser = builder.generator().contentType().xContent().createParser(NamedXContentRegistry.EMPTY,
|
||||||
|
LoggingDeprecationHandler.INSTANCE, BytesReference.bytes(builder).streamInput());
|
||||||
|
compositeAggregation = CompositeAggregationBuilder.parse(COMPOSITE_AGGREGATION_NAME, parser);
|
||||||
|
compositeAggregation.size(1000);
|
||||||
|
config.getAggregationConfig().getAggregatorFactories().forEach(agg -> compositeAggregation.subAggregation(agg));
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new RuntimeException(DataFrameMessages.DATA_FRAME_TRANSFORM_PIVOT_FAILED_TO_CREATE_COMPOSITE_AGGREGATION, e);
|
||||||
|
}
|
||||||
|
return compositeAggregation;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,147 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregationBuilder;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||||
|
import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Map.Entry;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg;
|
||||||
|
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg;
|
||||||
|
|
||||||
|
public class PivotConfig implements Writeable, ToXContentObject {
|
||||||
|
|
||||||
|
private static final String NAME = "data_frame_transform_pivot";
|
||||||
|
private final GroupConfig groups;
|
||||||
|
private final AggregationConfig aggregationConfig;
|
||||||
|
|
||||||
|
private static final ConstructingObjectParser<PivotConfig, Void> STRICT_PARSER = createParser(false);
|
||||||
|
private static final ConstructingObjectParser<PivotConfig, Void> LENIENT_PARSER = createParser(true);
|
||||||
|
|
||||||
|
private static ConstructingObjectParser<PivotConfig, Void> createParser(boolean lenient) {
|
||||||
|
ConstructingObjectParser<PivotConfig, Void> parser = new ConstructingObjectParser<>(NAME, lenient,
|
||||||
|
args -> {
|
||||||
|
GroupConfig groups = (GroupConfig) args[0];
|
||||||
|
|
||||||
|
// allow "aggs" and "aggregations" but require one to be specified
|
||||||
|
// if somebody specifies both: throw
|
||||||
|
AggregationConfig aggregationConfig = null;
|
||||||
|
if (args[1] != null) {
|
||||||
|
aggregationConfig = (AggregationConfig) args[1];
|
||||||
|
}
|
||||||
|
|
||||||
|
if (args[2] != null) {
|
||||||
|
if (aggregationConfig != null) {
|
||||||
|
throw new IllegalArgumentException("Found two aggregation definitions: [aggs] and [aggregations]");
|
||||||
|
}
|
||||||
|
aggregationConfig = (AggregationConfig) args[2];
|
||||||
|
}
|
||||||
|
if (aggregationConfig == null) {
|
||||||
|
throw new IllegalArgumentException("Required [aggregations]");
|
||||||
|
}
|
||||||
|
|
||||||
|
return new PivotConfig(groups, aggregationConfig);
|
||||||
|
});
|
||||||
|
|
||||||
|
parser.declareObject(constructorArg(),
|
||||||
|
(p, c) -> (GroupConfig.fromXContent(p, lenient)), DataFrameField.GROUP_BY);
|
||||||
|
|
||||||
|
parser.declareObject(optionalConstructorArg(), (p, c) -> AggregationConfig.fromXContent(p, lenient), DataFrameField.AGGREGATIONS);
|
||||||
|
parser.declareObject(optionalConstructorArg(), (p, c) -> AggregationConfig.fromXContent(p, lenient), DataFrameField.AGGS);
|
||||||
|
|
||||||
|
return parser;
|
||||||
|
}
|
||||||
|
|
||||||
|
public PivotConfig(final GroupConfig groups, final AggregationConfig aggregationConfig) {
|
||||||
|
this.groups = ExceptionsHelper.requireNonNull(groups, DataFrameField.GROUP_BY.getPreferredName());
|
||||||
|
this.aggregationConfig = ExceptionsHelper.requireNonNull(aggregationConfig, DataFrameField.AGGREGATIONS.getPreferredName());
|
||||||
|
}
|
||||||
|
|
||||||
|
public PivotConfig(StreamInput in) throws IOException {
|
||||||
|
this.groups = new GroupConfig(in);
|
||||||
|
this.aggregationConfig = new AggregationConfig(in);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
builder.field(DataFrameField.GROUP_BY.getPreferredName(), groups);
|
||||||
|
builder.field(DataFrameField.AGGREGATIONS.getPreferredName(), aggregationConfig);
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void toCompositeAggXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
builder.field(CompositeAggregationBuilder.SOURCES_FIELD_NAME.getPreferredName());
|
||||||
|
builder.startArray();
|
||||||
|
|
||||||
|
for (Entry<String, SingleGroupSource<?>> groupBy : groups.getGroups().entrySet()) {
|
||||||
|
builder.startObject();
|
||||||
|
builder.startObject(groupBy.getKey());
|
||||||
|
builder.field(groupBy.getValue().getType().value(), groupBy.getValue());
|
||||||
|
builder.endObject();
|
||||||
|
builder.endObject();
|
||||||
|
}
|
||||||
|
|
||||||
|
builder.endArray();
|
||||||
|
builder.endObject(); // sources
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
groups.writeTo(out);
|
||||||
|
aggregationConfig.writeTo(out);
|
||||||
|
}
|
||||||
|
|
||||||
|
public AggregationConfig getAggregationConfig() {
|
||||||
|
return aggregationConfig;
|
||||||
|
}
|
||||||
|
|
||||||
|
public GroupConfig getGroupConfig() {
|
||||||
|
return groups;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object other) {
|
||||||
|
if (this == other) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (other == null || getClass() != other.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
final PivotConfig that = (PivotConfig) other;
|
||||||
|
|
||||||
|
return Objects.equals(this.groups, that.groups) && Objects.equals(this.aggregationConfig, that.aggregationConfig);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(groups, aggregationConfig);
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isValid() {
|
||||||
|
return groups.isValid() && aggregationConfig.isValid();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static PivotConfig fromXContent(final XContentParser parser, boolean lenient) throws IOException {
|
||||||
|
return lenient ? LENIENT_PARSER.apply(parser, null) : STRICT_PARSER.apply(parser, null);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,143 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsAction;
|
||||||
|
import org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsRequest;
|
||||||
|
import org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsResponse.FieldMappingMetaData;
|
||||||
|
import org.elasticsearch.client.Client;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
|
||||||
|
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class SchemaUtil {
|
||||||
|
private static final Logger logger = LogManager.getLogger(SchemaUtil.class);
|
||||||
|
|
||||||
|
private SchemaUtil() {
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void deduceMappings(final Client client, final PivotConfig config, final String source,
|
||||||
|
final ActionListener<Map<String, String>> listener) {
|
||||||
|
// collects the fieldnames used as source for aggregations
|
||||||
|
Map<String, String> aggregationSourceFieldNames = new HashMap<>();
|
||||||
|
// collects the aggregation types by source name
|
||||||
|
Map<String, String> aggregationTypes = new HashMap<>();
|
||||||
|
// collects the fieldnames and target fieldnames used for grouping
|
||||||
|
Map<String, String> fieldNamesForGrouping = new HashMap<>();
|
||||||
|
|
||||||
|
config.getGroupConfig().getGroups().forEach((destinationFieldName, group) -> {
|
||||||
|
fieldNamesForGrouping.put(destinationFieldName, group.getField());
|
||||||
|
});
|
||||||
|
|
||||||
|
for (AggregationBuilder agg : config.getAggregationConfig().getAggregatorFactories()) {
|
||||||
|
if (agg instanceof ValuesSourceAggregationBuilder) {
|
||||||
|
ValuesSourceAggregationBuilder<?, ?> valueSourceAggregation = (ValuesSourceAggregationBuilder<?, ?>) agg;
|
||||||
|
aggregationSourceFieldNames.put(valueSourceAggregation.getName(), valueSourceAggregation.field());
|
||||||
|
aggregationTypes.put(valueSourceAggregation.getName(), valueSourceAggregation.getType());
|
||||||
|
} else {
|
||||||
|
// execution should not reach this point
|
||||||
|
listener.onFailure(new RuntimeException("Unsupported aggregation type [" + agg.getType() + "]"));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Map<String, String> allFieldNames = new HashMap<>();
|
||||||
|
allFieldNames.putAll(aggregationSourceFieldNames);
|
||||||
|
allFieldNames.putAll(fieldNamesForGrouping);
|
||||||
|
|
||||||
|
getSourceFieldMappings(client, source, allFieldNames.values().toArray(new String[0]),
|
||||||
|
ActionListener.wrap(sourceMappings -> {
|
||||||
|
Map<String, String> targetMapping = resolveMappings(aggregationSourceFieldNames, aggregationTypes,
|
||||||
|
fieldNamesForGrouping, sourceMappings);
|
||||||
|
|
||||||
|
listener.onResponse(targetMapping);
|
||||||
|
}, e -> {
|
||||||
|
listener.onFailure(e);
|
||||||
|
}));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Map<String, String> resolveMappings(Map<String, String> aggregationSourceFieldNames,
|
||||||
|
Map<String, String> aggregationTypes, Map<String, String> fieldNamesForGrouping, Map<String, String> sourceMappings) {
|
||||||
|
Map<String, String> targetMapping = new HashMap<>();
|
||||||
|
|
||||||
|
aggregationTypes.forEach((targetFieldName, aggregationName) -> {
|
||||||
|
String sourceFieldName = aggregationSourceFieldNames.get(targetFieldName);
|
||||||
|
String destinationMapping = Aggregations.resolveTargetMapping(aggregationName, sourceMappings.get(sourceFieldName));
|
||||||
|
|
||||||
|
logger.debug(
|
||||||
|
"Deduced mapping for: [" + targetFieldName + "], agg type [" + aggregationName + "] to [" + destinationMapping + "]");
|
||||||
|
if (destinationMapping != null) {
|
||||||
|
targetMapping.put(targetFieldName, destinationMapping);
|
||||||
|
} else {
|
||||||
|
logger.warn("Failed to deduce mapping for [" + targetFieldName + "], fall back to double.");
|
||||||
|
targetMapping.put(targetFieldName, "double");
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
fieldNamesForGrouping.forEach((targetFieldName, sourceFieldName) -> {
|
||||||
|
String destinationMapping = sourceMappings.get(sourceFieldName);
|
||||||
|
logger.debug(
|
||||||
|
"Deduced mapping for: [" + targetFieldName + "] to [" + destinationMapping + "]");
|
||||||
|
if (destinationMapping != null) {
|
||||||
|
targetMapping.put(targetFieldName, destinationMapping);
|
||||||
|
} else {
|
||||||
|
logger.warn("Failed to deduce mapping for [" + targetFieldName + "], fall back to keyword.");
|
||||||
|
targetMapping.put(targetFieldName, "keyword");
|
||||||
|
}
|
||||||
|
});
|
||||||
|
return targetMapping;
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Very "magic" helper method to extract the source mappings
|
||||||
|
*/
|
||||||
|
private static void getSourceFieldMappings(Client client, String index, String[] fields,
|
||||||
|
ActionListener<Map<String, String>> listener) {
|
||||||
|
GetFieldMappingsRequest fieldMappingRequest = new GetFieldMappingsRequest();
|
||||||
|
fieldMappingRequest.indices(index);
|
||||||
|
fieldMappingRequest.fields(fields);
|
||||||
|
|
||||||
|
client.execute(GetFieldMappingsAction.INSTANCE, fieldMappingRequest, ActionListener.wrap(response -> {
|
||||||
|
listener.onResponse(extractSourceFieldMappings(response.mappings()));
|
||||||
|
}, e -> {
|
||||||
|
listener.onFailure(e);
|
||||||
|
}));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Map<String, String> extractSourceFieldMappings(Map<String, Map<String, Map<String, FieldMappingMetaData>>> mappings) {
|
||||||
|
Map<String, String> extractedTypes = new HashMap<>();
|
||||||
|
|
||||||
|
mappings.forEach((indexName, docTypeToMapping) -> {
|
||||||
|
// "_doc" ->
|
||||||
|
docTypeToMapping.forEach((docType, fieldNameToMapping) -> {
|
||||||
|
// "my_field" ->
|
||||||
|
fieldNameToMapping.forEach((fieldName, fieldMapping) -> {
|
||||||
|
// "mapping" -> "my_field" ->
|
||||||
|
fieldMapping.sourceAsMap().forEach((name, typeMap) -> {
|
||||||
|
// expected object: { "type": type }
|
||||||
|
if (typeMap instanceof Map) {
|
||||||
|
final Map<?, ?> map = (Map<?, ?>) typeMap;
|
||||||
|
if (map.containsKey("type")) {
|
||||||
|
String type = map.get("type").toString();
|
||||||
|
logger.debug("Extracted type for [" + fieldName + "] : [" + type + "]");
|
||||||
|
// TODO: overwrites types, requires resolve if
|
||||||
|
// types are mixed
|
||||||
|
extractedTypes.put(fieldName, type);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
});
|
||||||
|
});
|
||||||
|
});
|
||||||
|
});
|
||||||
|
return extractedTypes;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,121 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.ParseField;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.common.xcontent.AbstractObjectParser;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Locale;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Base class for a single source for group_by
|
||||||
|
*/
|
||||||
|
public abstract class SingleGroupSource<AB extends SingleGroupSource<AB>> implements Writeable, ToXContentObject {
|
||||||
|
|
||||||
|
public enum Type {
|
||||||
|
TERMS(0),
|
||||||
|
HISTOGRAM(1),
|
||||||
|
DATE_HISTOGRAM(2);
|
||||||
|
|
||||||
|
private final byte id;
|
||||||
|
|
||||||
|
Type(int id) {
|
||||||
|
this.id = (byte) id;
|
||||||
|
}
|
||||||
|
|
||||||
|
public byte getId() {
|
||||||
|
return id;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Type fromId(byte id) {
|
||||||
|
switch (id) {
|
||||||
|
case 0:
|
||||||
|
return TERMS;
|
||||||
|
case 1:
|
||||||
|
return HISTOGRAM;
|
||||||
|
case 2:
|
||||||
|
return DATE_HISTOGRAM;
|
||||||
|
default:
|
||||||
|
throw new IllegalArgumentException("unknown type");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public String value() {
|
||||||
|
return name().toLowerCase(Locale.ROOT);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static final ParseField FIELD = new ParseField("field");
|
||||||
|
|
||||||
|
// TODO: add script
|
||||||
|
protected final String field;
|
||||||
|
|
||||||
|
static <VB extends SingleGroupSource<?>, T> void declareValuesSourceFields(AbstractObjectParser<VB, T> parser,
|
||||||
|
ValueType targetValueType) {
|
||||||
|
// either script or field
|
||||||
|
parser.declareString(optionalConstructorArg(), FIELD);
|
||||||
|
}
|
||||||
|
|
||||||
|
public SingleGroupSource(final String field) {
|
||||||
|
this.field = field;
|
||||||
|
}
|
||||||
|
|
||||||
|
public SingleGroupSource(StreamInput in) throws IOException {
|
||||||
|
field = in.readOptionalString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
if (field != null) {
|
||||||
|
builder.field(FIELD.getPreferredName(), field);
|
||||||
|
}
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeOptionalString(field);
|
||||||
|
}
|
||||||
|
|
||||||
|
public abstract Type getType();
|
||||||
|
|
||||||
|
public String getField() {
|
||||||
|
return field;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object other) {
|
||||||
|
if (this == other) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (other == null || getClass() != other.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
final SingleGroupSource<?> that = (SingleGroupSource<?>) other;
|
||||||
|
|
||||||
|
return Objects.equals(this.field, that.field);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(field);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,50 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* A terms aggregation source for group_by
|
||||||
|
*/
|
||||||
|
public class TermsGroupSource extends SingleGroupSource<TermsGroupSource> {
|
||||||
|
private static final String NAME = "data_frame_terms_group";
|
||||||
|
|
||||||
|
private static final ConstructingObjectParser<TermsGroupSource, Void> STRICT_PARSER = createParser(false);
|
||||||
|
private static final ConstructingObjectParser<TermsGroupSource, Void> LENIENT_PARSER = createParser(true);
|
||||||
|
|
||||||
|
private static ConstructingObjectParser<TermsGroupSource, Void> createParser(boolean lenient) {
|
||||||
|
ConstructingObjectParser<TermsGroupSource, Void> parser = new ConstructingObjectParser<>(NAME, lenient, (args) -> {
|
||||||
|
String field = (String) args[0];
|
||||||
|
return new TermsGroupSource(field);
|
||||||
|
});
|
||||||
|
|
||||||
|
SingleGroupSource.declareValuesSourceFields(parser, null);
|
||||||
|
return parser;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TermsGroupSource(final String field) {
|
||||||
|
super(field);
|
||||||
|
}
|
||||||
|
|
||||||
|
public TermsGroupSource(StreamInput in) throws IOException {
|
||||||
|
super(in);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Type getType() {
|
||||||
|
return Type.TERMS;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static TermsGroupSource fromXContent(final XContentParser parser, boolean lenient) throws IOException {
|
||||||
|
return lenient ? LENIENT_PARSER.apply(parser, null) : STRICT_PARSER.apply(parser, null);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,152 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.support.PlainActionFuture;
|
||||||
|
import org.elasticsearch.client.Client;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||||
|
import org.elasticsearch.license.XPackLicenseState;
|
||||||
|
import org.elasticsearch.test.ESTestCase;
|
||||||
|
import org.elasticsearch.xpack.core.XPackFeatureSet;
|
||||||
|
import org.elasticsearch.xpack.core.XPackFeatureSet.Usage;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameIndexerTransformStats;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.DataFrameTransformStateAndStats;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.DataFrameTransformStateAndStatsTests;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.GetDataFrameTransformsStatsAction;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.GetDataFrameTransformsStatsAction.Response;
|
||||||
|
import org.junit.Before;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.ExecutionException;
|
||||||
|
|
||||||
|
import static java.lang.Math.toIntExact;
|
||||||
|
import static org.hamcrest.core.Is.is;
|
||||||
|
import static org.mockito.Matchers.any;
|
||||||
|
import static org.mockito.Matchers.same;
|
||||||
|
import static org.mockito.Mockito.doAnswer;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
|
public class DataFrameFeatureSetTests extends ESTestCase {
|
||||||
|
private XPackLicenseState licenseState;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void init() {
|
||||||
|
licenseState = mock(XPackLicenseState.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testAvailable() {
|
||||||
|
DataFrameFeatureSet featureSet = new DataFrameFeatureSet(Settings.EMPTY, mock(Client.class), licenseState);
|
||||||
|
boolean available = randomBoolean();
|
||||||
|
when(licenseState.isDataFrameAllowed()).thenReturn(available);
|
||||||
|
assertThat(featureSet.available(), is(available));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testEnabledSetting() {
|
||||||
|
boolean enabled = randomBoolean();
|
||||||
|
Settings.Builder settings = Settings.builder();
|
||||||
|
settings.put("xpack.data_frame.enabled", enabled);
|
||||||
|
DataFrameFeatureSet featureSet = new DataFrameFeatureSet(settings.build(), mock(Client.class), licenseState);
|
||||||
|
assertThat(featureSet.enabled(), is(enabled));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testEnabledDefault() {
|
||||||
|
DataFrameFeatureSet featureSet = new DataFrameFeatureSet(Settings.EMPTY, mock(Client.class), licenseState);
|
||||||
|
assertTrue(featureSet.enabled());
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testUsage() throws InterruptedException, ExecutionException, IOException {
|
||||||
|
Client client = mock(Client.class);
|
||||||
|
when(licenseState.isDataFrameAllowed()).thenReturn(true);
|
||||||
|
|
||||||
|
DataFrameFeatureSet featureSet = new DataFrameFeatureSet(Settings.EMPTY, client, licenseState);
|
||||||
|
|
||||||
|
List<DataFrameTransformStateAndStats> transformsStateAndStats = new ArrayList<>();
|
||||||
|
for (int i = 0; i < randomIntBetween(0, 10); ++i) {
|
||||||
|
transformsStateAndStats.add(DataFrameTransformStateAndStatsTests.randomDataFrameTransformStateAndStats());
|
||||||
|
}
|
||||||
|
|
||||||
|
GetDataFrameTransformsStatsAction.Response mockResponse = new GetDataFrameTransformsStatsAction.Response(transformsStateAndStats);
|
||||||
|
|
||||||
|
doAnswer(invocationOnMock -> {
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
ActionListener<Response> listener = (ActionListener<Response>) invocationOnMock.getArguments()[2];
|
||||||
|
listener.onResponse(mockResponse);
|
||||||
|
return Void.TYPE;
|
||||||
|
}).when(client).execute(same(GetDataFrameTransformsStatsAction.INSTANCE), any(), any());
|
||||||
|
|
||||||
|
PlainActionFuture<Usage> future = new PlainActionFuture<>();
|
||||||
|
featureSet.usage(future);
|
||||||
|
XPackFeatureSet.Usage usage = future.get();
|
||||||
|
|
||||||
|
assertTrue(usage.enabled());
|
||||||
|
try (XContentBuilder builder = XContentFactory.jsonBuilder()) {
|
||||||
|
usage.toXContent(builder, ToXContent.EMPTY_PARAMS);
|
||||||
|
|
||||||
|
XContentParser parser = createParser(builder);
|
||||||
|
Map<String, Object> usageAsMap = parser.map();
|
||||||
|
assertTrue((boolean) XContentMapValues.extractValue("available", usageAsMap));
|
||||||
|
|
||||||
|
if (transformsStateAndStats.isEmpty()) {
|
||||||
|
// no transforms, no stats
|
||||||
|
assertEquals(null, XContentMapValues.extractValue("transforms", usageAsMap));
|
||||||
|
assertEquals(null, XContentMapValues.extractValue("stats", usageAsMap));
|
||||||
|
} else {
|
||||||
|
assertEquals(transformsStateAndStats.size(), XContentMapValues.extractValue("transforms._all", usageAsMap));
|
||||||
|
|
||||||
|
Map<String, Integer> stateCounts = new HashMap<>();
|
||||||
|
transformsStateAndStats.stream().map(x -> x.getTransformState().getIndexerState().value())
|
||||||
|
.forEach(x -> stateCounts.merge(x, 1, Integer::sum));
|
||||||
|
stateCounts.forEach((k, v) -> assertEquals(v, XContentMapValues.extractValue("transforms." + k, usageAsMap)));
|
||||||
|
|
||||||
|
DataFrameIndexerTransformStats combinedStats = transformsStateAndStats.stream().map(x -> x.getTransformStats())
|
||||||
|
.reduce((l, r) -> l.merge(r)).get();
|
||||||
|
|
||||||
|
assertEquals(toIntExact(combinedStats.getIndexFailures()),
|
||||||
|
XContentMapValues.extractValue("stats.index_failures", usageAsMap));
|
||||||
|
assertEquals(toIntExact(combinedStats.getIndexTotal()), XContentMapValues.extractValue("stats.index_total", usageAsMap));
|
||||||
|
assertEquals(toIntExact(combinedStats.getSearchTime()),
|
||||||
|
XContentMapValues.extractValue("stats.search_time_in_ms", usageAsMap));
|
||||||
|
assertEquals(toIntExact(combinedStats.getNumDocuments()),
|
||||||
|
XContentMapValues.extractValue("stats.documents_processed", usageAsMap));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testUsageDisabled() throws IOException, InterruptedException, ExecutionException {
|
||||||
|
when(licenseState.isDataFrameAllowed()).thenReturn(true);
|
||||||
|
Settings.Builder settings = Settings.builder();
|
||||||
|
settings.put("xpack.data_frame.enabled", false);
|
||||||
|
DataFrameFeatureSet featureSet = new DataFrameFeatureSet(settings.build(), mock(Client.class), licenseState);
|
||||||
|
PlainActionFuture<Usage> future = new PlainActionFuture<>();
|
||||||
|
featureSet.usage(future);
|
||||||
|
XPackFeatureSet.Usage usage = future.get();
|
||||||
|
|
||||||
|
assertFalse(usage.enabled());
|
||||||
|
try (XContentBuilder builder = XContentFactory.jsonBuilder()) {
|
||||||
|
usage.toXContent(builder, ToXContent.EMPTY_PARAMS);
|
||||||
|
|
||||||
|
XContentParser parser = createParser(builder);
|
||||||
|
Map<String, Object> usageAsMap = parser.map();
|
||||||
|
assertTrue((boolean) XContentMapValues.extractValue("available", usageAsMap));
|
||||||
|
assertFalse((boolean) XContentMapValues.extractValue("enabled", usageAsMap));
|
||||||
|
// not enabled -> no transforms, no stats
|
||||||
|
assertEquals(null, XContentMapValues.extractValue("transforms", usageAsMap));
|
||||||
|
assertEquals(null, XContentMapValues.extractValue("stats", usageAsMap));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,27 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.xpack.dataframe;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.license.XPackLicenseState;
|
||||||
|
import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin;
|
||||||
|
import java.nio.file.Path;
|
||||||
|
|
||||||
|
public class LocalStateDataFrame extends LocalStateCompositeXPackPlugin {
|
||||||
|
|
||||||
|
public LocalStateDataFrame(final Settings settings, final Path configPath) throws Exception {
|
||||||
|
super(settings, configPath);
|
||||||
|
@SuppressWarnings("resource")
|
||||||
|
LocalStateDataFrame thisVar = this;
|
||||||
|
|
||||||
|
plugins.add(new DataFrame(settings) {
|
||||||
|
@Override
|
||||||
|
protected XPackLicenseState getLicenseState() {
|
||||||
|
return thisVar.getLicenseState();
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,40 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable.Reader;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameIndexerTransformStatsTests;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameTransformStateTests;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.AbstractSerializingDataFrameTestCase;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public class DataFrameTransformStateAndStatsTests extends AbstractSerializingDataFrameTestCase<DataFrameTransformStateAndStats> {
|
||||||
|
|
||||||
|
public static DataFrameTransformStateAndStats randomDataFrameTransformStateAndStats() {
|
||||||
|
return new DataFrameTransformStateAndStats(randomAlphaOfLengthBetween(1, 10),
|
||||||
|
DataFrameTransformStateTests.randomDataFrameTransformState(),
|
||||||
|
DataFrameIndexerTransformStatsTests.randomStats());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected DataFrameTransformStateAndStats doParseInstance(XContentParser parser) throws IOException {
|
||||||
|
return DataFrameTransformStateAndStats.PARSER.apply(parser, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected DataFrameTransformStateAndStats createTestInstance() {
|
||||||
|
return randomDataFrameTransformStateAndStats();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Reader<DataFrameTransformStateAndStats> instanceReader() {
|
||||||
|
return DataFrameTransformStateAndStats::new;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,23 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.test.AbstractWireSerializingTestCase;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.DeleteDataFrameTransformAction.Request;
|
||||||
|
|
||||||
|
public class DeleteDataFrameTransformActionRequestTests extends AbstractWireSerializingTestCase<Request> {
|
||||||
|
@Override
|
||||||
|
protected Request createTestInstance() {
|
||||||
|
return new Request(randomAlphaOfLengthBetween(1, 20));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Writeable.Reader<Request> instanceReader() {
|
||||||
|
return Request::new;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,28 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.test.AbstractWireSerializingTestCase;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.GetDataFrameTransformsAction.Request;
|
||||||
|
|
||||||
|
public class GetDataFrameTransformsActionRequestTests extends AbstractWireSerializingTestCase<Request> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Request createTestInstance() {
|
||||||
|
if (randomBoolean()) {
|
||||||
|
return new Request(MetaData.ALL);
|
||||||
|
}
|
||||||
|
return new Request(randomAlphaOfLengthBetween(1, 20));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Writeable.Reader<Request> instanceReader() {
|
||||||
|
return Request::new;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,46 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.logging.LoggerMessageFormat;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||||
|
import org.elasticsearch.test.ESTestCase;
|
||||||
|
import org.elasticsearch.xpack.core.watcher.watch.Payload.XContent;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.GetDataFrameTransformsAction.Response;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformConfig;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformConfigTests;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class GetDataFrameTransformsActionResponseTests extends ESTestCase {
|
||||||
|
|
||||||
|
public void testInvalidTransforms() throws IOException {
|
||||||
|
List<DataFrameTransformConfig> transforms = new ArrayList<>();
|
||||||
|
|
||||||
|
transforms.add(DataFrameTransformConfigTests.randomDataFrameTransformConfig());
|
||||||
|
transforms.add(DataFrameTransformConfigTests.randomInvalidDataFrameTransformConfig());
|
||||||
|
transforms.add(DataFrameTransformConfigTests.randomDataFrameTransformConfig());
|
||||||
|
transforms.add(DataFrameTransformConfigTests.randomInvalidDataFrameTransformConfig());
|
||||||
|
|
||||||
|
Response r = new Response(transforms);
|
||||||
|
XContentBuilder builder = XContentFactory.contentBuilder(randomFrom(XContentType.values()));
|
||||||
|
r.toXContent(builder, XContent.EMPTY_PARAMS);
|
||||||
|
Map<String, Object> responseAsMap = createParser(builder).map();
|
||||||
|
assertEquals(2, XContentMapValues.extractValue("invalid_transforms.count", responseAsMap));
|
||||||
|
List<String> expectedInvalidTransforms = new ArrayList<>();
|
||||||
|
expectedInvalidTransforms.add(transforms.get(1).getId());
|
||||||
|
expectedInvalidTransforms.add(transforms.get(3).getId());
|
||||||
|
assertEquals(expectedInvalidTransforms, XContentMapValues.extractValue("invalid_transforms.transforms", responseAsMap));
|
||||||
|
assertWarnings(LoggerMessageFormat.format(Response.INVALID_TRANSFORMS_DEPRECATION_WARNING, 2));
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,27 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.test.AbstractWireSerializingTestCase;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.GetDataFrameTransformsStatsAction.Request;
|
||||||
|
|
||||||
|
public class GetDataFrameTransformsStatsActionRequestTests extends AbstractWireSerializingTestCase<Request> {
|
||||||
|
@Override
|
||||||
|
protected Request createTestInstance() {
|
||||||
|
if (randomBoolean()) {
|
||||||
|
return new Request(MetaData.ALL);
|
||||||
|
}
|
||||||
|
return new Request(randomAlphaOfLengthBetween(1, 20));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Writeable.Reader<Request> instanceReader() {
|
||||||
|
return Request::new;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,75 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.search.SearchModule;
|
||||||
|
import org.elasticsearch.test.AbstractStreamableXContentTestCase;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.PutDataFrameTransformAction.Request;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformConfig;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformConfigTests;
|
||||||
|
import org.junit.Before;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import static java.util.Collections.emptyList;
|
||||||
|
|
||||||
|
public class PutDataFrameTransformActionRequestTests extends AbstractStreamableXContentTestCase<Request> {
|
||||||
|
|
||||||
|
private String transformId;
|
||||||
|
|
||||||
|
private NamedWriteableRegistry namedWriteableRegistry;
|
||||||
|
private NamedXContentRegistry namedXContentRegistry;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void registerAggregationNamedObjects() throws Exception {
|
||||||
|
// register aggregations as NamedWriteable
|
||||||
|
SearchModule searchModule = new SearchModule(Settings.EMPTY, false, emptyList());
|
||||||
|
namedWriteableRegistry = new NamedWriteableRegistry(searchModule.getNamedWriteables());
|
||||||
|
namedXContentRegistry = new NamedXContentRegistry(searchModule.getNamedXContents());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected NamedWriteableRegistry getNamedWriteableRegistry() {
|
||||||
|
return namedWriteableRegistry;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected NamedXContentRegistry xContentRegistry() {
|
||||||
|
return namedXContentRegistry;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setupTransformId() {
|
||||||
|
transformId = randomAlphaOfLengthBetween(1, 10);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Request doParseInstance(XContentParser parser) throws IOException {
|
||||||
|
return Request.fromXContent(parser, transformId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Request createBlankInstance() {
|
||||||
|
return new Request();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected boolean supportsUnknownFields() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Request createTestInstance() {
|
||||||
|
DataFrameTransformConfig config = DataFrameTransformConfigTests.randomDataFrameTransformConfig();
|
||||||
|
return new Request(config);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,22 @@
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.test.AbstractWireSerializingTestCase;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.StartDataFrameTransformAction.Request;
|
||||||
|
|
||||||
|
public class StartDataFrameTransformActionTests extends AbstractWireSerializingTestCase<Request> {
|
||||||
|
@Override
|
||||||
|
protected Request createTestInstance() {
|
||||||
|
return new Request(randomAlphaOfLengthBetween(1, 20));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Writeable.Reader<Request> instanceReader() {
|
||||||
|
return Request::new;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,37 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
|
import org.elasticsearch.test.AbstractWireSerializingTestCase;
|
||||||
|
import org.elasticsearch.xpack.dataframe.action.StopDataFrameTransformAction.Request;
|
||||||
|
|
||||||
|
public class StopDataFrameTransformActionRequestTests extends AbstractWireSerializingTestCase<Request> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Request createTestInstance() {
|
||||||
|
TimeValue timeout = randomBoolean() ? TimeValue.timeValueMinutes(randomIntBetween(1, 10)) : null;
|
||||||
|
return new Request(randomAlphaOfLengthBetween(1, 10), randomBoolean(), timeout);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Writeable.Reader<Request> instanceReader() {
|
||||||
|
return Request::new;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testSameButDifferentTimeout() {
|
||||||
|
String id = randomAlphaOfLengthBetween(1, 10);
|
||||||
|
boolean waitForCompletion = randomBoolean();
|
||||||
|
|
||||||
|
Request r1 = new Request(id, waitForCompletion, TimeValue.timeValueSeconds(10));
|
||||||
|
Request r2 = new Request(id, waitForCompletion, TimeValue.timeValueSeconds(20));
|
||||||
|
|
||||||
|
assertNotEquals(r1,r2);
|
||||||
|
assertNotEquals(r1.hashCode(),r2.hashCode());
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,75 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.persistence;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.LatchedActionListener;
|
||||||
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
|
import org.elasticsearch.common.CheckedConsumer;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.plugins.Plugin;
|
||||||
|
import org.elasticsearch.test.ESSingleNodeTestCase;
|
||||||
|
import org.elasticsearch.xpack.core.template.TemplateUtils;
|
||||||
|
import org.elasticsearch.xpack.dataframe.LocalStateDataFrame;
|
||||||
|
import org.junit.Before;
|
||||||
|
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.function.Consumer;
|
||||||
|
|
||||||
|
public abstract class DataFrameSingleNodeTestCase extends ESSingleNodeTestCase {
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void waitForTemplates() throws Exception {
|
||||||
|
assertBusy(() -> {
|
||||||
|
ClusterState state = client().admin().cluster().prepareState().get().getState();
|
||||||
|
assertTrue("Timed out waiting for the data frame templates to be installed",
|
||||||
|
TemplateUtils.checkTemplateExistsAndVersionIsGTECurrentVersion(DataFrameInternalIndex.INDEX_TEMPLATE_NAME, state));
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Settings nodeSettings() {
|
||||||
|
Settings.Builder newSettings = Settings.builder();
|
||||||
|
newSettings.put(super.nodeSettings());
|
||||||
|
|
||||||
|
return newSettings.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Collection<Class<? extends Plugin>> getPlugins() {
|
||||||
|
return pluginList(LocalStateDataFrame.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected <T> void assertAsync(Consumer<ActionListener<T>> function, T expected, CheckedConsumer<T, ? extends Exception> onAnswer,
|
||||||
|
Consumer<Exception> onException) throws InterruptedException {
|
||||||
|
|
||||||
|
CountDownLatch latch = new CountDownLatch(1);
|
||||||
|
|
||||||
|
LatchedActionListener<T> listener = new LatchedActionListener<>(ActionListener.wrap(r -> {
|
||||||
|
if (expected == null) {
|
||||||
|
fail("expected an exception but got a response");
|
||||||
|
} else {
|
||||||
|
assertEquals(r, expected);
|
||||||
|
}
|
||||||
|
if (onAnswer != null) {
|
||||||
|
onAnswer.accept(r);
|
||||||
|
}
|
||||||
|
}, e -> {
|
||||||
|
if (onException == null) {
|
||||||
|
fail("got unexpected exception: " + e.getMessage());
|
||||||
|
} else {
|
||||||
|
onException.accept(e);
|
||||||
|
}
|
||||||
|
}), latch);
|
||||||
|
|
||||||
|
function.accept(listener);
|
||||||
|
latch.await(10, TimeUnit.SECONDS);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,105 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.persistence;
|
||||||
|
|
||||||
|
import org.elasticsearch.ResourceAlreadyExistsException;
|
||||||
|
import org.elasticsearch.ResourceNotFoundException;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.DataFrameMessages;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformConfig;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformConfigTests;
|
||||||
|
import org.junit.Before;
|
||||||
|
|
||||||
|
public class DataFrameTransformsConfigManagerTests extends DataFrameSingleNodeTestCase {
|
||||||
|
|
||||||
|
private DataFrameTransformsConfigManager transformsConfigManager;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void createComponents() {
|
||||||
|
transformsConfigManager = new DataFrameTransformsConfigManager(client(), xContentRegistry());
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testGetMissingTransform() throws InterruptedException {
|
||||||
|
// the index does not exist yet
|
||||||
|
assertAsync(listener -> transformsConfigManager.getTransformConfiguration("not_there", listener), (DataFrameTransformConfig) null,
|
||||||
|
null, e -> {
|
||||||
|
assertEquals(ResourceNotFoundException.class, e.getClass());
|
||||||
|
assertEquals(DataFrameMessages.getMessage(DataFrameMessages.REST_DATA_FRAME_UNKNOWN_TRANSFORM, "not_there"),
|
||||||
|
e.getMessage());
|
||||||
|
});
|
||||||
|
|
||||||
|
// create one transform and test with an existing index
|
||||||
|
assertAsync(
|
||||||
|
listener -> transformsConfigManager
|
||||||
|
.putTransformConfiguration(DataFrameTransformConfigTests.randomDataFrameTransformConfig(), listener),
|
||||||
|
true, null, null);
|
||||||
|
|
||||||
|
// same test, but different code path
|
||||||
|
assertAsync(listener -> transformsConfigManager.getTransformConfiguration("not_there", listener), (DataFrameTransformConfig) null,
|
||||||
|
null, e -> {
|
||||||
|
assertEquals(ResourceNotFoundException.class, e.getClass());
|
||||||
|
assertEquals(DataFrameMessages.getMessage(DataFrameMessages.REST_DATA_FRAME_UNKNOWN_TRANSFORM, "not_there"),
|
||||||
|
e.getMessage());
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testDeleteMissingTransform() throws InterruptedException {
|
||||||
|
// the index does not exist yet
|
||||||
|
assertAsync(listener -> transformsConfigManager.deleteTransformConfiguration("not_there", listener), (Boolean) null, null, e -> {
|
||||||
|
assertEquals(ResourceNotFoundException.class, e.getClass());
|
||||||
|
assertEquals(DataFrameMessages.getMessage(DataFrameMessages.REST_DATA_FRAME_UNKNOWN_TRANSFORM, "not_there"), e.getMessage());
|
||||||
|
});
|
||||||
|
|
||||||
|
// create one transform and test with an existing index
|
||||||
|
assertAsync(
|
||||||
|
listener -> transformsConfigManager
|
||||||
|
.putTransformConfiguration(DataFrameTransformConfigTests.randomDataFrameTransformConfig(), listener),
|
||||||
|
true, null, null);
|
||||||
|
|
||||||
|
// same test, but different code path
|
||||||
|
assertAsync(listener -> transformsConfigManager.deleteTransformConfiguration("not_there", listener), (Boolean) null, null, e -> {
|
||||||
|
assertEquals(ResourceNotFoundException.class, e.getClass());
|
||||||
|
assertEquals(DataFrameMessages.getMessage(DataFrameMessages.REST_DATA_FRAME_UNKNOWN_TRANSFORM, "not_there"), e.getMessage());
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testCreateReadDelete() throws InterruptedException {
|
||||||
|
DataFrameTransformConfig transformConfig = DataFrameTransformConfigTests.randomDataFrameTransformConfig();
|
||||||
|
|
||||||
|
// create transform
|
||||||
|
assertAsync(listener -> transformsConfigManager.putTransformConfiguration(transformConfig, listener), true, null, null);
|
||||||
|
|
||||||
|
// read transform
|
||||||
|
assertAsync(listener -> transformsConfigManager.getTransformConfiguration(transformConfig.getId(), listener), transformConfig, null,
|
||||||
|
null);
|
||||||
|
|
||||||
|
// try to create again
|
||||||
|
assertAsync(listener -> transformsConfigManager.putTransformConfiguration(transformConfig, listener), (Boolean) null, null, e -> {
|
||||||
|
assertEquals(ResourceAlreadyExistsException.class, e.getClass());
|
||||||
|
assertEquals(DataFrameMessages.getMessage(DataFrameMessages.REST_PUT_DATA_FRAME_TRANSFORM_EXISTS, transformConfig.getId()),
|
||||||
|
e.getMessage());
|
||||||
|
});
|
||||||
|
|
||||||
|
// delete transform
|
||||||
|
assertAsync(listener -> transformsConfigManager.deleteTransformConfiguration(transformConfig.getId(), listener), true, null, null);
|
||||||
|
|
||||||
|
// delete again
|
||||||
|
assertAsync(listener -> transformsConfigManager.deleteTransformConfiguration(transformConfig.getId(), listener), (Boolean) null,
|
||||||
|
null, e -> {
|
||||||
|
assertEquals(ResourceNotFoundException.class, e.getClass());
|
||||||
|
assertEquals(DataFrameMessages.getMessage(DataFrameMessages.REST_DATA_FRAME_UNKNOWN_TRANSFORM, transformConfig.getId()),
|
||||||
|
e.getMessage());
|
||||||
|
});
|
||||||
|
|
||||||
|
// try to get deleted transform
|
||||||
|
assertAsync(listener -> transformsConfigManager.getTransformConfiguration(transformConfig.getId(), listener),
|
||||||
|
(DataFrameTransformConfig) null, null, e -> {
|
||||||
|
assertEquals(ResourceNotFoundException.class, e.getClass());
|
||||||
|
assertEquals(DataFrameMessages.getMessage(DataFrameMessages.REST_DATA_FRAME_UNKNOWN_TRANSFORM, transformConfig.getId()),
|
||||||
|
e.getMessage());
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,65 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.ParseField;
|
||||||
|
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.index.query.QueryBuilder;
|
||||||
|
import org.elasticsearch.search.SearchModule;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.BaseAggregationBuilder;
|
||||||
|
import org.elasticsearch.test.AbstractSerializingTestCase;
|
||||||
|
import org.junit.Before;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import static java.util.Collections.emptyList;
|
||||||
|
|
||||||
|
public abstract class AbstractSerializingDataFrameTestCase<T extends ToXContent & Writeable>
|
||||||
|
extends AbstractSerializingTestCase<T> {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test case that ensures aggregation named objects are registered
|
||||||
|
*/
|
||||||
|
private NamedWriteableRegistry namedWriteableRegistry;
|
||||||
|
private NamedXContentRegistry namedXContentRegistry;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void registerAggregationNamedObjects() throws Exception {
|
||||||
|
// register aggregations as NamedWriteable
|
||||||
|
SearchModule searchModule = new SearchModule(Settings.EMPTY, false, emptyList());
|
||||||
|
|
||||||
|
List<NamedWriteableRegistry.Entry> namedWriteables = searchModule.getNamedWriteables();
|
||||||
|
namedWriteables.add(new NamedWriteableRegistry.Entry(QueryBuilder.class, MockDeprecatedQueryBuilder.NAME,
|
||||||
|
MockDeprecatedQueryBuilder::new));
|
||||||
|
namedWriteables.add(new NamedWriteableRegistry.Entry(AggregationBuilder.class, MockDeprecatedAggregationBuilder.NAME,
|
||||||
|
MockDeprecatedAggregationBuilder::new));
|
||||||
|
|
||||||
|
List<NamedXContentRegistry.Entry> namedXContents = searchModule.getNamedXContents();
|
||||||
|
namedXContents.add(new NamedXContentRegistry.Entry(QueryBuilder.class,
|
||||||
|
new ParseField(MockDeprecatedQueryBuilder.NAME), (p, c) -> MockDeprecatedQueryBuilder.fromXContent(p)));
|
||||||
|
namedXContents.add(new NamedXContentRegistry.Entry(BaseAggregationBuilder.class,
|
||||||
|
new ParseField(MockDeprecatedAggregationBuilder.NAME), (p, c) -> MockDeprecatedAggregationBuilder.fromXContent(p)));
|
||||||
|
|
||||||
|
namedWriteableRegistry = new NamedWriteableRegistry(namedWriteables);
|
||||||
|
namedXContentRegistry = new NamedXContentRegistry(namedXContents);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected NamedWriteableRegistry getNamedWriteableRegistry() {
|
||||||
|
return namedWriteableRegistry;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected NamedXContentRegistry xContentRegistry() {
|
||||||
|
return namedXContentRegistry;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.Strings;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable.Reader;
|
||||||
|
import org.elasticsearch.common.xcontent.DeprecationHandler;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.pivot.PivotConfigTests;
|
||||||
|
import org.junit.Before;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import static org.elasticsearch.test.TestMatchers.matchesPattern;
|
||||||
|
|
||||||
|
public class DataFrameTransformConfigTests extends AbstractSerializingDataFrameTestCase<DataFrameTransformConfig> {
|
||||||
|
|
||||||
|
private String transformId;
|
||||||
|
|
||||||
|
public static DataFrameTransformConfig randomDataFrameTransformConfig() {
|
||||||
|
return new DataFrameTransformConfig(randomAlphaOfLengthBetween(1, 10), randomAlphaOfLengthBetween(1, 10),
|
||||||
|
randomAlphaOfLengthBetween(1, 10), QueryConfigTests.randomQueryConfig(), PivotConfigTests.randomPivotConfig());
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DataFrameTransformConfig randomInvalidDataFrameTransformConfig() {
|
||||||
|
if (randomBoolean()) {
|
||||||
|
return new DataFrameTransformConfig(randomAlphaOfLengthBetween(1, 10), randomAlphaOfLengthBetween(1, 10),
|
||||||
|
randomAlphaOfLengthBetween(1, 10), QueryConfigTests.randomInvalidQueryConfig(), PivotConfigTests.randomPivotConfig());
|
||||||
|
} // else
|
||||||
|
return new DataFrameTransformConfig(randomAlphaOfLengthBetween(1, 10), randomAlphaOfLengthBetween(1, 10),
|
||||||
|
randomAlphaOfLengthBetween(1, 10), QueryConfigTests.randomQueryConfig(), PivotConfigTests.randomInvalidPivotConfig());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUpOptionalId() {
|
||||||
|
transformId = randomAlphaOfLengthBetween(1, 10);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected DataFrameTransformConfig doParseInstance(XContentParser parser) throws IOException {
|
||||||
|
if (randomBoolean()) {
|
||||||
|
return DataFrameTransformConfig.fromXContent(parser, transformId, false);
|
||||||
|
} else {
|
||||||
|
return DataFrameTransformConfig.fromXContent(parser, null, false);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected DataFrameTransformConfig createTestInstance() {
|
||||||
|
return randomDataFrameTransformConfig();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Reader<DataFrameTransformConfig> instanceReader() {
|
||||||
|
return DataFrameTransformConfig::new;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testDefaultMatchAll( ) throws IOException {
|
||||||
|
String pivotTransform = "{"
|
||||||
|
+ " \"source\" : \"src\","
|
||||||
|
+ " \"dest\" : \"dest\","
|
||||||
|
+ " \"pivot\" : {"
|
||||||
|
+ " \"group_by\": {"
|
||||||
|
+ " \"id\": {"
|
||||||
|
+ " \"terms\": {"
|
||||||
|
+ " \"field\": \"id\""
|
||||||
|
+ "} } },"
|
||||||
|
+ " \"aggs\": {"
|
||||||
|
+ " \"avg\": {"
|
||||||
|
+ " \"avg\": {"
|
||||||
|
+ " \"field\": \"points\""
|
||||||
|
+ "} } } } }";
|
||||||
|
|
||||||
|
DataFrameTransformConfig dataFrameTransformConfig = createDataFrameTransformConfigFromString(pivotTransform, "test_match_all");
|
||||||
|
assertNotNull(dataFrameTransformConfig.getQueryConfig());
|
||||||
|
assertTrue(dataFrameTransformConfig.getQueryConfig().isValid());
|
||||||
|
|
||||||
|
try (XContentBuilder xContentBuilder = XContentFactory.jsonBuilder()) {
|
||||||
|
XContentBuilder content = dataFrameTransformConfig.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS);
|
||||||
|
String pivotTransformWithIdAndDefaults = Strings.toString(content);
|
||||||
|
|
||||||
|
assertThat(pivotTransformWithIdAndDefaults, matchesPattern(".*\"match_all\"\\s*:\\s*\\{\\}.*"));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private DataFrameTransformConfig createDataFrameTransformConfigFromString(String json, String id) throws IOException {
|
||||||
|
final XContentParser parser = XContentType.JSON.xContent().createParser(xContentRegistry(),
|
||||||
|
DeprecationHandler.THROW_UNSUPPORTED_OPERATION, json);
|
||||||
|
return DataFrameTransformConfig.fromXContent(parser, id, false);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,92 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.logging.DeprecationLogger;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||||
|
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||||
|
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||||
|
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||||
|
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||||
|
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
||||||
|
import org.elasticsearch.search.internal.SearchContext;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class MockDeprecatedAggregationBuilder extends ValuesSourceAggregationBuilder<ValuesSource, MockDeprecatedAggregationBuilder> {
|
||||||
|
|
||||||
|
public static final String NAME = "deprecated_agg";
|
||||||
|
public static final String DEPRECATION_MESSAGE = "expected deprecation message from MockDeprecatedAggregationBuilder";
|
||||||
|
|
||||||
|
private static final DeprecationLogger deprecationLogger = new DeprecationLogger(
|
||||||
|
LogManager.getLogger(MockDeprecatedAggregationBuilder.class));
|
||||||
|
|
||||||
|
protected MockDeprecatedAggregationBuilder(MockDeprecatedAggregationBuilder clone, Builder factoriesBuilder,
|
||||||
|
Map<String, Object> metaData) {
|
||||||
|
super(clone, factoriesBuilder, metaData);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map<String, Object> metaData) {
|
||||||
|
return new MockDeprecatedAggregationBuilder(this, factoriesBuilder, metaData);
|
||||||
|
}
|
||||||
|
|
||||||
|
public MockDeprecatedAggregationBuilder() {
|
||||||
|
super(NAME, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Read from a stream.
|
||||||
|
*/
|
||||||
|
protected MockDeprecatedAggregationBuilder(StreamInput in) throws IOException {
|
||||||
|
super(in, null, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getType() {
|
||||||
|
return NAME;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource> config,
|
||||||
|
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected int innerHashCode() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected boolean innerEquals(Object obj) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MockDeprecatedAggregationBuilder fromXContent(XContentParser p) {
|
||||||
|
deprecationLogger.deprecatedAndMaybeLog("deprecated_mock", DEPRECATION_MESSAGE);
|
||||||
|
return new MockDeprecatedAggregationBuilder();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,88 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.apache.lucene.search.Query;
|
||||||
|
import org.elasticsearch.common.ParsingException;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.logging.DeprecationLogger;
|
||||||
|
import org.elasticsearch.common.lucene.search.Queries;
|
||||||
|
import org.elasticsearch.common.xcontent.ObjectParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.index.query.AbstractQueryBuilder;
|
||||||
|
import org.elasticsearch.index.query.QueryShardContext;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Utility test class to write a deprecation message on usage
|
||||||
|
*/
|
||||||
|
public class MockDeprecatedQueryBuilder extends AbstractQueryBuilder<MockDeprecatedQueryBuilder> {
|
||||||
|
|
||||||
|
public static final String NAME = "deprecated_match_all";
|
||||||
|
public static final String DEPRECATION_MESSAGE = "expected deprecation message from MockDeprecatedQueryBuilder";
|
||||||
|
|
||||||
|
private static final DeprecationLogger deprecationLogger = new DeprecationLogger(
|
||||||
|
LogManager.getLogger(MockDeprecatedQueryBuilder.class));
|
||||||
|
|
||||||
|
private static final ObjectParser<MockDeprecatedQueryBuilder, Void> PARSER = new ObjectParser<>(NAME, MockDeprecatedQueryBuilder::new);
|
||||||
|
|
||||||
|
static {
|
||||||
|
declareStandardFields(PARSER);
|
||||||
|
}
|
||||||
|
|
||||||
|
public MockDeprecatedQueryBuilder() {
|
||||||
|
}
|
||||||
|
|
||||||
|
public MockDeprecatedQueryBuilder(StreamInput in) throws IOException {
|
||||||
|
super(in);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MockDeprecatedQueryBuilder fromXContent(XContentParser parser) {
|
||||||
|
try {
|
||||||
|
deprecationLogger.deprecatedAndMaybeLog("deprecated_mock", DEPRECATION_MESSAGE);
|
||||||
|
|
||||||
|
return PARSER.apply(parser, null);
|
||||||
|
} catch (IllegalArgumentException e) {
|
||||||
|
throw new ParsingException(parser.getTokenLocation(), e.getMessage(), e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getWriteableName() {
|
||||||
|
return NAME;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void doXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject(NAME);
|
||||||
|
printBoostAndQueryName(builder);
|
||||||
|
builder.endObject();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Query doToQuery(QueryShardContext context) throws IOException {
|
||||||
|
return Queries.newMatchAllQuery();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected boolean doEquals(MockDeprecatedQueryBuilder other) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected int doHashCode() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,142 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.ParsingException;
|
||||||
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable.Reader;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
||||||
|
import org.elasticsearch.index.query.MatchAllQueryBuilder;
|
||||||
|
import org.elasticsearch.index.query.MatchNoneQueryBuilder;
|
||||||
|
import org.elasticsearch.index.query.MatchQueryBuilder;
|
||||||
|
import org.elasticsearch.index.query.QueryBuilder;
|
||||||
|
import org.junit.Before;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
|
|
||||||
|
public class QueryConfigTests extends AbstractSerializingDataFrameTestCase<QueryConfig> {
|
||||||
|
|
||||||
|
private boolean lenient;
|
||||||
|
|
||||||
|
public static QueryConfig randomQueryConfig() {
|
||||||
|
|
||||||
|
QueryBuilder queryBuilder = randomBoolean() ? new MatchAllQueryBuilder() : new MatchNoneQueryBuilder();
|
||||||
|
LinkedHashMap<String, Object> source = null;
|
||||||
|
|
||||||
|
try (XContentBuilder xContentBuilder = XContentFactory.jsonBuilder()) {
|
||||||
|
XContentBuilder content = queryBuilder.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS);
|
||||||
|
source = (LinkedHashMap<String, Object>) XContentHelper.convertToMap(BytesReference.bytes(content), true, XContentType.JSON)
|
||||||
|
.v2();
|
||||||
|
} catch (IOException e) {
|
||||||
|
// should not happen
|
||||||
|
fail("failed to create random query config");
|
||||||
|
}
|
||||||
|
|
||||||
|
return new QueryConfig(source, queryBuilder);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static QueryConfig randomInvalidQueryConfig() {
|
||||||
|
// create something broken but with a source
|
||||||
|
LinkedHashMap<String, Object> source = new LinkedHashMap<>();
|
||||||
|
for (String key : randomUnique(() -> randomAlphaOfLengthBetween(1, 20), randomIntBetween(1, 10))) {
|
||||||
|
source.put(key, randomAlphaOfLengthBetween(1, 20));
|
||||||
|
}
|
||||||
|
|
||||||
|
return new QueryConfig(source, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setRandomFeatures() {
|
||||||
|
lenient = randomBoolean();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected QueryConfig doParseInstance(XContentParser parser) throws IOException {
|
||||||
|
return QueryConfig.fromXContent(parser, lenient);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected QueryConfig createTestInstance() {
|
||||||
|
return lenient ? randomBoolean() ? randomQueryConfig() : randomInvalidQueryConfig() : randomQueryConfig();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Reader<QueryConfig> instanceReader() {
|
||||||
|
return QueryConfig::new;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testValidQueryParsing() throws IOException {
|
||||||
|
QueryBuilder query = new MatchQueryBuilder("key", "value");
|
||||||
|
String source = query.toString();
|
||||||
|
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
|
||||||
|
QueryConfig queryConfig = QueryConfig.fromXContent(parser, true);
|
||||||
|
assertEquals(query, queryConfig.getQuery());
|
||||||
|
assertTrue(queryConfig.isValid());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testFailOnStrictPassOnLenient() throws IOException {
|
||||||
|
String source = "{\"query_element_does_not_exist\" : {}}";
|
||||||
|
|
||||||
|
// lenient, passes but reports invalid
|
||||||
|
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
|
||||||
|
QueryConfig query = QueryConfig.fromXContent(parser, true);
|
||||||
|
assertFalse(query.isValid());
|
||||||
|
}
|
||||||
|
|
||||||
|
// strict throws
|
||||||
|
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
|
||||||
|
expectThrows(ParsingException.class, () -> QueryConfig.fromXContent(parser, false));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testFailOnEmptyQuery() throws IOException {
|
||||||
|
String source = "";
|
||||||
|
|
||||||
|
// lenient, passes but reports invalid
|
||||||
|
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
|
||||||
|
QueryConfig query = QueryConfig.fromXContent(parser, true);
|
||||||
|
assertFalse(query.isValid());
|
||||||
|
}
|
||||||
|
|
||||||
|
// strict throws
|
||||||
|
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
|
||||||
|
expectThrows(IllegalArgumentException.class, () -> QueryConfig.fromXContent(parser, false));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testFailOnEmptyQueryClause() throws IOException {
|
||||||
|
String source = "{}";
|
||||||
|
|
||||||
|
// lenient, passes but reports invalid
|
||||||
|
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
|
||||||
|
QueryConfig query = QueryConfig.fromXContent(parser, true);
|
||||||
|
assertFalse(query.isValid());
|
||||||
|
}
|
||||||
|
|
||||||
|
// strict throws
|
||||||
|
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
|
||||||
|
expectThrows(IllegalArgumentException.class, () -> QueryConfig.fromXContent(parser, false));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testDeprecation() throws IOException {
|
||||||
|
String source = "{\"" + MockDeprecatedQueryBuilder.NAME + "\" : {}}";
|
||||||
|
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
|
||||||
|
QueryConfig query = QueryConfig.fromXContent(parser, false);
|
||||||
|
assertTrue(query.isValid());
|
||||||
|
assertWarnings(MockDeprecatedQueryBuilder.DEPRECATION_MESSAGE);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,150 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable.Reader;
|
||||||
|
import org.elasticsearch.common.xcontent.NamedObjectNotFoundException;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationBuilders;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.AbstractSerializingDataFrameTestCase;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.MockDeprecatedAggregationBuilder;
|
||||||
|
import org.junit.Before;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
public class AggregationConfigTests extends AbstractSerializingDataFrameTestCase<AggregationConfig> {
|
||||||
|
|
||||||
|
private boolean lenient;
|
||||||
|
|
||||||
|
public static AggregationConfig randomAggregationConfig() {
|
||||||
|
|
||||||
|
AggregatorFactories.Builder builder = new AggregatorFactories.Builder();
|
||||||
|
Map<String, Object> source = null;
|
||||||
|
|
||||||
|
// ensure that the unlikely does not happen: 2 aggs share the same name
|
||||||
|
Set<String> names = new HashSet<>();
|
||||||
|
for (int i = 0; i < randomIntBetween(1, 20); ++i) {
|
||||||
|
AggregationBuilder aggBuilder = getRandomSupportedAggregation();
|
||||||
|
if (names.add(aggBuilder.getName())) {
|
||||||
|
builder.addAggregator(aggBuilder);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
try (XContentBuilder xContentBuilder = XContentFactory.jsonBuilder()) {
|
||||||
|
|
||||||
|
XContentBuilder content = builder.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS);
|
||||||
|
source = XContentHelper.convertToMap(BytesReference.bytes(content), true, XContentType.JSON).v2();
|
||||||
|
} catch (IOException e) {
|
||||||
|
fail("failed to create random aggregation config: " + e.getMessage());
|
||||||
|
}
|
||||||
|
|
||||||
|
return new AggregationConfig(source, builder);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static AggregationConfig randomInvalidAggregationConfig() {
|
||||||
|
// create something broken but with a source
|
||||||
|
Map<String, Object> source = new LinkedHashMap<>();
|
||||||
|
for (String key : randomUnique(() -> randomAlphaOfLengthBetween(1, 20), randomIntBetween(1, 10))) {
|
||||||
|
source.put(key, randomAlphaOfLengthBetween(1, 20));
|
||||||
|
}
|
||||||
|
|
||||||
|
return new AggregationConfig(source, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setRandomFeatures() {
|
||||||
|
lenient = randomBoolean();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected AggregationConfig doParseInstance(XContentParser parser) throws IOException {
|
||||||
|
return AggregationConfig.fromXContent(parser, lenient);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected AggregationConfig createTestInstance() {
|
||||||
|
return lenient ? randomBoolean() ? randomAggregationConfig() : randomInvalidAggregationConfig() : randomAggregationConfig();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Reader<AggregationConfig> instanceReader() {
|
||||||
|
return AggregationConfig::new;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testEmptyAggregation() throws IOException {
|
||||||
|
String source = "{}";
|
||||||
|
|
||||||
|
// lenient, passes but reports invalid
|
||||||
|
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
|
||||||
|
AggregationConfig aggregationConfig = AggregationConfig.fromXContent(parser, true);
|
||||||
|
assertFalse(aggregationConfig.isValid());
|
||||||
|
}
|
||||||
|
|
||||||
|
// strict throws
|
||||||
|
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
|
||||||
|
expectThrows(IllegalArgumentException.class, () -> AggregationConfig.fromXContent(parser, false));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testFailOnStrictPassOnLenient() throws IOException {
|
||||||
|
String source = "{\n" +
|
||||||
|
" \"avg_rating\": { \"some_removed_agg\": { \"field\": \"rating\" } }\n" +
|
||||||
|
" },\n" +
|
||||||
|
" {\n" +
|
||||||
|
" \"max_rating\": { \"max_rating\" : { \"field\" : \"rating\" } }\n" +
|
||||||
|
" }";
|
||||||
|
|
||||||
|
// lenient, passes but reports invalid
|
||||||
|
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
|
||||||
|
AggregationConfig aggregationConfig = AggregationConfig.fromXContent(parser, true);
|
||||||
|
assertFalse(aggregationConfig.isValid());
|
||||||
|
}
|
||||||
|
|
||||||
|
// strict throws
|
||||||
|
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
|
||||||
|
expectThrows(NamedObjectNotFoundException.class, () -> AggregationConfig.fromXContent(parser, false));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testDeprecation() throws IOException {
|
||||||
|
String source = "{\"dep_agg\": {\"" + MockDeprecatedAggregationBuilder.NAME + "\" : {}}}";
|
||||||
|
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
|
||||||
|
AggregationConfig agg = AggregationConfig.fromXContent(parser, false);
|
||||||
|
assertTrue(agg.isValid());
|
||||||
|
assertWarnings(MockDeprecatedAggregationBuilder.DEPRECATION_MESSAGE);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static AggregationBuilder getRandomSupportedAggregation() {
|
||||||
|
final int numberOfSupportedAggs = 4;
|
||||||
|
switch (randomIntBetween(1, numberOfSupportedAggs)) {
|
||||||
|
case 1:
|
||||||
|
return AggregationBuilders.avg(randomAlphaOfLengthBetween(1, 10)).field(randomAlphaOfLengthBetween(1, 10));
|
||||||
|
case 2:
|
||||||
|
return AggregationBuilders.min(randomAlphaOfLengthBetween(1, 10)).field(randomAlphaOfLengthBetween(1, 10));
|
||||||
|
case 3:
|
||||||
|
return AggregationBuilders.max(randomAlphaOfLengthBetween(1, 10)).field(randomAlphaOfLengthBetween(1, 10));
|
||||||
|
case 4:
|
||||||
|
return AggregationBuilders.sum(randomAlphaOfLengthBetween(1, 10)).field(randomAlphaOfLengthBetween(1, 10));
|
||||||
|
}
|
||||||
|
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,323 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.ParseField;
|
||||||
|
import org.elasticsearch.common.xcontent.ContextParser;
|
||||||
|
import org.elasticsearch.common.xcontent.DeprecationHandler;
|
||||||
|
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregation;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationBuilders;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.composite.ParsedComposite;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.terms.DoubleTerms;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.terms.LongTerms;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.terms.ParsedDoubleTerms;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.terms.ParsedLongTerms;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.terms.ParsedStringTerms;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.terms.StringTerms;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.CardinalityAggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.ExtendedStatsAggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.MinAggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.ParsedAvg;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.ParsedCardinality;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.ParsedExtendedStats;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.ParsedMax;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.ParsedMin;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.ParsedStats;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.ParsedSum;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.ParsedValueCount;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.StatsAggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.pipeline.ParsedStatsBucket;
|
||||||
|
import org.elasticsearch.search.aggregations.pipeline.StatsBucketPipelineAggregationBuilder;
|
||||||
|
import org.elasticsearch.test.ESTestCase;
|
||||||
|
import org.elasticsearch.xpack.core.dataframe.transform.DataFrameIndexerTransformStats;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import static java.util.Arrays.asList;
|
||||||
|
|
||||||
|
public class AggregationResultUtilsTests extends ESTestCase {
|
||||||
|
|
||||||
|
private final NamedXContentRegistry namedXContentRegistry = new NamedXContentRegistry(namedXContents);
|
||||||
|
|
||||||
|
private final String KEY = Aggregation.CommonFields.KEY.getPreferredName();
|
||||||
|
private final String DOC_COUNT = Aggregation.CommonFields.DOC_COUNT.getPreferredName();
|
||||||
|
|
||||||
|
// aggregations potentially useful for writing tests, to be expanded as necessary
|
||||||
|
private static final List<NamedXContentRegistry.Entry> namedXContents;
|
||||||
|
static {
|
||||||
|
Map<String, ContextParser<Object, ? extends Aggregation>> map = new HashMap<>();
|
||||||
|
map.put(CardinalityAggregationBuilder.NAME, (p, c) -> ParsedCardinality.fromXContent(p, (String) c));
|
||||||
|
map.put(MinAggregationBuilder.NAME, (p, c) -> ParsedMin.fromXContent(p, (String) c));
|
||||||
|
map.put(MaxAggregationBuilder.NAME, (p, c) -> ParsedMax.fromXContent(p, (String) c));
|
||||||
|
map.put(SumAggregationBuilder.NAME, (p, c) -> ParsedSum.fromXContent(p, (String) c));
|
||||||
|
map.put(AvgAggregationBuilder.NAME, (p, c) -> ParsedAvg.fromXContent(p, (String) c));
|
||||||
|
map.put(ValueCountAggregationBuilder.NAME, (p, c) -> ParsedValueCount.fromXContent(p, (String) c));
|
||||||
|
map.put(StatsAggregationBuilder.NAME, (p, c) -> ParsedStats.fromXContent(p, (String) c));
|
||||||
|
map.put(StatsBucketPipelineAggregationBuilder.NAME, (p, c) -> ParsedStatsBucket.fromXContent(p, (String) c));
|
||||||
|
map.put(ExtendedStatsAggregationBuilder.NAME, (p, c) -> ParsedExtendedStats.fromXContent(p, (String) c));
|
||||||
|
map.put(StringTerms.NAME, (p, c) -> ParsedStringTerms.fromXContent(p, (String) c));
|
||||||
|
map.put(LongTerms.NAME, (p, c) -> ParsedLongTerms.fromXContent(p, (String) c));
|
||||||
|
map.put(DoubleTerms.NAME, (p, c) -> ParsedDoubleTerms.fromXContent(p, (String) c));
|
||||||
|
|
||||||
|
namedXContents = map.entrySet().stream()
|
||||||
|
.map(entry -> new NamedXContentRegistry.Entry(Aggregation.class, new ParseField(entry.getKey()), entry.getValue()))
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected NamedXContentRegistry xContentRegistry() {
|
||||||
|
return namedXContentRegistry;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testExtractCompositeAggregationResults() throws IOException {
|
||||||
|
String targetField = randomAlphaOfLengthBetween(5, 10);
|
||||||
|
|
||||||
|
GroupConfig groupBy = parseGroupConfig("{ \"" + targetField + "\" : {"
|
||||||
|
+ "\"terms\" : {"
|
||||||
|
+ " \"field\" : \"doesn't_matter_for_this_test\""
|
||||||
|
+ "} } }");
|
||||||
|
|
||||||
|
String aggName = randomAlphaOfLengthBetween(5, 10);
|
||||||
|
String aggTypedName = "avg#" + aggName;
|
||||||
|
Collection<AggregationBuilder> aggregationBuilders = Collections.singletonList(AggregationBuilders.avg(aggName));
|
||||||
|
|
||||||
|
Map<String, Object> input = asMap(
|
||||||
|
"buckets",
|
||||||
|
asList(
|
||||||
|
asMap(
|
||||||
|
KEY, asMap(
|
||||||
|
targetField, "ID1"),
|
||||||
|
aggTypedName, asMap(
|
||||||
|
"value", 42.33),
|
||||||
|
DOC_COUNT, 8),
|
||||||
|
asMap(
|
||||||
|
KEY, asMap(
|
||||||
|
targetField, "ID2"),
|
||||||
|
aggTypedName, asMap(
|
||||||
|
"value", 28.99),
|
||||||
|
DOC_COUNT, 3),
|
||||||
|
asMap(
|
||||||
|
KEY, asMap(
|
||||||
|
targetField, "ID3"),
|
||||||
|
aggTypedName, asMap(
|
||||||
|
"value", 12.55),
|
||||||
|
DOC_COUNT, 9)
|
||||||
|
));
|
||||||
|
|
||||||
|
List<Map<String, Object>> expected = asList(
|
||||||
|
asMap(
|
||||||
|
targetField, "ID1",
|
||||||
|
aggName, 42.33
|
||||||
|
),
|
||||||
|
asMap(
|
||||||
|
targetField, "ID2",
|
||||||
|
aggName, 28.99
|
||||||
|
),
|
||||||
|
asMap(
|
||||||
|
targetField, "ID3",
|
||||||
|
aggName, 12.55
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
executeTest(groupBy, aggregationBuilders, input, expected, 20);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testExtractCompositeAggregationResultsMultiSources() throws IOException {
|
||||||
|
String targetField = randomAlphaOfLengthBetween(5, 10);
|
||||||
|
String targetField2 = randomAlphaOfLengthBetween(5, 10) + "_2";
|
||||||
|
|
||||||
|
GroupConfig groupBy = parseGroupConfig("{"
|
||||||
|
+ "\"" + targetField + "\" : {"
|
||||||
|
+ " \"terms\" : {"
|
||||||
|
+ " \"field\" : \"doesn't_matter_for_this_test\""
|
||||||
|
+ " } },"
|
||||||
|
+ "\"" + targetField2 + "\" : {"
|
||||||
|
+ " \"terms\" : {"
|
||||||
|
+ " \"field\" : \"doesn't_matter_for_this_test\""
|
||||||
|
+ " } }"
|
||||||
|
+ "}");
|
||||||
|
|
||||||
|
String aggName = randomAlphaOfLengthBetween(5, 10);
|
||||||
|
String aggTypedName = "avg#" + aggName;
|
||||||
|
Collection<AggregationBuilder> aggregationBuilders = Collections.singletonList(AggregationBuilders.avg(aggName));
|
||||||
|
|
||||||
|
Map<String, Object> input = asMap(
|
||||||
|
"buckets",
|
||||||
|
asList(
|
||||||
|
asMap(
|
||||||
|
KEY, asMap(
|
||||||
|
targetField, "ID1",
|
||||||
|
targetField2, "ID1_2"
|
||||||
|
),
|
||||||
|
aggTypedName, asMap(
|
||||||
|
"value", 42.33),
|
||||||
|
DOC_COUNT, 1),
|
||||||
|
asMap(
|
||||||
|
KEY, asMap(
|
||||||
|
targetField, "ID1",
|
||||||
|
targetField2, "ID2_2"
|
||||||
|
),
|
||||||
|
aggTypedName, asMap(
|
||||||
|
"value", 8.4),
|
||||||
|
DOC_COUNT, 2),
|
||||||
|
asMap(
|
||||||
|
KEY, asMap(
|
||||||
|
targetField, "ID2",
|
||||||
|
targetField2, "ID1_2"
|
||||||
|
),
|
||||||
|
aggTypedName, asMap(
|
||||||
|
"value", 28.99),
|
||||||
|
DOC_COUNT, 3),
|
||||||
|
asMap(
|
||||||
|
KEY, asMap(
|
||||||
|
targetField, "ID3",
|
||||||
|
targetField2, "ID2_2"
|
||||||
|
),
|
||||||
|
aggTypedName, asMap(
|
||||||
|
"value", 12.55),
|
||||||
|
DOC_COUNT, 4)
|
||||||
|
));
|
||||||
|
|
||||||
|
List<Map<String, Object>> expected = asList(
|
||||||
|
asMap(
|
||||||
|
targetField, "ID1",
|
||||||
|
targetField2, "ID1_2",
|
||||||
|
aggName, 42.33
|
||||||
|
),
|
||||||
|
asMap(
|
||||||
|
targetField, "ID1",
|
||||||
|
targetField2, "ID2_2",
|
||||||
|
aggName, 8.4
|
||||||
|
),
|
||||||
|
asMap(
|
||||||
|
targetField, "ID2",
|
||||||
|
targetField2, "ID1_2",
|
||||||
|
aggName, 28.99
|
||||||
|
),
|
||||||
|
asMap(
|
||||||
|
targetField, "ID3",
|
||||||
|
targetField2, "ID2_2",
|
||||||
|
aggName, 12.55
|
||||||
|
)
|
||||||
|
);
|
||||||
|
executeTest(groupBy, aggregationBuilders, input, expected, 10);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testExtractCompositeAggregationResultsMultiAggregations() throws IOException {
|
||||||
|
String targetField = randomAlphaOfLengthBetween(5, 10);
|
||||||
|
|
||||||
|
GroupConfig groupBy = parseGroupConfig("{\"" + targetField + "\" : {"
|
||||||
|
+ "\"terms\" : {"
|
||||||
|
+ " \"field\" : \"doesn't_matter_for_this_test\""
|
||||||
|
+ "} } }");
|
||||||
|
|
||||||
|
String aggName = randomAlphaOfLengthBetween(5, 10);
|
||||||
|
String aggTypedName = "avg#" + aggName;
|
||||||
|
|
||||||
|
String aggName2 = randomAlphaOfLengthBetween(5, 10) + "_2";
|
||||||
|
String aggTypedName2 = "max#" + aggName2;
|
||||||
|
|
||||||
|
Collection<AggregationBuilder> aggregationBuilders = asList(AggregationBuilders.avg(aggName), AggregationBuilders.max(aggName2));
|
||||||
|
|
||||||
|
Map<String, Object> input = asMap(
|
||||||
|
"buckets",
|
||||||
|
asList(
|
||||||
|
asMap(
|
||||||
|
KEY, asMap(
|
||||||
|
targetField, "ID1"),
|
||||||
|
aggTypedName, asMap(
|
||||||
|
"value", 42.33),
|
||||||
|
aggTypedName2, asMap(
|
||||||
|
"value", 9.9),
|
||||||
|
DOC_COUNT, 111),
|
||||||
|
asMap(
|
||||||
|
KEY, asMap(
|
||||||
|
targetField, "ID2"),
|
||||||
|
aggTypedName, asMap(
|
||||||
|
"value", 28.99),
|
||||||
|
aggTypedName2, asMap(
|
||||||
|
"value", 222.33),
|
||||||
|
DOC_COUNT, 88),
|
||||||
|
asMap(
|
||||||
|
KEY, asMap(
|
||||||
|
targetField, "ID3"),
|
||||||
|
aggTypedName, asMap(
|
||||||
|
"value", 12.55),
|
||||||
|
aggTypedName2, asMap(
|
||||||
|
"value", -2.44),
|
||||||
|
DOC_COUNT, 1)
|
||||||
|
));
|
||||||
|
|
||||||
|
List<Map<String, Object>> expected = asList(
|
||||||
|
asMap(
|
||||||
|
targetField, "ID1",
|
||||||
|
aggName, 42.33,
|
||||||
|
aggName2, 9.9
|
||||||
|
),
|
||||||
|
asMap(
|
||||||
|
targetField, "ID2",
|
||||||
|
aggName, 28.99,
|
||||||
|
aggName2, 222.33
|
||||||
|
),
|
||||||
|
asMap(
|
||||||
|
targetField, "ID3",
|
||||||
|
aggName, 12.55,
|
||||||
|
aggName2, -2.44
|
||||||
|
)
|
||||||
|
);
|
||||||
|
executeTest(groupBy, aggregationBuilders, input, expected, 200);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void executeTest(GroupConfig groups, Collection<AggregationBuilder> aggregationBuilders, Map<String, Object> input,
|
||||||
|
List<Map<String, Object>> expected, long expectedDocCounts) throws IOException {
|
||||||
|
DataFrameIndexerTransformStats stats = new DataFrameIndexerTransformStats();
|
||||||
|
XContentBuilder builder = XContentFactory.contentBuilder(randomFrom(XContentType.values()));
|
||||||
|
builder.map(input);
|
||||||
|
|
||||||
|
try (XContentParser parser = createParser(builder)) {
|
||||||
|
CompositeAggregation agg = ParsedComposite.fromXContent(parser, "my_feature");
|
||||||
|
List<Map<String, Object>> result = AggregationResultUtils
|
||||||
|
.extractCompositeAggregationResults(agg, groups, aggregationBuilders, stats).collect(Collectors.toList());
|
||||||
|
|
||||||
|
assertEquals(expected, result);
|
||||||
|
assertEquals(expectedDocCounts, stats.getNumDocuments());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private GroupConfig parseGroupConfig(String json) throws IOException {
|
||||||
|
final XContentParser parser = XContentType.JSON.xContent().createParser(xContentRegistry(),
|
||||||
|
DeprecationHandler.THROW_UNSUPPORTED_OPERATION, json);
|
||||||
|
return GroupConfig.fromXContent(parser, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
static Map<String, Object> asMap(Object... fields) {
|
||||||
|
assert fields.length % 2 == 0;
|
||||||
|
final Map<String, Object> map = new HashMap<>();
|
||||||
|
for (int i = 0; i < fields.length; i += 2) {
|
||||||
|
String field = (String) fields[i];
|
||||||
|
map.put(field, fields[i + 1]);
|
||||||
|
}
|
||||||
|
return map;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,23 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.elasticsearch.test.ESTestCase;
|
||||||
|
|
||||||
|
public class AggregationsTests extends ESTestCase {
|
||||||
|
public void testResolveTargetMapping() {
|
||||||
|
|
||||||
|
// avg
|
||||||
|
assertEquals("double", Aggregations.resolveTargetMapping("avg", "int"));
|
||||||
|
assertEquals("double", Aggregations.resolveTargetMapping("avg", "double"));
|
||||||
|
|
||||||
|
// max
|
||||||
|
assertEquals("int", Aggregations.resolveTargetMapping("max", "int"));
|
||||||
|
assertEquals("double", Aggregations.resolveTargetMapping("max", "double"));
|
||||||
|
assertEquals("half_float", Aggregations.resolveTargetMapping("max", "half_float"));
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,51 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable.Reader;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
|
||||||
|
import org.elasticsearch.test.AbstractSerializingTestCase;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public class DateHistogramGroupSourceTests extends AbstractSerializingTestCase<DateHistogramGroupSource> {
|
||||||
|
|
||||||
|
public static DateHistogramGroupSource randomDateHistogramGroupSource() {
|
||||||
|
String field = randomAlphaOfLengthBetween(1, 20);
|
||||||
|
DateHistogramGroupSource dateHistogramGroupSource = new DateHistogramGroupSource(field);
|
||||||
|
if (randomBoolean()) {
|
||||||
|
dateHistogramGroupSource.setInterval(randomLongBetween(1, 10_000));
|
||||||
|
} else {
|
||||||
|
dateHistogramGroupSource.setDateHistogramInterval(randomFrom(DateHistogramInterval.days(10),
|
||||||
|
DateHistogramInterval.minutes(1), DateHistogramInterval.weeks(1)));
|
||||||
|
}
|
||||||
|
if (randomBoolean()) {
|
||||||
|
dateHistogramGroupSource.setTimeZone(randomZone());
|
||||||
|
}
|
||||||
|
if (randomBoolean()) {
|
||||||
|
dateHistogramGroupSource.setFormat(randomAlphaOfLength(10));
|
||||||
|
}
|
||||||
|
return dateHistogramGroupSource;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected DateHistogramGroupSource doParseInstance(XContentParser parser) throws IOException {
|
||||||
|
return DateHistogramGroupSource.fromXContent(parser, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected DateHistogramGroupSource createTestInstance() {
|
||||||
|
return randomDateHistogramGroupSource();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Reader<DateHistogramGroupSource> instanceReader() {
|
||||||
|
return DateHistogramGroupSource::new;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,101 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable.Reader;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
||||||
|
import org.elasticsearch.test.AbstractSerializingTestCase;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.pivot.SingleGroupSource.Type;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
public class GroupConfigTests extends AbstractSerializingTestCase<GroupConfig> {
|
||||||
|
|
||||||
|
public static GroupConfig randomGroupConfig() {
|
||||||
|
Map<String, Object> source = new LinkedHashMap<>();
|
||||||
|
Map<String, SingleGroupSource<?>> groups = new LinkedHashMap<>();
|
||||||
|
|
||||||
|
// ensure that the unlikely does not happen: 2 group_by's share the same name
|
||||||
|
Set<String> names = new HashSet<>();
|
||||||
|
for (int i = 0; i < randomIntBetween(1, 20); ++i) {
|
||||||
|
String targetFieldName = randomAlphaOfLengthBetween(1, 20);
|
||||||
|
if (names.add(targetFieldName)) {
|
||||||
|
SingleGroupSource<?> groupBy;
|
||||||
|
Type type = randomFrom(SingleGroupSource.Type.values());
|
||||||
|
switch (type) {
|
||||||
|
case TERMS:
|
||||||
|
groupBy = TermsGroupSourceTests.randomTermsGroupSource();
|
||||||
|
break;
|
||||||
|
case HISTOGRAM:
|
||||||
|
groupBy = HistogramGroupSourceTests.randomHistogramGroupSource();
|
||||||
|
break;
|
||||||
|
case DATE_HISTOGRAM:
|
||||||
|
default:
|
||||||
|
groupBy = DateHistogramGroupSourceTests.randomDateHistogramGroupSource();
|
||||||
|
}
|
||||||
|
|
||||||
|
source.put(targetFieldName, Collections.singletonMap(type.value(), getSource(groupBy)));
|
||||||
|
groups.put(targetFieldName, groupBy);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return new GroupConfig(source, groups);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected GroupConfig doParseInstance(XContentParser parser) throws IOException {
|
||||||
|
return GroupConfig.fromXContent(parser, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected GroupConfig createTestInstance() {
|
||||||
|
return randomGroupConfig();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Reader<GroupConfig> instanceReader() {
|
||||||
|
return GroupConfig::new;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testEmptyGroupBy() throws IOException {
|
||||||
|
String source = "{}";
|
||||||
|
|
||||||
|
// lenient, passes but reports invalid
|
||||||
|
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
|
||||||
|
GroupConfig groupConfig = GroupConfig.fromXContent(parser, true);
|
||||||
|
assertFalse(groupConfig.isValid());
|
||||||
|
}
|
||||||
|
|
||||||
|
// strict throws
|
||||||
|
try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
|
||||||
|
expectThrows(IllegalArgumentException.class, () -> GroupConfig.fromXContent(parser, false));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Map<String, Object> getSource(SingleGroupSource<?> groupSource) {
|
||||||
|
try (XContentBuilder xContentBuilder = XContentFactory.jsonBuilder()) {
|
||||||
|
XContentBuilder content = groupSource.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS);
|
||||||
|
return XContentHelper.convertToMap(BytesReference.bytes(content), true, XContentType.JSON).v2();
|
||||||
|
} catch (IOException e) {
|
||||||
|
// should not happen
|
||||||
|
fail("failed to create random single group source");
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,38 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable.Reader;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.test.AbstractSerializingTestCase;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public class HistogramGroupSourceTests extends AbstractSerializingTestCase<HistogramGroupSource> {
|
||||||
|
|
||||||
|
public static HistogramGroupSource randomHistogramGroupSource() {
|
||||||
|
String field = randomAlphaOfLengthBetween(1, 20);
|
||||||
|
double interval = randomDoubleBetween(Math.nextUp(0), Double.MAX_VALUE, false);
|
||||||
|
return new HistogramGroupSource(field, interval);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected HistogramGroupSource doParseInstance(XContentParser parser) throws IOException {
|
||||||
|
return HistogramGroupSource.fromXContent(parser, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected HistogramGroupSource createTestInstance() {
|
||||||
|
return randomHistogramGroupSource();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Reader<HistogramGroupSource> instanceReader() {
|
||||||
|
return HistogramGroupSource::new;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,144 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable.Reader;
|
||||||
|
import org.elasticsearch.common.xcontent.DeprecationHandler;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.AbstractSerializingDataFrameTestCase;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public class PivotConfigTests extends AbstractSerializingDataFrameTestCase<PivotConfig> {
|
||||||
|
|
||||||
|
public static PivotConfig randomPivotConfig() {
|
||||||
|
return new PivotConfig(GroupConfigTests.randomGroupConfig(), AggregationConfigTests.randomAggregationConfig());
|
||||||
|
}
|
||||||
|
|
||||||
|
public static PivotConfig randomInvalidPivotConfig() {
|
||||||
|
return new PivotConfig(GroupConfigTests.randomGroupConfig(), AggregationConfigTests.randomInvalidAggregationConfig());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected PivotConfig doParseInstance(XContentParser parser) throws IOException {
|
||||||
|
return PivotConfig.fromXContent(parser, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected PivotConfig createTestInstance() {
|
||||||
|
return randomPivotConfig();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Reader<PivotConfig> instanceReader() {
|
||||||
|
return PivotConfig::new;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testAggsAbbreviations() throws IOException {
|
||||||
|
String pivotAggs = "{"
|
||||||
|
+ " \"group_by\": {"
|
||||||
|
+ " \"id\": {"
|
||||||
|
+ " \"terms\": {"
|
||||||
|
+ " \"field\": \"id\""
|
||||||
|
+ "} } },"
|
||||||
|
+ " \"aggs\": {"
|
||||||
|
+ " \"avg\": {"
|
||||||
|
+ " \"avg\": {"
|
||||||
|
+ " \"field\": \"points\""
|
||||||
|
+ "} } } }";
|
||||||
|
|
||||||
|
PivotConfig p1 = createPivotConfigFromString(pivotAggs, false);
|
||||||
|
String pivotAggregations = pivotAggs.replace("aggs", "aggregations");
|
||||||
|
assertNotEquals(pivotAggs, pivotAggregations);
|
||||||
|
PivotConfig p2 = createPivotConfigFromString(pivotAggregations, false);
|
||||||
|
assertEquals(p1,p2);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testMissingAggs() throws IOException {
|
||||||
|
String pivot = "{"
|
||||||
|
+ " \"group_by\": {"
|
||||||
|
+ " \"id\": {"
|
||||||
|
+ " \"terms\": {"
|
||||||
|
+ " \"field\": \"id\""
|
||||||
|
+ "} } } }";
|
||||||
|
|
||||||
|
expectThrows(IllegalArgumentException.class, () -> createPivotConfigFromString(pivot, false));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testEmptyAggs() throws IOException {
|
||||||
|
String pivot = "{"
|
||||||
|
+ " \"group_by\": {"
|
||||||
|
+ " \"id\": {"
|
||||||
|
+ " \"terms\": {"
|
||||||
|
+ " \"field\": \"id\""
|
||||||
|
+ "} } },"
|
||||||
|
+ "\"aggs\": {}"
|
||||||
|
+ " }";
|
||||||
|
|
||||||
|
expectThrows(IllegalArgumentException.class, () -> createPivotConfigFromString(pivot, false));
|
||||||
|
|
||||||
|
// lenient passes but reports invalid
|
||||||
|
PivotConfig pivotConfig = createPivotConfigFromString(pivot, true);
|
||||||
|
assertFalse(pivotConfig.isValid());
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testEmptyGroupBy() throws IOException {
|
||||||
|
String pivot = "{"
|
||||||
|
+ " \"group_by\": {},"
|
||||||
|
+ " \"aggs\": {"
|
||||||
|
+ " \"avg\": {"
|
||||||
|
+ " \"avg\": {"
|
||||||
|
+ " \"field\": \"points\""
|
||||||
|
+ "} } } }";
|
||||||
|
|
||||||
|
expectThrows(IllegalArgumentException.class, () -> createPivotConfigFromString(pivot, false));
|
||||||
|
|
||||||
|
// lenient passes but reports invalid
|
||||||
|
PivotConfig pivotConfig = createPivotConfigFromString(pivot, true);
|
||||||
|
assertFalse(pivotConfig.isValid());
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testMissingGroupBy() throws IOException {
|
||||||
|
String pivot = "{"
|
||||||
|
+ " \"aggs\": {"
|
||||||
|
+ " \"avg\": {"
|
||||||
|
+ " \"avg\": {"
|
||||||
|
+ " \"field\": \"points\""
|
||||||
|
+ "} } } }";
|
||||||
|
|
||||||
|
expectThrows(IllegalArgumentException.class, () -> createPivotConfigFromString(pivot, false));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testDoubleAggs() throws IOException {
|
||||||
|
String pivot = "{"
|
||||||
|
+ " \"group_by\": {"
|
||||||
|
+ " \"id\": {"
|
||||||
|
+ " \"terms\": {"
|
||||||
|
+ " \"field\": \"id\""
|
||||||
|
+ "} } },"
|
||||||
|
+ " \"aggs\": {"
|
||||||
|
+ " \"avg\": {"
|
||||||
|
+ " \"avg\": {"
|
||||||
|
+ " \"field\": \"points\""
|
||||||
|
+ "} } },"
|
||||||
|
+ " \"aggregations\": {"
|
||||||
|
+ " \"avg\": {"
|
||||||
|
+ " \"avg\": {"
|
||||||
|
+ " \"field\": \"points\""
|
||||||
|
+ "} } }"
|
||||||
|
+ "}";
|
||||||
|
|
||||||
|
expectThrows(IllegalArgumentException.class, () -> createPivotConfigFromString(pivot, false));
|
||||||
|
}
|
||||||
|
|
||||||
|
private PivotConfig createPivotConfigFromString(String json, boolean lenient) throws IOException {
|
||||||
|
final XContentParser parser = XContentType.JSON.xContent().createParser(xContentRegistry(),
|
||||||
|
DeprecationHandler.THROW_UNSUPPORTED_OPERATION, json);
|
||||||
|
return PivotConfig.fromXContent(parser, lenient);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,212 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.apache.lucene.search.TotalHits;
|
||||||
|
import org.elasticsearch.action.Action;
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.ActionRequest;
|
||||||
|
import org.elasticsearch.action.ActionResponse;
|
||||||
|
import org.elasticsearch.action.search.SearchRequest;
|
||||||
|
import org.elasticsearch.action.search.SearchResponse;
|
||||||
|
import org.elasticsearch.action.search.SearchResponseSections;
|
||||||
|
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||||
|
import org.elasticsearch.client.Client;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.common.xcontent.DeprecationHandler;
|
||||||
|
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.index.IndexNotFoundException;
|
||||||
|
import org.elasticsearch.index.query.MatchAllQueryBuilder;
|
||||||
|
import org.elasticsearch.search.SearchHit;
|
||||||
|
import org.elasticsearch.search.SearchHits;
|
||||||
|
import org.elasticsearch.search.SearchModule;
|
||||||
|
import org.elasticsearch.test.ESTestCase;
|
||||||
|
import org.elasticsearch.test.client.NoOpClient;
|
||||||
|
import org.elasticsearch.xpack.dataframe.transforms.pivot.Aggregations.AggregationType;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Before;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
import static java.util.Collections.emptyList;
|
||||||
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
|
||||||
|
public class PivotTests extends ESTestCase {
|
||||||
|
|
||||||
|
private NamedXContentRegistry namedXContentRegistry;
|
||||||
|
private Client client;
|
||||||
|
|
||||||
|
private final Set<String> supportedAggregations = Stream.of(AggregationType.values()).map(AggregationType::getName)
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
private final String[] unsupportedAggregations = { "stats" };
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void registerAggregationNamedObjects() throws Exception {
|
||||||
|
// register aggregations as NamedWriteable
|
||||||
|
SearchModule searchModule = new SearchModule(Settings.EMPTY, false, emptyList());
|
||||||
|
namedXContentRegistry = new NamedXContentRegistry(searchModule.getNamedXContents());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setupClient() {
|
||||||
|
if (client != null) {
|
||||||
|
client.close();
|
||||||
|
}
|
||||||
|
client = new MyMockClient(getTestName());
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDownClient() {
|
||||||
|
client.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected NamedXContentRegistry xContentRegistry() {
|
||||||
|
return namedXContentRegistry;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testValidateExistingIndex() throws Exception {
|
||||||
|
Pivot pivot = new Pivot("existing_source_index", new MatchAllQueryBuilder(), getValidPivotConfig());
|
||||||
|
|
||||||
|
assertValidTransform(client, pivot);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testValidateNonExistingIndex() throws Exception {
|
||||||
|
Pivot pivot = new Pivot("non_existing_source_index", new MatchAllQueryBuilder(), getValidPivotConfig());
|
||||||
|
|
||||||
|
assertInvalidTransform(client, pivot);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testSearchFailure() throws Exception {
|
||||||
|
// test a failure during the search operation, transform creation fails if
|
||||||
|
// search has failures although they might just be temporary
|
||||||
|
Pivot pivot = new Pivot("existing_source_index_with_failing_shards", new MatchAllQueryBuilder(), getValidPivotConfig());
|
||||||
|
|
||||||
|
assertInvalidTransform(client, pivot);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testValidateAllSupportedAggregations() throws Exception {
|
||||||
|
for (String agg : supportedAggregations) {
|
||||||
|
AggregationConfig aggregationConfig = getAggregationConfig(agg);
|
||||||
|
|
||||||
|
Pivot pivot = new Pivot("existing_source", new MatchAllQueryBuilder(), getValidPivotConfig(aggregationConfig));
|
||||||
|
|
||||||
|
assertValidTransform(client, pivot);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testValidateAllUnsupportedAggregations() throws Exception {
|
||||||
|
for (String agg : unsupportedAggregations) {
|
||||||
|
AggregationConfig aggregationConfig = getAggregationConfig(agg);
|
||||||
|
|
||||||
|
Pivot pivot = new Pivot("existing_source", new MatchAllQueryBuilder(), getValidPivotConfig(aggregationConfig));
|
||||||
|
|
||||||
|
assertInvalidTransform(client, pivot);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private class MyMockClient extends NoOpClient {
|
||||||
|
MyMockClient(String testName) {
|
||||||
|
super(testName);
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
@Override
|
||||||
|
protected <Request extends ActionRequest, Response extends ActionResponse> void doExecute(Action<Response> action, Request request,
|
||||||
|
ActionListener<Response> listener) {
|
||||||
|
|
||||||
|
if (request instanceof SearchRequest) {
|
||||||
|
SearchRequest searchRequest = (SearchRequest) request;
|
||||||
|
List<ShardSearchFailure> searchFailures = new ArrayList<>();
|
||||||
|
|
||||||
|
for (String index : searchRequest.indices()) {
|
||||||
|
if (index.contains("non_existing")) {
|
||||||
|
listener.onFailure(new IndexNotFoundException(index));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (index.contains("with_failing_shards")) {
|
||||||
|
searchFailures.add(new ShardSearchFailure(new RuntimeException("shard failed")));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
final SearchResponseSections sections = new SearchResponseSections(
|
||||||
|
new SearchHits(new SearchHit[0], new TotalHits(0L, TotalHits.Relation.EQUAL_TO), 0), null, null, false, null, null,
|
||||||
|
1);
|
||||||
|
final SearchResponse response = new SearchResponse(sections, null, 10, searchFailures.size() > 0 ? 0 : 5, 0, 0,
|
||||||
|
searchFailures.toArray(new ShardSearchFailure[searchFailures.size()]), null);
|
||||||
|
|
||||||
|
listener.onResponse((Response) response);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
super.doExecute(action, request, listener);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private PivotConfig getValidPivotConfig() throws IOException {
|
||||||
|
return new PivotConfig(GroupConfigTests.randomGroupConfig(), getValidAggregationConfig());
|
||||||
|
}
|
||||||
|
|
||||||
|
private PivotConfig getValidPivotConfig(AggregationConfig aggregationConfig) throws IOException {
|
||||||
|
return new PivotConfig(GroupConfigTests.randomGroupConfig(), aggregationConfig);
|
||||||
|
}
|
||||||
|
|
||||||
|
private AggregationConfig getValidAggregationConfig() throws IOException {
|
||||||
|
return getAggregationConfig(randomFrom(supportedAggregations));
|
||||||
|
}
|
||||||
|
|
||||||
|
private AggregationConfig getAggregationConfig(String agg) throws IOException {
|
||||||
|
return parseAggregations("{\n" + " \"pivot_" + agg + "\": {\n" + " \"" + agg + "\": {\n" + " \"field\": \"values\"\n"
|
||||||
|
+ " }\n" + " }" + "}");
|
||||||
|
}
|
||||||
|
|
||||||
|
private AggregationConfig parseAggregations(String json) throws IOException {
|
||||||
|
final XContentParser parser = XContentType.JSON.xContent().createParser(xContentRegistry(),
|
||||||
|
DeprecationHandler.THROW_UNSUPPORTED_OPERATION, json);
|
||||||
|
// parseAggregators expects to be already inside the xcontent object
|
||||||
|
assertThat(parser.nextToken(), equalTo(XContentParser.Token.START_OBJECT));
|
||||||
|
return AggregationConfig.fromXContent(parser, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void assertValidTransform(Client client, Pivot pivot) throws Exception {
|
||||||
|
validate(client, pivot, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void assertInvalidTransform(Client client, Pivot pivot) throws Exception {
|
||||||
|
validate(client, pivot, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void validate(Client client, Pivot pivot, boolean expectValid) throws Exception {
|
||||||
|
CountDownLatch latch = new CountDownLatch(1);
|
||||||
|
final AtomicReference<Exception> exceptionHolder = new AtomicReference<>();
|
||||||
|
pivot.validate(client, ActionListener.wrap(validity -> {
|
||||||
|
assertEquals(expectValid, validity);
|
||||||
|
latch.countDown();
|
||||||
|
}, e -> {
|
||||||
|
exceptionHolder.set(e);
|
||||||
|
latch.countDown();
|
||||||
|
}));
|
||||||
|
|
||||||
|
assertTrue(latch.await(100, TimeUnit.MILLISECONDS));
|
||||||
|
if (expectValid == true && exceptionHolder.get() != null) {
|
||||||
|
throw exceptionHolder.get();
|
||||||
|
} else if (expectValid == false && exceptionHolder.get() == null) {
|
||||||
|
fail("Expected config to be invalid");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,38 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.xpack.dataframe.transforms.pivot;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable.Reader;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.test.AbstractSerializingTestCase;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public class TermsGroupSourceTests extends AbstractSerializingTestCase<TermsGroupSource> {
|
||||||
|
|
||||||
|
public static TermsGroupSource randomTermsGroupSource() {
|
||||||
|
String field = randomAlphaOfLengthBetween(1, 20);
|
||||||
|
|
||||||
|
return new TermsGroupSource(field);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected TermsGroupSource doParseInstance(XContentParser parser) throws IOException {
|
||||||
|
return TermsGroupSource.fromXContent(parser, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected TermsGroupSource createTestInstance() {
|
||||||
|
return randomTermsGroupSource();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Reader<TermsGroupSource> instanceReader() {
|
||||||
|
return TermsGroupSource::new;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -19,6 +19,7 @@ import java.util.function.Consumer;
|
||||||
import java.util.function.Predicate;
|
import java.util.function.Predicate;
|
||||||
|
|
||||||
import static org.elasticsearch.action.admin.cluster.node.tasks.get.GetTaskAction.TASKS_ORIGIN;
|
import static org.elasticsearch.action.admin.cluster.node.tasks.get.GetTaskAction.TASKS_ORIGIN;
|
||||||
|
import static org.elasticsearch.xpack.core.ClientHelper.DATA_FRAME_ORIGIN;
|
||||||
import static org.elasticsearch.xpack.core.ClientHelper.DEPRECATION_ORIGIN;
|
import static org.elasticsearch.xpack.core.ClientHelper.DEPRECATION_ORIGIN;
|
||||||
import static org.elasticsearch.xpack.core.ClientHelper.INDEX_LIFECYCLE_ORIGIN;
|
import static org.elasticsearch.xpack.core.ClientHelper.INDEX_LIFECYCLE_ORIGIN;
|
||||||
import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN;
|
import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN;
|
||||||
|
@ -105,6 +106,7 @@ public final class AuthorizationUtils {
|
||||||
case WATCHER_ORIGIN:
|
case WATCHER_ORIGIN:
|
||||||
case ML_ORIGIN:
|
case ML_ORIGIN:
|
||||||
case MONITORING_ORIGIN:
|
case MONITORING_ORIGIN:
|
||||||
|
case DATA_FRAME_ORIGIN:
|
||||||
case DEPRECATION_ORIGIN:
|
case DEPRECATION_ORIGIN:
|
||||||
case PERSISTENT_TASK_ORIGIN:
|
case PERSISTENT_TASK_ORIGIN:
|
||||||
case ROLLUP_ORIGIN:
|
case ROLLUP_ORIGIN:
|
||||||
|
|
Loading…
Add table
Add a link
Reference in a new issue